@@ -29,107 +29,109 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes
29
29
import testImplicits ._
30
30
31
31
test(" Paimon Procedure: create and delete tag" ) {
32
- failAfter(streamingTimeout) {
33
- withTempDir {
34
- checkpointDir =>
35
- // define a pk table and test `forEachBatch` api
36
- spark.sql(s """
37
- |CREATE TABLE T (a INT, b STRING)
38
- |TBLPROPERTIES ('primary-key'='a', 'bucket'='3')
39
- | """ .stripMargin)
40
- val location = loadTable(" T" ).location().toString
41
-
42
- val inputData = MemoryStream [(Int , String )]
43
- val stream = inputData
44
- .toDS()
45
- .toDF(" a" , " b" )
46
- .writeStream
47
- .option(" checkpointLocation" , checkpointDir.getCanonicalPath)
48
- .foreachBatch {
49
- (batch : Dataset [Row ], _ : Long ) =>
50
- batch.write.format(" paimon" ).mode(" append" ).save(location)
32
+ withSparkSQLConf((" spark.paimon.write.use-v2-write" , " false" )) {
33
+ failAfter(streamingTimeout) {
34
+ withTempDir {
35
+ checkpointDir =>
36
+ // define a pk table and test `forEachBatch` api
37
+ spark.sql(s """
38
+ |CREATE TABLE T (a INT, b STRING)
39
+ |TBLPROPERTIES ('primary-key'='a', 'bucket'='3')
40
+ | """ .stripMargin)
41
+ val location = loadTable(" T" ).location().toString
42
+
43
+ val inputData = MemoryStream [(Int , String )]
44
+ val stream = inputData
45
+ .toDS()
46
+ .toDF(" a" , " b" )
47
+ .writeStream
48
+ .option(" checkpointLocation" , checkpointDir.getCanonicalPath)
49
+ .foreachBatch {
50
+ (batch : Dataset [Row ], _ : Long ) =>
51
+ batch.write.format(" paimon" ).mode(" append" ).save(location)
52
+ }
53
+ .start()
54
+
55
+ val query = () => spark.sql(" SELECT * FROM T ORDER BY a" )
56
+
57
+ try {
58
+ // snapshot-1
59
+ inputData.addData((1 , " a" ))
60
+ stream.processAllAvailable()
61
+ checkAnswer(query(), Row (1 , " a" ) :: Nil )
62
+
63
+ // snapshot-2
64
+ inputData.addData((2 , " b" ))
65
+ stream.processAllAvailable()
66
+ checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " b" ) :: Nil )
67
+
68
+ // snapshot-3
69
+ inputData.addData((2 , " b2" ))
70
+ stream.processAllAvailable()
71
+ checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " b2" ) :: Nil )
72
+ checkAnswer(
73
+ spark.sql(
74
+ " CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag', snapshot => 2)" ),
75
+ Row (true ) :: Nil )
76
+ checkAnswer(
77
+ spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
78
+ Row (" test_tag" ) :: Nil )
79
+ // test rename_tag
80
+ checkAnswer(
81
+ spark.sql(
82
+ " CALL paimon.sys.rename_tag(table => 'test.T', tag => 'test_tag', target_tag => 'test_tag_1')" ),
83
+ Row (true ) :: Nil )
84
+ checkAnswer(
85
+ spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
86
+ Row (" test_tag_1" ) :: Nil )
87
+ checkAnswer(
88
+ spark.sql(" CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag_1')" ),
89
+ Row (true ) :: Nil )
90
+ checkAnswer(spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ), Nil )
91
+ checkAnswer(
92
+ spark.sql(
93
+ " CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_latestSnapshot_tag')" ),
94
+ Row (true ) :: Nil )
95
+ checkAnswer(
96
+ spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
97
+ Row (" test_latestSnapshot_tag" ) :: Nil )
98
+ checkAnswer(
99
+ spark.sql(
100
+ " CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_latestSnapshot_tag')" ),
101
+ Row (true ) :: Nil )
102
+ checkAnswer(spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ), Nil )
103
+
104
+ // snapshot-4
105
+ inputData.addData((2 , " c1" ))
106
+ stream.processAllAvailable()
107
+ checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " c1" ) :: Nil )
108
+
109
+ checkAnswer(
110
+ spark.sql(" CALL paimon.sys.create_tag(table => 'test.T', tag => 's4')" ),
111
+ Row (true ) :: Nil )
112
+
113
+ // snapshot-5
114
+ inputData.addData((3 , " c2" ))
115
+ stream.processAllAvailable()
116
+ checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " c1" ) :: Row (3 , " c2" ) :: Nil )
117
+
118
+ checkAnswer(
119
+ spark.sql(" CALL paimon.sys.create_tag(table => 'test.T', tag => 's5')" ),
120
+ Row (true ) :: Nil )
121
+
122
+ checkAnswer(
123
+ spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
124
+ Row (" s4" ) :: Row (" s5" ) :: Nil )
125
+
126
+ checkAnswer(
127
+ spark.sql(" CALL paimon.sys.delete_tag(table => 'test.T', tag => 's4,s5')" ),
128
+ Row (true ) :: Nil )
129
+
130
+ checkAnswer(spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ), Nil )
131
+ } finally {
132
+ stream.stop()
51
133
}
52
- .start()
53
-
54
- val query = () => spark.sql(" SELECT * FROM T ORDER BY a" )
55
-
56
- try {
57
- // snapshot-1
58
- inputData.addData((1 , " a" ))
59
- stream.processAllAvailable()
60
- checkAnswer(query(), Row (1 , " a" ) :: Nil )
61
-
62
- // snapshot-2
63
- inputData.addData((2 , " b" ))
64
- stream.processAllAvailable()
65
- checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " b" ) :: Nil )
66
-
67
- // snapshot-3
68
- inputData.addData((2 , " b2" ))
69
- stream.processAllAvailable()
70
- checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " b2" ) :: Nil )
71
- checkAnswer(
72
- spark.sql(
73
- " CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag', snapshot => 2)" ),
74
- Row (true ) :: Nil )
75
- checkAnswer(
76
- spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
77
- Row (" test_tag" ) :: Nil )
78
- // test rename_tag
79
- checkAnswer(
80
- spark.sql(
81
- " CALL paimon.sys.rename_tag(table => 'test.T', tag => 'test_tag', target_tag => 'test_tag_1')" ),
82
- Row (true ) :: Nil )
83
- checkAnswer(
84
- spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
85
- Row (" test_tag_1" ) :: Nil )
86
- checkAnswer(
87
- spark.sql(" CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag_1')" ),
88
- Row (true ) :: Nil )
89
- checkAnswer(spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ), Nil )
90
- checkAnswer(
91
- spark.sql(
92
- " CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_latestSnapshot_tag')" ),
93
- Row (true ) :: Nil )
94
- checkAnswer(
95
- spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
96
- Row (" test_latestSnapshot_tag" ) :: Nil )
97
- checkAnswer(
98
- spark.sql(
99
- " CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_latestSnapshot_tag')" ),
100
- Row (true ) :: Nil )
101
- checkAnswer(spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ), Nil )
102
-
103
- // snapshot-4
104
- inputData.addData((2 , " c1" ))
105
- stream.processAllAvailable()
106
- checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " c1" ) :: Nil )
107
-
108
- checkAnswer(
109
- spark.sql(" CALL paimon.sys.create_tag(table => 'test.T', tag => 's4')" ),
110
- Row (true ) :: Nil )
111
-
112
- // snapshot-5
113
- inputData.addData((3 , " c2" ))
114
- stream.processAllAvailable()
115
- checkAnswer(query(), Row (1 , " a" ) :: Row (2 , " c1" ) :: Row (3 , " c2" ) :: Nil )
116
-
117
- checkAnswer(
118
- spark.sql(" CALL paimon.sys.create_tag(table => 'test.T', tag => 's5')" ),
119
- Row (true ) :: Nil )
120
-
121
- checkAnswer(
122
- spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ),
123
- Row (" s4" ) :: Row (" s5" ) :: Nil )
124
-
125
- checkAnswer(
126
- spark.sql(" CALL paimon.sys.delete_tag(table => 'test.T', tag => 's4,s5')" ),
127
- Row (true ) :: Nil )
128
-
129
- checkAnswer(spark.sql(" SELECT tag_name FROM paimon.test.`T$tags`" ), Nil )
130
- } finally {
131
- stream.stop()
132
- }
134
+ }
133
135
}
134
136
}
135
137
}
0 commit comments