19
19
package org .apache .iceberg .flink .source ;
20
20
21
21
import static org .apache .iceberg .types .Types .NestedField .required ;
22
+ import static org .assertj .core .api .Assertions .assertThatThrownBy ;
22
23
23
24
import java .io .IOException ;
24
25
import java .time .Instant ;
40
41
import org .apache .iceberg .relocated .com .google .common .collect .ImmutableMap ;
41
42
import org .apache .iceberg .relocated .com .google .common .collect .Lists ;
42
43
import org .apache .iceberg .types .Types ;
44
+ import org .junit .jupiter .api .AfterEach ;
43
45
import org .junit .jupiter .api .BeforeEach ;
44
46
import org .junit .jupiter .api .Test ;
45
47
@@ -53,7 +55,11 @@ public class TestIcebergSourceSql extends TestSqlBase {
53
55
@ BeforeEach
54
56
@ Override
55
57
public void before () throws IOException {
56
- TableEnvironment tableEnvironment = getTableEnv ();
58
+ setUpTableEnv (getTableEnv ());
59
+ setUpTableEnv (getStreamingTableEnv ());
60
+ }
61
+
62
+ private static void setUpTableEnv (TableEnvironment tableEnvironment ) {
57
63
Configuration tableConf = tableEnvironment .getConfig ().getConfiguration ();
58
64
tableConf .set (FlinkConfigOptions .TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE , true );
59
65
// Disable inferring parallelism to avoid interfering watermark tests
@@ -72,6 +78,11 @@ public void before() throws IOException {
72
78
tableConf .set (TableConfigOptions .TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED , true );
73
79
}
74
80
81
+ @ AfterEach
82
+ public void after () throws IOException {
83
+ CATALOG_EXTENSION .catalog ().dropTable (TestFixtures .TABLE_IDENTIFIER );
84
+ }
85
+
75
86
private Record generateRecord (Instant t1 , long t2 ) {
76
87
Record record = GenericRecord .create (SCHEMA_TS );
77
88
record .setField ("t1" , t1 .atZone (ZoneId .systemDefault ()).toLocalDateTime ());
@@ -178,4 +189,45 @@ public void testReadFlinkDynamicTable() throws Exception {
178
189
expected ,
179
190
SCHEMA_TS );
180
191
}
192
+
193
+ @ Test
194
+ public void testWatermarkInvalidConfig () {
195
+ CATALOG_EXTENSION .catalog ().createTable (TestFixtures .TABLE_IDENTIFIER , SCHEMA_TS );
196
+
197
+ String flinkTable = "`default_catalog`.`default_database`.flink_table" ;
198
+ SqlHelpers .sql (
199
+ getStreamingTableEnv (),
200
+ "CREATE TABLE %s "
201
+ + "(eventTS AS CAST(t1 AS TIMESTAMP(3)), "
202
+ + "WATERMARK FOR eventTS AS SOURCE_WATERMARK()) LIKE iceberg_catalog.`default`.%s" ,
203
+ flinkTable ,
204
+ TestFixtures .TABLE );
205
+
206
+ assertThatThrownBy (() -> SqlHelpers .sql (getStreamingTableEnv (), "SELECT * FROM %s" , flinkTable ))
207
+ .isInstanceOf (NullPointerException .class )
208
+ .hasMessage ("watermark-column needs to be configured to use source watermark." );
209
+ }
210
+
211
+ @ Test
212
+ public void testWatermarkValidConfig () throws Exception {
213
+ List <Record > expected = generateExpectedRecords (true );
214
+
215
+ String flinkTable = "`default_catalog`.`default_database`.flink_table" ;
216
+
217
+ SqlHelpers .sql (
218
+ getStreamingTableEnv (),
219
+ "CREATE TABLE %s "
220
+ + "(eventTS AS CAST(t1 AS TIMESTAMP(3)), "
221
+ + "WATERMARK FOR eventTS AS SOURCE_WATERMARK()) WITH ('watermark-column'='t1') LIKE iceberg_catalog.`default`.%s" ,
222
+ flinkTable ,
223
+ TestFixtures .TABLE );
224
+
225
+ TestHelpers .assertRecordsWithOrder (
226
+ SqlHelpers .sql (
227
+ getStreamingTableEnv (),
228
+ "SELECT t1, t2 FROM TABLE(TUMBLE(TABLE %s, DESCRIPTOR(eventTS), INTERVAL '1' SECOND))" ,
229
+ flinkTable ),
230
+ expected ,
231
+ SCHEMA_TS );
232
+ }
181
233
}
0 commit comments