22
22
import org .apache .flink .table .api .DataTypes ;
23
23
import org .apache .flink .table .api .Schema ;
24
24
import org .apache .flink .table .catalog .CatalogDescriptor ;
25
+ import org .apache .flink .table .catalog .CatalogMaterializedTable ;
25
26
import org .apache .flink .table .catalog .CatalogTable ;
26
27
import org .apache .flink .table .catalog .CatalogView ;
27
28
import org .apache .flink .table .catalog .Column ;
29
+ import org .apache .flink .table .catalog .IntervalFreshness ;
28
30
import org .apache .flink .table .catalog .ObjectIdentifier ;
31
+ import org .apache .flink .table .catalog .ResolvedCatalogMaterializedTable ;
29
32
import org .apache .flink .table .catalog .ResolvedCatalogTable ;
30
33
import org .apache .flink .table .catalog .ResolvedCatalogView ;
31
34
import org .apache .flink .table .catalog .ResolvedSchema ;
32
35
import org .apache .flink .table .catalog .TableDistribution ;
36
+ import org .apache .flink .table .catalog .UniqueConstraint ;
33
37
import org .apache .flink .table .expressions .DefaultSqlFactory ;
34
38
35
39
import org .junit .jupiter .params .ParameterizedTest ;
@@ -61,6 +65,16 @@ public class ShowCreateUtilTest {
61
65
Column .physical ("id" , DataTypes .INT ()),
62
66
Column .physical ("name" , DataTypes .STRING ()));
63
67
68
+ private static final ResolvedSchema THREE_COLUMNS_PK_SCHEMA =
69
+ new ResolvedSchema (
70
+ Arrays .asList (
71
+ Column .physical ("id" , DataTypes .INT ()),
72
+ Column .physical ("name" , DataTypes .STRING ()),
73
+ Column .physical ("birthday" , DataTypes .STRING ())),
74
+ Collections .emptyList (),
75
+ UniqueConstraint .primaryKey ("pk" , Collections .singletonList ("id" )));
76
+ ;
77
+
64
78
@ ParameterizedTest (name = "{index}: {1}" )
65
79
@ MethodSource ("argsForShowCreateTable" )
66
80
void showCreateTable (ResolvedCatalogTable resolvedCatalogTable , String expected ) {
@@ -70,6 +84,16 @@ void showCreateTable(ResolvedCatalogTable resolvedCatalogTable, String expected)
70
84
assertThat (createTableString ).isEqualTo (expected );
71
85
}
72
86
87
+ @ ParameterizedTest (name = "{index}: {1}" )
88
+ @ MethodSource ("argsForShowCreateMaterializedTable" )
89
+ void showCreateMaterializedTable (
90
+ ResolvedCatalogMaterializedTable resolvedCatalogTable , String expected ) {
91
+ final String createTableString =
92
+ ShowCreateUtil .buildShowCreateTableRow (
93
+ resolvedCatalogTable , TABLE_IDENTIFIER , false , DefaultSqlFactory .INSTANCE );
94
+ assertThat (createTableString ).isEqualTo (expected );
95
+ }
96
+
73
97
@ ParameterizedTest (name = "{index}: {1}" )
74
98
@ MethodSource ("argsForShowCreateView" )
75
99
void showCreateView (ResolvedCatalogView resolvedCatalogView , String expected ) {
@@ -241,6 +265,96 @@ private static Collection<Arguments> argsForShowCreateTable() {
241
265
return argList ;
242
266
}
243
267
268
+ private static Collection <Arguments > argsForShowCreateMaterializedTable () {
269
+ Collection <Arguments > argList = new ArrayList <>();
270
+
271
+ final Map <String , String > options = new HashMap <>();
272
+ options .put ("option_key_a" , "option_value_a" );
273
+ options .put ("option_key_b" , "option_value_b" );
274
+ options .put ("option_key_c" , "option_value_c" );
275
+
276
+ argList .add (
277
+ Arguments .of (
278
+ createResolvedCatalogMaterializedTable (
279
+ THREE_COLUMNS_PK_SCHEMA ,
280
+ options ,
281
+ List .of ("birthday" ),
282
+ "table comment" ,
283
+ "( SELECT id, birthday, name\n "
284
+ + " FROM json_source\n "
285
+ + ") AS tmp" ,
286
+ CatalogMaterializedTable .LogicalRefreshMode .CONTINUOUS ,
287
+ CatalogMaterializedTable .RefreshMode .CONTINUOUS ),
288
+ "CREATE MATERIALIZED TABLE `catalogName`.`dbName`.`tableName`\n "
289
+ + "( CONSTRAINT `pk` PRIMARY KEY (`id`) NOT ENFORCED)\n "
290
+ + "COMMENT 'table comment'\n "
291
+ + "PARTITIONED BY (`birthday`)\n "
292
+ + "WITH (\n "
293
+ + " 'option_key_a' = 'option_value_a',\n "
294
+ + " 'option_key_b' = 'option_value_b',\n "
295
+ + " 'option_key_c' = 'option_value_c'\n "
296
+ + ")\n "
297
+ + "FRESHNESS = INTERVAL '5' MINUTE \n "
298
+ + "REFRESH_MODE = CONTINUOUS\n "
299
+ + "AS ( SELECT id, birthday, name\n "
300
+ + " FROM json_source\n "
301
+ + ") AS tmp" ));
302
+
303
+ argList .add (
304
+ Arguments .of (
305
+ createResolvedCatalogMaterializedTable (
306
+ THREE_COLUMNS_PK_SCHEMA ,
307
+ options ,
308
+ List .of ("birthday" ),
309
+ "table comment" ,
310
+ "( SELECT id, birthday, name\n "
311
+ + " FROM json_source\n "
312
+ + ") AS tmp" ,
313
+ CatalogMaterializedTable .LogicalRefreshMode .CONTINUOUS ,
314
+ CatalogMaterializedTable .RefreshMode .FULL ),
315
+ "CREATE MATERIALIZED TABLE `catalogName`.`dbName`.`tableName`\n "
316
+ + "( CONSTRAINT `pk` PRIMARY KEY (`id`) NOT ENFORCED)\n "
317
+ + "COMMENT 'table comment'\n "
318
+ + "PARTITIONED BY (`birthday`)\n "
319
+ + "WITH (\n "
320
+ + " 'option_key_a' = 'option_value_a',\n "
321
+ + " 'option_key_b' = 'option_value_b',\n "
322
+ + " 'option_key_c' = 'option_value_c'\n "
323
+ + ")\n "
324
+ + "FRESHNESS = INTERVAL '5' MINUTE \n "
325
+ + "REFRESH_MODE = FULL\n "
326
+ + "AS ( SELECT id, birthday, name\n "
327
+ + " FROM json_source\n "
328
+ + ") AS tmp" ));
329
+
330
+ argList .add (
331
+ Arguments .of (
332
+ createResolvedCatalogMaterializedTable (
333
+ THREE_COLUMNS_PK_SCHEMA ,
334
+ options ,
335
+ List .of ("birthday" ),
336
+ "table comment" ,
337
+ "( SELECT id, birthday, name\n "
338
+ + " FROM json_source\n "
339
+ + ") AS tmp" ,
340
+ CatalogMaterializedTable .LogicalRefreshMode .AUTOMATIC ,
341
+ CatalogMaterializedTable .RefreshMode .CONTINUOUS ),
342
+ "CREATE MATERIALIZED TABLE `catalogName`.`dbName`.`tableName`\n "
343
+ + "( CONSTRAINT `pk` PRIMARY KEY (`id`) NOT ENFORCED)\n "
344
+ + "COMMENT 'table comment'\n "
345
+ + "PARTITIONED BY (`birthday`)\n "
346
+ + "WITH (\n "
347
+ + " 'option_key_a' = 'option_value_a',\n "
348
+ + " 'option_key_b' = 'option_value_b',\n "
349
+ + " 'option_key_c' = 'option_value_c'\n "
350
+ + ")\n "
351
+ + "FRESHNESS = INTERVAL '5' MINUTE \n "
352
+ + "AS ( SELECT id, birthday, name\n "
353
+ + " FROM json_source\n "
354
+ + ") AS tmp" ));
355
+ return argList ;
356
+ }
357
+
244
358
private static ResolvedCatalogTable createResolvedTable (
245
359
ResolvedSchema resolvedSchema ,
246
360
Map <String , String > options ,
@@ -273,4 +387,29 @@ private static ResolvedCatalogView createResolvedView(
273
387
Collections .emptyMap ()),
274
388
resolvedSchema );
275
389
}
390
+
391
+ private static ResolvedCatalogMaterializedTable createResolvedCatalogMaterializedTable (
392
+ ResolvedSchema resolvedSchema ,
393
+ Map <String , String > options ,
394
+ List <String > partitionKeys ,
395
+ String comment ,
396
+ String query ,
397
+ CatalogMaterializedTable .LogicalRefreshMode logicalRefreshMode ,
398
+ CatalogMaterializedTable .RefreshMode refreshMode ) {
399
+
400
+ CatalogMaterializedTable .Builder materializedTableBuilder =
401
+ CatalogMaterializedTable .newBuilder ()
402
+ .schema (Schema .newBuilder ().fromResolvedSchema (resolvedSchema ).build ())
403
+ .comment (comment )
404
+ .partitionKeys (partitionKeys )
405
+ .options (options )
406
+ .definitionQuery (query )
407
+ .freshness (IntervalFreshness .ofMinute ("5" ))
408
+ .logicalRefreshMode (logicalRefreshMode )
409
+ .refreshMode (refreshMode )
410
+ .refreshStatus (CatalogMaterializedTable .RefreshStatus .INITIALIZING );
411
+
412
+ return new ResolvedCatalogMaterializedTable (
413
+ materializedTableBuilder .build (), resolvedSchema );
414
+ }
276
415
}
0 commit comments