forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
/
SqlDdlToOperationConverterTest.java
2361 lines (2147 loc) · 118 KB
/
SqlDdlToOperationConverterTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.table.planner.operations;
import org.apache.flink.sql.parser.ddl.SqlCreateTable;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.api.SqlDialect;
import org.apache.flink.table.api.TableColumn;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.api.constraints.UniqueConstraint;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.CatalogDatabaseImpl;
import org.apache.flink.table.catalog.CatalogFunction;
import org.apache.flink.table.catalog.CatalogFunctionImpl;
import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.ContextResolvedTable;
import org.apache.flink.table.catalog.FunctionLanguage;
import org.apache.flink.table.catalog.GenericInMemoryCatalog;
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.TableChange;
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException;
import org.apache.flink.table.expressions.SqlCallExpression;
import org.apache.flink.table.factories.TestManagedTableFactory;
import org.apache.flink.table.operations.NopOperation;
import org.apache.flink.table.operations.Operation;
import org.apache.flink.table.operations.SinkModifyOperation;
import org.apache.flink.table.operations.SourceQueryOperation;
import org.apache.flink.table.operations.ddl.AlterDatabaseOperation;
import org.apache.flink.table.operations.ddl.AlterTableChangeOperation;
import org.apache.flink.table.operations.ddl.AlterTableRenameOperation;
import org.apache.flink.table.operations.ddl.CreateCatalogFunctionOperation;
import org.apache.flink.table.operations.ddl.CreateDatabaseOperation;
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.table.operations.ddl.CreateTempSystemFunctionOperation;
import org.apache.flink.table.operations.ddl.CreateViewOperation;
import org.apache.flink.table.operations.ddl.DropDatabaseOperation;
import org.apache.flink.table.planner.calcite.FlinkPlannerImpl;
import org.apache.flink.table.planner.expressions.utils.Func0$;
import org.apache.flink.table.planner.expressions.utils.Func1$;
import org.apache.flink.table.planner.expressions.utils.Func8$;
import org.apache.flink.table.planner.parse.CalciteParser;
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions;
import org.apache.flink.table.resource.ResourceType;
import org.apache.flink.table.resource.ResourceUri;
import org.apache.flink.table.types.DataType;
import org.apache.calcite.sql.SqlNode;
import org.assertj.core.api.HamcrestCondition;
import org.junit.jupiter.api.Test;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import static org.apache.flink.table.api.Expressions.$;
import static org.apache.flink.table.planner.utils.OperationMatchers.entry;
import static org.apache.flink.table.planner.utils.OperationMatchers.isCreateTableOperation;
import static org.apache.flink.table.planner.utils.OperationMatchers.partitionedBy;
import static org.apache.flink.table.planner.utils.OperationMatchers.withOptions;
import static org.apache.flink.table.planner.utils.OperationMatchers.withSchema;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
/** Test cases for the DDL statements for {@link SqlToOperationConverter}. */
public class SqlDdlToOperationConverterTest extends SqlToOperationConverterTestBase {
@Test
public void testCreateDatabase() {
final String[] createDatabaseSqls =
new String[] {
"create database db1",
"create database if not exists cat1.db1",
"create database cat1.db1 comment 'db1_comment'",
"create database cat1.db1 comment 'db1_comment' with ('k1' = 'v1', 'K2' = 'V2')"
};
final String[] expectedCatalogs = new String[] {"builtin", "cat1", "cat1", "cat1"};
final String expectedDatabase = "db1";
final String[] expectedComments = new String[] {null, null, "db1_comment", "db1_comment"};
final boolean[] expectedIgnoreIfExists = new boolean[] {false, true, false, false};
Map<String, String> properties = new HashMap<>();
properties.put("k1", "v1");
properties.put("K2", "V2");
final Map[] expectedProperties =
new Map[] {
new HashMap<String, String>(),
new HashMap<String, String>(),
new HashMap<String, String>(),
new HashMap(properties)
};
for (int i = 0; i < createDatabaseSqls.length; i++) {
Operation operation = parse(createDatabaseSqls[i]);
assertThat(operation).isInstanceOf(CreateDatabaseOperation.class);
final CreateDatabaseOperation createDatabaseOperation =
(CreateDatabaseOperation) operation;
assertThat(createDatabaseOperation.getCatalogName()).isEqualTo(expectedCatalogs[i]);
assertThat(createDatabaseOperation.getDatabaseName()).isEqualTo(expectedDatabase);
assertThat(createDatabaseOperation.getCatalogDatabase().getComment())
.isEqualTo(expectedComments[i]);
assertThat(createDatabaseOperation.isIgnoreIfExists())
.isEqualTo(expectedIgnoreIfExists[i]);
assertThat(createDatabaseOperation.getCatalogDatabase().getProperties())
.isEqualTo(expectedProperties[i]);
}
}
@Test
public void testCreateDatabaseWithNewLineInTableOptions() {
String sql =
"create database cat1.db1 comment 'db1_comment' with ('k\n1' = 'v1', 'K2' = 'V\n2')";
Operation operation = parse(sql);
Map<String, String> properties = new HashMap<>();
properties.put("k1", "v1");
properties.put("K2", "V2");
assertThat(operation).isInstanceOf(CreateDatabaseOperation.class);
final CreateDatabaseOperation createDatabaseOperation = (CreateDatabaseOperation) operation;
assertThat(createDatabaseOperation.getCatalogDatabase().getProperties())
.isEqualTo(new HashMap<>(properties));
}
@Test
public void testDropDatabase() {
final String[] dropDatabaseSqls =
new String[] {
"drop database db1",
"drop database if exists db1",
"drop database if exists cat1.db1 CASCADE",
"drop database if exists cat1.db1 RESTRICT"
};
final String[] expectedCatalogs = new String[] {"builtin", "builtin", "cat1", "cat1"};
final String expectedDatabase = "db1";
final boolean[] expectedIfExists = new boolean[] {false, true, true, true};
final boolean[] expectedIsCascades = new boolean[] {false, false, true, false};
for (int i = 0; i < dropDatabaseSqls.length; i++) {
Operation operation = parse(dropDatabaseSqls[i]);
assertThat(operation).isInstanceOf(DropDatabaseOperation.class);
final DropDatabaseOperation dropDatabaseOperation = (DropDatabaseOperation) operation;
assertThat(dropDatabaseOperation.getCatalogName()).isEqualTo(expectedCatalogs[i]);
assertThat(dropDatabaseOperation.getDatabaseName()).isEqualTo(expectedDatabase);
assertThat(dropDatabaseOperation.isIfExists()).isEqualTo(expectedIfExists[i]);
assertThat(dropDatabaseOperation.isCascade()).isEqualTo(expectedIsCascades[i]);
}
}
@Test
public void testAlterDatabase() throws Exception {
catalogManager.registerCatalog("cat1", new GenericInMemoryCatalog("default", "default"));
catalogManager
.getCatalog("cat1")
.get()
.createDatabase(
"db1", new CatalogDatabaseImpl(new HashMap<>(), "db1_comment"), true);
final String sql = "alter database cat1.db1 set ('k1'='v1', 'K2'='V2')";
Operation operation = parse(sql);
assertThat(operation).isInstanceOf(AlterDatabaseOperation.class);
Map<String, String> properties = new HashMap<>();
properties.put("k1", "v1");
properties.put("K2", "V2");
AlterDatabaseOperation alterDatabaseOperation = (AlterDatabaseOperation) operation;
assertThat(alterDatabaseOperation.getDatabaseName()).isEqualTo("db1");
assertThat(alterDatabaseOperation.getCatalogName()).isEqualTo("cat1");
assertThat(alterDatabaseOperation.getCatalogDatabase().getComment())
.isEqualTo("db1_comment");
assertThat(alterDatabaseOperation.getCatalogDatabase().getProperties())
.isEqualTo(properties);
}
@Test
public void testCreateTable() {
final String sql =
"CREATE TABLE tbl1 (\n"
+ " a bigint,\n"
+ " b varchar, \n"
+ " c int, \n"
+ " d varchar"
+ ")\n"
+ " PARTITIONED BY (a, d)\n"
+ " with (\n"
+ " 'connector' = 'kafka', \n"
+ " 'kafka.topic' = 'log.test'\n"
+ ")\n";
FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT);
Operation operation = parse(sql, planner, parser);
assertThat(operation).isInstanceOf(CreateTableOperation.class);
CreateTableOperation op = (CreateTableOperation) operation;
CatalogTable catalogTable = op.getCatalogTable();
assertThat(catalogTable.getPartitionKeys()).hasSameElementsAs(Arrays.asList("a", "d"));
assertThat(catalogTable.getSchema().getFieldNames())
.isEqualTo(new String[] {"a", "b", "c", "d"});
assertThat(catalogTable.getSchema().getFieldDataTypes())
.isEqualTo(
new DataType[] {
DataTypes.BIGINT(),
DataTypes.VARCHAR(Integer.MAX_VALUE),
DataTypes.INT(),
DataTypes.VARCHAR(Integer.MAX_VALUE)
});
}
@Test
public void testCreateTableWithPrimaryKey() {
final String sql =
"CREATE TABLE tbl1 (\n"
+ " a bigint,\n"
+ " b varchar, \n"
+ " c int, \n"
+ " d varchar, \n"
+ " constraint ct1 primary key(a, b) not enforced\n"
+ ") with (\n"
+ " 'connector' = 'kafka', \n"
+ " 'kafka.topic' = 'log.test'\n"
+ ")\n";
FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT);
Operation operation = parse(sql, planner, parser);
assertThat(operation).isInstanceOf(CreateTableOperation.class);
CreateTableOperation op = (CreateTableOperation) operation;
CatalogTable catalogTable = op.getCatalogTable();
TableSchema tableSchema = catalogTable.getSchema();
assertThat(
tableSchema
.getPrimaryKey()
.map(UniqueConstraint::asSummaryString)
.orElse("fakeVal"))
.isEqualTo("CONSTRAINT ct1 PRIMARY KEY (a, b)");
assertThat(tableSchema.getFieldNames()).isEqualTo(new String[] {"a", "b", "c", "d"});
assertThat(tableSchema.getFieldDataTypes())
.isEqualTo(
new DataType[] {
DataTypes.BIGINT().notNull(),
DataTypes.STRING().notNull(),
DataTypes.INT(),
DataTypes.STRING()
});
}
@Test
public void testPrimaryKeyOnGeneratedColumn() {
final String sql =
"CREATE TABLE tbl1 (\n"
+ " a bigint not null,\n"
+ " b varchar not null,\n"
+ " c as 2 * (a + 1),\n"
+ " constraint ct1 primary key (b, c) not enforced"
+ ") with (\n"
+ " 'connector' = 'kafka',\n"
+ " 'kafka.topic' = 'log.test'\n"
+ ")\n";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"Could not create a PRIMARY KEY with column 'c' at line 5, column 34.\n"
+ "A PRIMARY KEY constraint must be declared on physical columns.");
}
@Test
public void testPrimaryKeyNonExistentColumn() {
final String sql =
"CREATE TABLE tbl1 (\n"
+ " a bigint not null,\n"
+ " b varchar not null,\n"
+ " c as 2 * (a + 1),\n"
+ " constraint ct1 primary key (b, d) not enforced"
+ ") with (\n"
+ " 'connector' = 'kafka',\n"
+ " 'kafka.topic' = 'log.test'\n"
+ ")\n";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"Primary key column 'd' is not defined in the schema at line 5, column 34");
}
@Test
public void testCreateTableWithMinusInOptionKey() {
final String sql =
"create table source_table(\n"
+ " a int,\n"
+ " b bigint,\n"
+ " c varchar\n"
+ ") with (\n"
+ " 'a-B-c-d124' = 'Ab',\n"
+ " 'a.b-c-d.e-f.g' = 'ada',\n"
+ " 'a.b-c-d.e-f1231.g' = 'ada',\n"
+ " 'a.b-c-d.*' = 'adad')\n";
final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT);
SqlNode node = parser.parse(sql);
assertThat(node).isInstanceOf(SqlCreateTable.class);
Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get();
assertThat(operation).isInstanceOf(CreateTableOperation.class);
CreateTableOperation op = (CreateTableOperation) operation;
CatalogTable catalogTable = op.getCatalogTable();
Map<String, String> options =
catalogTable.getOptions().entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
Map<String, String> sortedProperties = new TreeMap<>(options);
final String expected =
"{a-B-c-d124=Ab, "
+ "a.b-c-d.*=adad, "
+ "a.b-c-d.e-f.g=ada, "
+ "a.b-c-d.e-f1231.g=ada}";
assertThat(sortedProperties.toString()).isEqualTo(expected);
}
@Test
public void testCreateTableWithWatermark()
throws FunctionAlreadyExistException, DatabaseNotExistException {
CatalogFunction cf =
new CatalogFunctionImpl(JavaUserDefinedScalarFunctions.JavaFunc5.class.getName());
catalog.createFunction(ObjectPath.fromString("default.myfunc"), cf, true);
final String sql =
"create table source_table(\n"
+ " a int,\n"
+ " b bigint,\n"
+ " c timestamp(3),\n"
+ " watermark for `c` as myfunc(c, 1) - interval '5' second\n"
+ ") with (\n"
+ " 'connector.type' = 'kafka')\n";
final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT);
SqlNode node = parser.parse(sql);
assertThat(node).isInstanceOf(SqlCreateTable.class);
Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get();
assertThat(operation).isInstanceOf(CreateTableOperation.class);
CreateTableOperation op = (CreateTableOperation) operation;
CatalogTable catalogTable = op.getCatalogTable();
Map<String, String> properties = catalogTable.toProperties();
Map<String, String> expected = new HashMap<>();
expected.put("schema.0.name", "a");
expected.put("schema.0.data-type", "INT");
expected.put("schema.1.name", "b");
expected.put("schema.1.data-type", "BIGINT");
expected.put("schema.2.name", "c");
expected.put("schema.2.data-type", "TIMESTAMP(3)");
expected.put("schema.watermark.0.rowtime", "c");
expected.put(
"schema.watermark.0.strategy.expr",
"`builtin`.`default`.`myfunc`(`c`, 1) - INTERVAL '5' SECOND");
expected.put("schema.watermark.0.strategy.data-type", "TIMESTAMP(3)");
expected.put("connector.type", "kafka");
assertThat(properties).isEqualTo(expected);
}
@Test
public void testBasicCreateTableLike() {
Map<String, String> sourceProperties = new HashMap<>();
sourceProperties.put("format.type", "json");
CatalogTable catalogTable =
CatalogTable.of(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column("f1", DataTypes.TIMESTAMP(3))
.build(),
null,
Collections.emptyList(),
sourceProperties);
catalogManager.createTable(
catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false);
final String sql =
"create table derivedTable(\n"
+ " a int,\n"
+ " watermark for f1 as `f1` - interval '5' second\n"
+ ")\n"
+ "PARTITIONED BY (a, f0)\n"
+ "with (\n"
+ " 'connector.type' = 'kafka'"
+ ")\n"
+ "like sourceTable";
Operation operation = parseAndConvert(sql);
assertThat(operation)
.is(
new HamcrestCondition<>(
isCreateTableOperation(
withSchema(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column("f1", DataTypes.TIMESTAMP(3))
.column("a", DataTypes.INT())
.watermark(
"f1", "`f1` - INTERVAL '5' SECOND")
.build()),
withOptions(
entry("connector.type", "kafka"),
entry("format.type", "json")),
partitionedBy("a", "f0"))));
}
@Test
public void testCreateTableLikeWithFullPath() {
Map<String, String> sourceProperties = new HashMap<>();
sourceProperties.put("connector.type", "kafka");
sourceProperties.put("format.type", "json");
CatalogTable catalogTable =
CatalogTable.of(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column("f1", DataTypes.TIMESTAMP(3))
.build(),
null,
Collections.emptyList(),
sourceProperties);
catalogManager.createTable(
catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false);
final String sql = "create table mytable like `builtin`.`default`.sourceTable";
Operation operation = parseAndConvert(sql);
assertThat(operation)
.is(
new HamcrestCondition<>(
isCreateTableOperation(
withSchema(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column("f1", DataTypes.TIMESTAMP(3))
.build()),
withOptions(
entry("connector.type", "kafka"),
entry("format.type", "json")))));
}
@Test
public void testMergingCreateTableLike() {
Map<String, String> sourceProperties = new HashMap<>();
sourceProperties.put("format.type", "json");
CatalogTable catalogTable =
CatalogTable.of(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column("f1", DataTypes.TIMESTAMP(3))
.columnByExpression("f2", "`f0` + 12345")
.watermark("f1", "`f1` - interval '1' second")
.build(),
null,
Arrays.asList("f0", "f1"),
sourceProperties);
catalogManager.createTable(
catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false);
final String sql =
"create table derivedTable(\n"
+ " a int,\n"
+ " watermark for f1 as `f1` - interval '5' second\n"
+ ")\n"
+ "PARTITIONED BY (a, f0)\n"
+ "with (\n"
+ " 'connector.type' = 'kafka'"
+ ")\n"
+ "like sourceTable (\n"
+ " EXCLUDING GENERATED\n"
+ " EXCLUDING PARTITIONS\n"
+ " OVERWRITING OPTIONS\n"
+ " OVERWRITING WATERMARKS"
+ ")";
Operation operation = parseAndConvert(sql);
assertThat(operation)
.is(
new HamcrestCondition<>(
isCreateTableOperation(
withSchema(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column("f1", DataTypes.TIMESTAMP(3))
.column("a", DataTypes.INT())
.watermark(
"f1", "`f1` - INTERVAL '5' SECOND")
.build()),
withOptions(
entry("connector.type", "kafka"),
entry("format.type", "json")),
partitionedBy("a", "f0"))));
}
@Test
public void testCreateTableInvalidPartition() {
final String sql =
"create table derivedTable(\n" + " a int\n" + ")\n" + "PARTITIONED BY (f3)";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"Partition column 'f3' not defined in the table schema. Available columns: ['a']");
}
@Test
public void testCreateTableLikeInvalidPartition() {
CatalogTable catalogTable =
CatalogTable.of(
Schema.newBuilder().column("f0", DataTypes.INT().notNull()).build(),
null,
Collections.emptyList(),
Collections.emptyMap());
catalogManager.createTable(
catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false);
final String sql =
"create table derivedTable(\n"
+ " a int\n"
+ ")\n"
+ "PARTITIONED BY (f3)\n"
+ "like sourceTable";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"Partition column 'f3' not defined in the table schema. Available columns: ['f0', 'a']");
}
@Test
public void testCreateTableInvalidWatermark() {
final String sql =
"create table derivedTable(\n"
+ " a int,\n"
+ " watermark for f1 as `f1` - interval '5' second\n"
+ ")";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"The rowtime attribute field 'f1' is not defined in the table schema,"
+ " at line 3, column 17\n"
+ "Available fields: ['a']");
}
@Test
public void testCreateTableLikeInvalidWatermark() {
CatalogTable catalogTable =
CatalogTable.of(
Schema.newBuilder().column("f0", DataTypes.INT().notNull()).build(),
null,
Collections.emptyList(),
Collections.emptyMap());
catalogManager.createTable(
catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false);
final String sql =
"create table derivedTable(\n"
+ " a int,\n"
+ " watermark for f1 as `f1` - interval '5' second\n"
+ ")\n"
+ "like sourceTable";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"The rowtime attribute field 'f1' is not defined in the table schema,"
+ " at line 3, column 17\n"
+ "Available fields: ['f0', 'a']");
}
@Test
public void testCreateTableLikeNestedWatermark() {
CatalogTable catalogTable =
CatalogTable.of(
Schema.newBuilder()
.column("f0", DataTypes.INT().notNull())
.column(
"f1",
DataTypes.ROW(
DataTypes.FIELD("tmstmp", DataTypes.TIMESTAMP(3))))
.build(),
null,
Collections.emptyList(),
Collections.emptyMap());
catalogManager.createTable(
catalogTable, ObjectIdentifier.of("builtin", "default", "sourceTable"), false);
final String sql =
"create table derivedTable(\n"
+ " a int,\n"
+ " watermark for f1.t as f1.t - interval '5' second\n"
+ ")\n"
+ "like sourceTable";
assertThatThrownBy(() -> parseAndConvert(sql))
.isInstanceOf(ValidationException.class)
.hasMessageContaining(
"The rowtime attribute field 'f1.t' is not defined in the table schema,"
+ " at line 3, column 20\n"
+ "Nested field 't' was not found in a composite type:"
+ " ROW<`tmstmp` TIMESTAMP(3)>.");
}
@Test // TODO: tweak the tests when FLINK-13604 is fixed.
public void testCreateTableWithFullDataTypes() {
final List<TestItem> testItems =
Arrays.asList(
createTestItem("CHAR", DataTypes.CHAR(1)),
createTestItem("CHAR NOT NULL", DataTypes.CHAR(1).notNull()),
createTestItem("CHAR NULL", DataTypes.CHAR(1)),
createTestItem("CHAR(33)", DataTypes.CHAR(33)),
createTestItem("VARCHAR", DataTypes.STRING()),
createTestItem("VARCHAR(33)", DataTypes.VARCHAR(33)),
createTestItem("STRING", DataTypes.STRING()),
createTestItem("BOOLEAN", DataTypes.BOOLEAN()),
createTestItem("BINARY", DataTypes.BINARY(1)),
createTestItem("BINARY(33)", DataTypes.BINARY(33)),
createTestItem("VARBINARY", DataTypes.BYTES()),
createTestItem("VARBINARY(33)", DataTypes.VARBINARY(33)),
createTestItem("BYTES", DataTypes.BYTES()),
createTestItem("DECIMAL", DataTypes.DECIMAL(10, 0)),
createTestItem("DEC", DataTypes.DECIMAL(10, 0)),
createTestItem("NUMERIC", DataTypes.DECIMAL(10, 0)),
createTestItem("DECIMAL(10)", DataTypes.DECIMAL(10, 0)),
createTestItem("DEC(10)", DataTypes.DECIMAL(10, 0)),
createTestItem("NUMERIC(10)", DataTypes.DECIMAL(10, 0)),
createTestItem("DECIMAL(10, 3)", DataTypes.DECIMAL(10, 3)),
createTestItem("DEC(10, 3)", DataTypes.DECIMAL(10, 3)),
createTestItem("NUMERIC(10, 3)", DataTypes.DECIMAL(10, 3)),
createTestItem("TINYINT", DataTypes.TINYINT()),
createTestItem("SMALLINT", DataTypes.SMALLINT()),
createTestItem("INTEGER", DataTypes.INT()),
createTestItem("INT", DataTypes.INT()),
createTestItem("BIGINT", DataTypes.BIGINT()),
createTestItem("FLOAT", DataTypes.FLOAT()),
createTestItem("DOUBLE", DataTypes.DOUBLE()),
createTestItem("DOUBLE PRECISION", DataTypes.DOUBLE()),
createTestItem("DATE", DataTypes.DATE()),
createTestItem("TIME", DataTypes.TIME()),
createTestItem("TIME WITHOUT TIME ZONE", DataTypes.TIME()),
// Expect to be TIME(3).
createTestItem("TIME(3)", DataTypes.TIME()),
// Expect to be TIME(3).
createTestItem("TIME(3) WITHOUT TIME ZONE", DataTypes.TIME()),
createTestItem("TIMESTAMP", DataTypes.TIMESTAMP(6)),
createTestItem("TIMESTAMP WITHOUT TIME ZONE", DataTypes.TIMESTAMP(6)),
createTestItem("TIMESTAMP(3)", DataTypes.TIMESTAMP(3)),
createTestItem("TIMESTAMP(3) WITHOUT TIME ZONE", DataTypes.TIMESTAMP(3)),
createTestItem(
"TIMESTAMP WITH LOCAL TIME ZONE",
DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(6)),
createTestItem(
"TIMESTAMP(3) WITH LOCAL TIME ZONE",
DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)),
createTestItem(
"ARRAY<TIMESTAMP(3) WITH LOCAL TIME ZONE>",
DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3))),
createTestItem(
"ARRAY<INT NOT NULL>", DataTypes.ARRAY(DataTypes.INT().notNull())),
createTestItem("INT ARRAY", DataTypes.ARRAY(DataTypes.INT())),
createTestItem(
"INT NOT NULL ARRAY", DataTypes.ARRAY(DataTypes.INT().notNull())),
createTestItem(
"INT ARRAY NOT NULL", DataTypes.ARRAY(DataTypes.INT()).notNull()),
createTestItem(
"MULTISET<INT NOT NULL>",
DataTypes.MULTISET(DataTypes.INT().notNull())),
createTestItem("INT MULTISET", DataTypes.MULTISET(DataTypes.INT())),
createTestItem(
"INT NOT NULL MULTISET",
DataTypes.MULTISET(DataTypes.INT().notNull())),
createTestItem(
"INT MULTISET NOT NULL",
DataTypes.MULTISET(DataTypes.INT()).notNull()),
createTestItem(
"MAP<BIGINT, BOOLEAN>",
DataTypes.MAP(DataTypes.BIGINT(), DataTypes.BOOLEAN())),
// Expect to be ROW<`f0` INT NOT NULL, `f1` BOOLEAN>.
createTestItem(
"ROW<f0 INT NOT NULL, f1 BOOLEAN>",
DataTypes.ROW(
DataTypes.FIELD("f0", DataTypes.INT()),
DataTypes.FIELD("f1", DataTypes.BOOLEAN()))),
// Expect to be ROW<`f0` INT NOT NULL, `f1` BOOLEAN>.
createTestItem(
"ROW(f0 INT NOT NULL, f1 BOOLEAN)",
DataTypes.ROW(
DataTypes.FIELD("f0", DataTypes.INT()),
DataTypes.FIELD("f1", DataTypes.BOOLEAN()))),
createTestItem(
"ROW<`f0` INT>",
DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()))),
createTestItem(
"ROW(`f0` INT)",
DataTypes.ROW(DataTypes.FIELD("f0", DataTypes.INT()))),
createTestItem("ROW<>", DataTypes.ROW()),
createTestItem("ROW()", DataTypes.ROW()),
// Expect to be ROW<`f0` INT NOT NULL '...', `f1` BOOLEAN '...'>.
createTestItem(
"ROW<f0 INT NOT NULL 'This is a comment.',"
+ " f1 BOOLEAN 'This as well.'>",
DataTypes.ROW(
DataTypes.FIELD("f0", DataTypes.INT()),
DataTypes.FIELD("f1", DataTypes.BOOLEAN()))),
createTestItem(
"ARRAY<ROW<f0 INT, f1 BOOLEAN>>",
DataTypes.ARRAY(
DataTypes.ROW(
DataTypes.FIELD("f0", DataTypes.INT()),
DataTypes.FIELD("f1", DataTypes.BOOLEAN())))),
createTestItem(
"ROW<f0 INT, f1 BOOLEAN> MULTISET",
DataTypes.MULTISET(
DataTypes.ROW(
DataTypes.FIELD("f0", DataTypes.INT()),
DataTypes.FIELD("f1", DataTypes.BOOLEAN())))),
createTestItem(
"MULTISET<ROW<f0 INT, f1 BOOLEAN>>",
DataTypes.MULTISET(
DataTypes.ROW(
DataTypes.FIELD("f0", DataTypes.INT()),
DataTypes.FIELD("f1", DataTypes.BOOLEAN())))),
createTestItem(
"ROW<f0 Row<f00 INT, f01 BOOLEAN>, "
+ "f1 INT ARRAY, "
+ "f2 BOOLEAN MULTISET>",
DataTypes.ROW(
DataTypes.FIELD(
"f0",
DataTypes.ROW(
DataTypes.FIELD("f00", DataTypes.INT()),
DataTypes.FIELD(
"f01", DataTypes.BOOLEAN()))),
DataTypes.FIELD("f1", DataTypes.ARRAY(DataTypes.INT())),
DataTypes.FIELD(
"f2", DataTypes.MULTISET(DataTypes.BOOLEAN())))));
StringBuilder buffer = new StringBuilder("create table t1(\n");
for (int i = 0; i < testItems.size(); i++) {
buffer.append("f").append(i).append(" ").append(testItems.get(i).testExpr);
if (i == testItems.size() - 1) {
buffer.append(")");
} else {
buffer.append(",\n");
}
}
final String sql = buffer.toString();
final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT);
SqlNode node = parser.parse(sql);
assertThat(node).isInstanceOf(SqlCreateTable.class);
Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get();
TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema();
Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray();
assertThat(schema.getFieldDataTypes()).isEqualTo(expectedDataTypes);
}
@Test
public void testCreateTableWithComputedColumn() {
final String sql =
"CREATE TABLE tbl1 (\n"
+ " a int,\n"
+ " b varchar, \n"
+ " c as a - 1, \n"
+ " d as b || '$$', \n"
+ " e as my_udf1(a),"
+ " f as `default`.my_udf2(a) + 1,"
+ " g as builtin.`default`.my_udf3(a) || '##'\n"
+ ")\n"
+ " with (\n"
+ " 'connector' = 'kafka', \n"
+ " 'kafka.topic' = 'log.test'\n"
+ ")\n";
functionCatalog.registerTempCatalogScalarFunction(
ObjectIdentifier.of("builtin", "default", "my_udf1"), Func0$.MODULE$);
functionCatalog.registerTempCatalogScalarFunction(
ObjectIdentifier.of("builtin", "default", "my_udf2"), Func1$.MODULE$);
functionCatalog.registerTempCatalogScalarFunction(
ObjectIdentifier.of("builtin", "default", "my_udf3"), Func8$.MODULE$);
FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
Operation operation = parse(sql, planner, getParserBySqlDialect(SqlDialect.DEFAULT));
assertThat(operation).isInstanceOf(CreateTableOperation.class);
CreateTableOperation op = (CreateTableOperation) operation;
CatalogTable catalogTable = op.getCatalogTable();
assertThat(catalogTable.getSchema().getFieldNames())
.isEqualTo(new String[] {"a", "b", "c", "d", "e", "f", "g"});
assertThat(catalogTable.getSchema().getFieldDataTypes())
.isEqualTo(
new DataType[] {
DataTypes.INT(),
DataTypes.STRING(),
DataTypes.INT(),
DataTypes.STRING(),
DataTypes.INT().notNull(),
DataTypes.INT(),
DataTypes.STRING()
});
String[] columnExpressions =
catalogTable.getSchema().getTableColumns().stream()
.filter(TableColumn.ComputedColumn.class::isInstance)
.map(TableColumn.ComputedColumn.class::cast)
.map(TableColumn.ComputedColumn::getExpression)
.toArray(String[]::new);
String[] expected =
new String[] {
"`a` - 1",
"`b` || '$$'",
"`builtin`.`default`.`my_udf1`(`a`)",
"`builtin`.`default`.`my_udf2`(`a`) + 1",
"`builtin`.`default`.`my_udf3`(`a`) || '##'"
};
assertThat(columnExpressions).isEqualTo(expected);
}
@Test
public void testCreateTableWithMetadataColumn() {
final String sql =
"CREATE TABLE tbl1 (\n"
+ " a INT,\n"
+ " b STRING,\n"
+ " c INT METADATA,\n"
+ " d INT METADATA FROM 'other.key',\n"
+ " e INT METADATA VIRTUAL\n"
+ ")\n"
+ " WITH (\n"
+ " 'connector' = 'kafka',\n"
+ " 'kafka.topic' = 'log.test'\n"
+ ")\n";
final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
final Operation operation = parse(sql, planner, getParserBySqlDialect(SqlDialect.DEFAULT));
assertThat(operation).isInstanceOf(CreateTableOperation.class);
final CreateTableOperation op = (CreateTableOperation) operation;
final TableSchema actualSchema = op.getCatalogTable().getSchema();
final TableSchema expectedSchema =
TableSchema.builder()
.add(TableColumn.physical("a", DataTypes.INT()))
.add(TableColumn.physical("b", DataTypes.STRING()))
.add(TableColumn.metadata("c", DataTypes.INT()))
.add(TableColumn.metadata("d", DataTypes.INT(), "other.key"))
.add(TableColumn.metadata("e", DataTypes.INT(), true))
.build();
assertThat(actualSchema).isEqualTo(expectedSchema);
}
@Test
public void testCreateFunction() {
// test create catalog function
String sql =
"CREATE FUNCTION test_udf AS 'org.apache.fink.function.function1' "
+ "LANGUAGE JAVA USING JAR 'file:///path/to/test.jar'";
final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT);
Operation operation = parse(sql, planner, getParserBySqlDialect(SqlDialect.DEFAULT));
assertThat(operation).isInstanceOf(CreateCatalogFunctionOperation.class);
CatalogFunction actualFunction =
((CreateCatalogFunctionOperation) operation).getCatalogFunction();
assertThat(operation.asSummaryString())
.isEqualTo(
"CREATE CATALOG FUNCTION: (catalogFunction: [Optional[This is a user-defined function]], "
+ "identifier: [`builtin`.`default`.`test_udf`], ignoreIfExists: [false], isTemporary: [false])");
CatalogFunction expected =
new CatalogFunctionImpl(
"org.apache.fink.function.function1",
FunctionLanguage.JAVA,
Collections.singletonList(
new ResourceUri(ResourceType.JAR, "file:///path/to/test.jar")));
assertThat(actualFunction).isEqualTo(expected);
// test create temporary system function
sql =
"CREATE TEMPORARY SYSTEM FUNCTION test_udf2 AS 'org.apache.fink.function.function2' "
+ "LANGUAGE SCALA USING JAR 'file:///path/to/test.jar'";
operation = parse(sql, planner, getParserBySqlDialect(SqlDialect.DEFAULT));
assertThat(operation).isInstanceOf(CreateTempSystemFunctionOperation.class);
assertThat(operation.asSummaryString())
.isEqualTo(
"CREATE TEMPORARY SYSTEM FUNCTION: (functionName: [test_udf2], "
+ "catalogFunction: [CatalogFunctionImpl{className='org.apache.fink.function.function2', "
+ "functionLanguage='SCALA', "
+ "functionResource='[ResourceUri{resourceType=JAR, uri='file:///path/to/test.jar'}]'}], "
+ "ignoreIfExists: [false], functionLanguage: [SCALA])");
}
@Test
public void testAlterTable() throws Exception {
prepareNonManagedTable(false);
final String[] renameTableSqls =
new String[] {
"alter table cat1.db1.tb1 rename to tb2",
"alter table db1.tb1 rename to tb2",
"alter table tb1 rename to cat1.db1.tb2",
};
final ObjectIdentifier expectedIdentifier = ObjectIdentifier.of("cat1", "db1", "tb1");
final ObjectIdentifier expectedNewIdentifier = ObjectIdentifier.of("cat1", "db1", "tb2");
// test rename table converter
for (String renameTableSql : renameTableSqls) {
Operation operation = parse(renameTableSql);
assertThat(operation).isInstanceOf(AlterTableRenameOperation.class);
final AlterTableRenameOperation alterTableRenameOperation =
(AlterTableRenameOperation) operation;
assertThat(alterTableRenameOperation.getTableIdentifier())
.isEqualTo(expectedIdentifier);
assertThat(alterTableRenameOperation.getNewTableIdentifier())
.isEqualTo(expectedNewIdentifier);
}
// test alter nonexistent table
checkAlterNonExistTable("alter table %s nonexistent rename to tb2");
// test alter table options
checkAlterNonExistTable("alter table %s nonexistent set ('k1' = 'v1', 'K2' = 'V2')");
Operation operation =
parse("alter table if exists cat1.db1.tb1 set ('k1' = 'v1', 'K2' = 'V2')");
Map<String, String> expectedOptions = new HashMap<>();
expectedOptions.put("connector", "dummy");
expectedOptions.put("k", "v");
expectedOptions.put("k1", "v1");
expectedOptions.put("K2", "V2");
assertAlterTableOptions(
operation,
expectedIdentifier,
expectedOptions,
Arrays.asList(TableChange.set("k1", "v1"), TableChange.set("K2", "V2")),
"ALTER TABLE IF EXISTS cat1.db1.tb1\n SET 'k1' = 'v1',\n SET 'K2' = 'V2'");
// test alter table reset
checkAlterNonExistTable("alter table %s nonexistent reset ('k')");
operation = parse("alter table if exists cat1.db1.tb1 reset ('k')");
assertAlterTableOptions(
operation,
expectedIdentifier,
Collections.singletonMap("connector", "dummy"),
Collections.singletonList(TableChange.reset("k")),
"ALTER TABLE IF EXISTS cat1.db1.tb1\n RESET 'k'");
assertThatThrownBy(() -> parse("alter table cat1.db1.tb1 reset ('connector')"))
.isInstanceOf(ValidationException.class)
.hasMessageContaining("ALTER TABLE RESET does not support changing 'connector'");
assertThatThrownBy(() -> parse("alter table cat1.db1.tb1 reset ()"))
.isInstanceOf(ValidationException.class)
.hasMessageContaining("ALTER TABLE RESET does not support empty key");
}
@Test
public void testAlterTableRenameColumn() throws Exception {
prepareTable("tb1", false, false, true, 3);
// rename pk column c
Operation operation = parse("alter table tb1 rename c to c1");
assertThat(operation).isInstanceOf(AlterTableChangeOperation.class);
assertThat(operation.asSummaryString())
.isEqualTo("ALTER TABLE cat1.db1.tb1\n MODIFY `c` TO `c1`");
assertThat(((AlterTableChangeOperation) operation).getNewTable().getUnresolvedSchema())
.isEqualTo(
Schema.newBuilder()
.column("a", DataTypes.INT().notNull())
.column("b", DataTypes.BIGINT().notNull())
.column("c1", DataTypes.STRING().notNull())
.withComment("column comment")
.columnByExpression("d", "a*(b+2 + a*b)")
.column(
"e",
DataTypes.ROW(
DataTypes.STRING(),
DataTypes.INT(),
DataTypes.ROW(
DataTypes.DOUBLE(),
DataTypes.ARRAY(DataTypes.FLOAT()))))
.columnByExpression("f", "e.f1 + e.f2.f0")
.columnByMetadata("g", DataTypes.STRING(), null, true)
.column("ts", DataTypes.TIMESTAMP(3))
.withComment("just a comment")
.watermark("ts", "ts - interval '5' seconds")
.primaryKeyNamed("ct1", "a", "b", "c1")
.build());