-
Notifications
You must be signed in to change notification settings - Fork 2.4k
/
HoodieDataSourceITCase.java
1181 lines (1022 loc) · 49.4 KB
/
HoodieDataSourceITCase.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.hudi.table;
import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.hudi.utils.TestData;
import org.apache.hudi.utils.TestSQL;
import org.apache.hudi.utils.TestUtils;
import org.apache.hudi.utils.factory.CollectSinkTableFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.exceptions.TableNotExistException;
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.flink.types.Row;
import org.apache.flink.util.CollectionUtil;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.EnumSource;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.File;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.utils.TestConfigurations.sql;
import static org.apache.hudi.utils.TestData.assertRowsEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* IT cases for Hoodie table source and sink.
*/
public class HoodieDataSourceITCase extends AbstractTestBase {
private TableEnvironment streamTableEnv;
private TableEnvironment batchTableEnv;
@BeforeEach
void beforeEach() {
EnvironmentSettings settings = EnvironmentSettings.newInstance().build();
streamTableEnv = TableEnvironmentImpl.create(settings);
streamTableEnv.getConfig().getConfiguration()
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
Configuration execConf = streamTableEnv.getConfig().getConfiguration();
execConf.setString("execution.checkpointing.interval", "2s");
// configure not to retry after failure
execConf.setString("restart-strategy", "fixed-delay");
execConf.setString("restart-strategy.fixed-delay.attempts", "0");
settings = EnvironmentSettings.newInstance().inBatchMode().build();
batchTableEnv = TableEnvironmentImpl.create(settings);
batchTableEnv.getConfig().getConfiguration()
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
}
@TempDir
File tempFile;
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testStreamWriteAndReadFromSpecifiedCommit(HoodieTableType tableType) throws Exception {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
String firstCommit = TestUtils.getFirstCompleteInstant(tempFile.getAbsolutePath());
streamTableEnv.executeSql("drop table t1");
hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_START_COMMIT, firstCommit)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
// insert another batch of data
execInsertSql(streamTableEnv, insertInto);
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows2, TestData.DATA_SET_SOURCE_INSERT);
streamTableEnv.getConfig().getConfiguration()
.setBoolean("table.dynamic-table-options.enabled", true);
// specify the start commit as earliest
List<Row> rows3 = execSelectSql(streamTableEnv,
"select * from t1/*+options('read.start-commit'='earliest')*/", 10);
assertRowsEquals(rows3, TestData.DATA_SET_SOURCE_INSERT);
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testStreamWriteAndRead(HoodieTableType tableType) throws Exception {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
// reading from the latest commit instance.
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT);
// insert another batch of data
execInsertSql(streamTableEnv, insertInto);
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows2, TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT);
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testStreamReadAppendData(HoodieTableType tableType) throws Exception {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
String createSource2 = TestConfigurations.getFileSourceDDL("source2", "test_source_2.data");
streamTableEnv.executeSql(createSource);
streamTableEnv.executeSql(createSource2);
String createHoodieTable = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
streamTableEnv.executeSql(createHoodieTable);
String insertInto = "insert into t1 select * from source";
// execute 2 times
execInsertSql(streamTableEnv, insertInto);
// remember the commit
String specifiedCommit = TestUtils.getFirstCompleteInstant(tempFile.getAbsolutePath());
// another update batch
String insertInto2 = "insert into t1 select * from source2";
execInsertSql(streamTableEnv, insertInto2);
// now we consume starting from the oldest commit
String createHoodieTable2 = sql("t2")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_START_COMMIT, specifiedCommit)
.end();
streamTableEnv.executeSql(createHoodieTable2);
List<Row> rows = execSelectSql(streamTableEnv, "select * from t2", 10);
// all the data with same keys are appended within one data bucket and one log file,
// so when consume, the same keys are merged
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_MERGED);
}
@Test
void testStreamWriteBatchRead() {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
List<Row> rows = CollectionUtil.iterableToList(
() -> streamTableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
}
@Test
void testStreamWriteBatchReadOptimized() {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
// read optimized is supported for both MOR and COR table,
// test MOR streaming write with compaction then reads as
// query type 'read_optimized'.
.option(FlinkOptions.QUERY_TYPE, FlinkOptions.QUERY_TYPE_READ_OPTIMIZED)
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, 1)
.option(FlinkOptions.COMPACTION_TASKS, 1)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
List<Row> rows = CollectionUtil.iterableToList(
() -> streamTableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
}
@Test
void testStreamWriteReadSkippingCompaction() throws Exception {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source");
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_SKIP_COMPACT, true)
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, 1)
.option(FlinkOptions.COMPACTION_TASKS, 1)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT);
}
@Test
void testStreamWriteWithCleaning() {
// create filesystem table named source
// the source generates 4 commits but the cleaning task
// would always try to keep the remaining commits number as 1
String createSource = TestConfigurations.getFileSourceDDL(
"source", "test_source_3.data", 4);
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.CLEAN_RETAIN_COMMITS, 1)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
Configuration defaultConf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
Map<String, String> options1 = new HashMap<>(defaultConf.toMap());
options1.put(FlinkOptions.TABLE_NAME.key(), "t1");
Configuration conf = Configuration.fromMap(options1);
HoodieTimeline timeline = StreamerUtil.createMetaClient(conf).getActiveTimeline();
assertTrue(timeline.filterCompletedInstants()
.getInstants().anyMatch(instant -> instant.getAction().equals("clean")),
"some commits should be cleaned");
}
@Test
void testStreamReadWithDeletes() throws Exception {
// create filesystem table named source
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.TABLE_NAME, "t1");
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
conf.setBoolean(FlinkOptions.CHANGELOG_ENABLED, true);
// write one commit
TestData.writeData(TestData.DATA_SET_INSERT, conf);
// write another commit with deletes
TestData.writeData(TestData.DATA_SET_UPDATE_DELETE, conf);
String latestCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
.option(FlinkOptions.READ_START_COMMIT, latestCommit)
.option(FlinkOptions.CHANGELOG_ENABLED, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
final String sinkDDL = "create table sink(\n"
+ " name varchar(20),\n"
+ " age_sum int\n"
+ ") with (\n"
+ " 'connector' = '" + CollectSinkTableFactory.FACTORY_ID + "'"
+ ")";
List<Row> result = execSelectSql(streamTableEnv,
"select name, sum(age) from t1 group by name", sinkDDL, 10);
final String expected = "[+I(+I[Danny, 24]), +I(+I[Stephen, 34])]";
assertRowsEquals(result, expected, true);
}
@ParameterizedTest
@MethodSource("tableTypeAndPartitioningParams")
void testStreamReadFilterByPartition(HoodieTableType tableType, boolean hiveStylePartitioning) throws Exception {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.TABLE_NAME, "t1");
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning);
// write one commit
TestData.writeData(TestData.DATA_SET_INSERT, conf);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
List<Row> result = execSelectSql(streamTableEnv,
"select * from t1 where `partition`='par1'", 10);
final String expected = "["
+ "+I(+I[id1, Danny, 23, 1970-01-01T00:00:00.001, par1]), "
+ "+I(+I[id2, Stephen, 33, 1970-01-01T00:00:00.002, par1])]";
assertRowsEquals(result, expected, true);
}
@Test
void testStreamReadMorTableWithCompactionPlan() throws Exception {
String createSource = TestConfigurations.getFileSourceDDL("source");
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
// close the async compaction
.option(FlinkOptions.COMPACTION_ASYNC_ENABLED, false)
// generate compaction plan for each commit
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, 1)
.noPartition()
.end();
streamTableEnv.executeSql(hoodieTableDDL);
streamTableEnv.executeSql("insert into t1 select * from source");
List<Row> result = execSelectSql(streamTableEnv, "select * from t1", 10);
final String expected = "["
+ "+I[id1, Danny, 23, 1970-01-01T00:00:01, par1], "
+ "+I[id2, Stephen, 33, 1970-01-01T00:00:02, par1], "
+ "+I[id3, Julian, 53, 1970-01-01T00:00:03, par2], "
+ "+I[id4, Fabian, 31, 1970-01-01T00:00:04, par2], "
+ "+I[id5, Sophia, 18, 1970-01-01T00:00:05, par3], "
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]";
assertRowsEquals(result, expected);
}
@ParameterizedTest
@MethodSource("executionModeAndPartitioningParams")
void testWriteAndRead(ExecMode execMode, boolean hiveStylePartitioning) {
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_T1);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// apply filters
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1 where uuid > 'id5'").execute().collect());
assertRowsEquals(result2, "["
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
@ParameterizedTest
@MethodSource("tableTypeAndPartitioningParams")
void testWriteAndReadWithProctimeSequence(HoodieTableType tableType, boolean hiveStylePartitioning) {
TableEnvironment tableEnv = batchTableEnv;
String hoodieTableDDL = sql("t1")
.field("uuid varchar(20)")
.field("name varchar(10)")
.field("age int")
.field("tss timestamp(3)") // use a different field with default precombine field 'ts'
.field("`partition` varchar(10)")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_SAME_KEY_T1);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, "[+I[id1, Danny, 23, 1970-01-01T00:00:01, par1]]");
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testBatchModeUpsertWithoutPartition(HoodieTableType tableType) {
TableEnvironment tableEnv = batchTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_NAME, tableType.name())
.option("hoodie.parquet.small.file.limit", "0") // invalidate the small file strategy
.option("hoodie.parquet.max.file.size", "0")
.noPartition()
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_T1);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// batchMode update
execInsertSql(tableEnv, TestSQL.UPDATE_INSERT_T1);
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result2, TestData.DATA_SET_SOURCE_MERGED);
}
@ParameterizedTest
@MethodSource("tableTypeAndPartitioningParams")
void testBatchModeUpsert(HoodieTableType tableType, boolean hiveStylePartitioning) {
TableEnvironment tableEnv = batchTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_NAME, tableType)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_T1);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// batchMode update
execInsertSql(tableEnv, TestSQL.UPDATE_INSERT_T1);
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result2, TestData.DATA_SET_SOURCE_MERGED);
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testWriteAndReadParMiddle(ExecMode execMode) throws Exception {
boolean streaming = execMode == ExecMode.STREAM;
String hoodieTableDDL = "create table t1(\n"
+ " uuid varchar(20),\n"
+ " name varchar(10),\n"
+ " age int,\n"
+ " `partition` varchar(20),\n" // test streaming read with partition field in the middle
+ " ts timestamp(3),\n"
+ " PRIMARY KEY(uuid) NOT ENFORCED\n"
+ ")\n"
+ "PARTITIONED BY (`partition`)\n"
+ "with (\n"
+ " 'connector' = 'hudi',\n"
+ " 'path' = '" + tempFile.getAbsolutePath() + "',\n"
+ " 'read.streaming.enabled' = '" + streaming + "'\n"
+ ")";
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 values\n"
+ "('id1','Danny',23,'par1',TIMESTAMP '1970-01-01 00:00:01'),\n"
+ "('id2','Stephen',33,'par1',TIMESTAMP '1970-01-01 00:00:02'),\n"
+ "('id3','Julian',53,'par2',TIMESTAMP '1970-01-01 00:00:03'),\n"
+ "('id4','Fabian',31,'par2',TIMESTAMP '1970-01-01 00:00:04'),\n"
+ "('id5','Sophia',18,'par3',TIMESTAMP '1970-01-01 00:00:05'),\n"
+ "('id6','Emma',20,'par3',TIMESTAMP '1970-01-01 00:00:06'),\n"
+ "('id7','Bob',44,'par4',TIMESTAMP '1970-01-01 00:00:07'),\n"
+ "('id8','Han',56,'par4',TIMESTAMP '1970-01-01 00:00:08')";
execInsertSql(streamTableEnv, insertInto);
final String expected = "["
+ "+I[id1, Danny, 23, par1, 1970-01-01T00:00:01], "
+ "+I[id2, Stephen, 33, par1, 1970-01-01T00:00:02], "
+ "+I[id3, Julian, 53, par2, 1970-01-01T00:00:03], "
+ "+I[id4, Fabian, 31, par2, 1970-01-01T00:00:04], "
+ "+I[id5, Sophia, 18, par3, 1970-01-01T00:00:05], "
+ "+I[id6, Emma, 20, par3, 1970-01-01T00:00:06], "
+ "+I[id7, Bob, 44, par4, 1970-01-01T00:00:07], "
+ "+I[id8, Han, 56, par4, 1970-01-01T00:00:08]]";
List<Row> result = execSelectSql(streamTableEnv, "select * from t1", execMode);
assertRowsEquals(result, expected);
// insert another batch of data
execInsertSql(streamTableEnv, insertInto);
List<Row> result2 = execSelectSql(streamTableEnv, "select * from t1", execMode);
assertRowsEquals(result2, expected);
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testInsertOverwrite(ExecMode execMode) {
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_T1);
// overwrite partition 'par1' and increase in age by 1
final String insertInto2 = "insert overwrite t1 partition(`partition`='par1') values\n"
+ "('id1','Danny',24,TIMESTAMP '1970-01-01 00:00:01'),\n"
+ "('id2','Stephen',34,TIMESTAMP '1970-01-01 00:00:02')\n";
execInsertSql(tableEnv, insertInto2);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT_OVERWRITE);
// overwrite the whole table
final String insertInto3 = "insert overwrite t1 values\n"
+ "('id1','Danny',24,TIMESTAMP '1970-01-01 00:00:01', 'par1'),\n"
+ "('id2','Stephen',34,TIMESTAMP '1970-01-01 00:00:02', 'par2')\n";
execInsertSql(tableEnv, insertInto3);
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
final String expected = "["
+ "+I[id1, Danny, 24, 1970-01-01T00:00:01, par1], "
+ "+I[id2, Stephen, 34, 1970-01-01T00:00:02, par2]]";
assertRowsEquals(result2, expected);
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testStreamWriteAndReadWithMiniBatches(HoodieTableType tableType) throws Exception {
// create filesystem table named source
String createSource = TestConfigurations.getFileSourceDDL("source", 4);
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_START_COMMIT, "earliest")
.option(FlinkOptions.WRITE_BATCH_SIZE, 0.00001)
.noPartition()
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
// reading from the earliest commit instance.
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 20);
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
}
@ParameterizedTest
@MethodSource("executionModeAndTableTypeParams")
void testBatchUpsertWithMiniBatches(ExecMode execMode, HoodieTableType tableType) {
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.WRITE_BATCH_SIZE, "0.001")
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
tableEnv.executeSql(hoodieTableDDL);
final String insertInto1 = "insert into t1 values\n"
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1')";
execInsertSql(tableEnv, insertInto1);
final String insertInto2 = "insert into t1 values\n"
+ "('id1','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
+ "('id1','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par1'),\n"
+ "('id1','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par1'),\n"
+ "('id1','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par1')";
execInsertSql(tableEnv, insertInto2);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, "[+I[id1, Sophia, 18, 1970-01-01T00:00:05, par1]]");
}
@ParameterizedTest
@MethodSource("executionModeAndTableTypeParams")
void testBatchUpsertWithMiniBatchesGlobalIndex(ExecMode execMode, HoodieTableType tableType) {
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.WRITE_BATCH_SIZE, "0.001")
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, true)
.end();
tableEnv.executeSql(hoodieTableDDL);
final String insertInto1 = "insert into t1 values\n"
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1')";
execInsertSql(tableEnv, insertInto1);
final String insertInto2 = "insert into t1 values\n"
+ "('id1','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par2'),\n"
+ "('id1','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par1'),\n"
+ "('id1','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),\n"
+ "('id1','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3')";
execInsertSql(tableEnv, insertInto2);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, "[+I[id1, Sophia, 18, 1970-01-01T00:00:05, par3]]");
}
@Test
void testUpdateWithDefaultHoodieRecordPayload() {
TableEnvironment tableEnv = batchTableEnv;
String hoodieTableDDL = sql("t1")
.field("id int")
.field("name string")
.field("price double")
.field("ts bigint")
.pkField("id")
.noPartition()
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName())
.end();
tableEnv.executeSql(hoodieTableDDL);
final String insertInto1 = "insert into t1 values\n"
+ "(1,'a1',20,20)";
execInsertSql(tableEnv, insertInto1);
final String insertInto4 = "insert into t1 values\n"
+ "(1,'a1',20,1)";
execInsertSql(tableEnv, insertInto4);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, "[+I[1, a1, 20.0, 20]]");
}
@ParameterizedTest
@MethodSource("executionModeAndTableTypeParams")
void testWriteNonPartitionedTable(ExecMode execMode, HoodieTableType tableType) {
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType)
.noPartition()
.end();
tableEnv.executeSql(hoodieTableDDL);
final String insertInto1 = "insert into t1 values\n"
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1')";
execInsertSql(tableEnv, insertInto1);
final String insertInto2 = "insert into t1 values\n"
+ "('id1','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par2'),\n"
+ "('id1','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par3'),\n"
+ "('id1','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par4'),\n"
+ "('id1','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par5')";
execInsertSql(tableEnv, insertInto2);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, "[+I[id1, Sophia, 18, 1970-01-01T00:00:05, par5]]");
}
@Test
void testWriteGlobalIndex() {
// the source generates 4 commits
String createSource = TestConfigurations.getFileSourceDDL(
"source", "test_source_4.data", 4);
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, true)
.option(FlinkOptions.PRE_COMBINE, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
final String insertInto2 = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto2);
List<Row> result = CollectionUtil.iterableToList(
() -> streamTableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, "[+I[id1, Phoebe, 52, 1970-01-01T00:00:08, par4]]");
}
@Test
void testWriteLocalIndex() {
// the source generates 4 commits
String createSource = TestConfigurations.getFileSourceDDL(
"source", "test_source_4.data", 4);
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, false)
.option(FlinkOptions.PRE_COMBINE, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
final String insertInto2 = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto2);
List<Row> result = CollectionUtil.iterableToList(
() -> streamTableEnv.sqlQuery("select * from t1").execute().collect());
final String expected = "["
+ "+I[id1, Stephen, 34, 1970-01-01T00:00:02, par1], "
+ "+I[id1, Fabian, 32, 1970-01-01T00:00:04, par2], "
+ "+I[id1, Jane, 19, 1970-01-01T00:00:06, par3], "
+ "+I[id1, Phoebe, 52, 1970-01-01T00:00:08, par4]]";
assertRowsEquals(result, expected, 3);
}
@Test
void testStreamReadEmptyTablePath() throws Exception {
// case1: table metadata path does not exists
// create a flink source table
String createHoodieTable = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.end();
streamTableEnv.executeSql(createHoodieTable);
// no exception expects to be thrown
List<Row> rows1 = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows1, "[]");
// case2: empty table without data files
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
StreamerUtil.initTableIfNotExists(conf);
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows2, "[]");
}
@Test
void testBatchReadEmptyTablePath() throws Exception {
// case1: table metadata path does not exists
// create a flink source table
String createHoodieTable = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.end();
batchTableEnv.executeSql(createHoodieTable);
// no exception expects to be thrown
assertThrows(Exception.class,
() -> execSelectSql(batchTableEnv, "select * from t1", 10),
"Exception should throw when querying non-exists table in batch mode");
// case2: empty table without data files
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
StreamerUtil.initTableIfNotExists(conf);
List<Row> rows2 = CollectionUtil.iteratorToList(batchTableEnv.executeSql("select * from t1").collect());
assertRowsEquals(rows2, "[]");
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testWriteAndReadDebeziumJson(ExecMode execMode) throws Exception {
String sourcePath = Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource("debezium_json.data")).toString();
String sourceDDL = ""
+ "CREATE TABLE debezium_source(\n"
+ " id INT NOT NULL,\n"
+ " ts BIGINT,\n"
+ " name STRING,\n"
+ " description STRING,\n"
+ " weight DOUBLE\n"
+ ") WITH (\n"
+ " 'connector' = 'filesystem',\n"
+ " 'path' = '" + sourcePath + "',\n"
+ " 'format' = 'debezium-json'\n"
+ ")";
streamTableEnv.executeSql(sourceDDL);
String hoodieTableDDL = sql("hoodie_sink")
.field("id INT NOT NULL")
.field("ts BIGINT")
.field("name STRING")
.field("weight DOUBLE")
.pkField("id")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, execMode == ExecMode.STREAM)
.option(FlinkOptions.PRE_COMBINE, true)
.noPartition()
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into hoodie_sink select id, ts, name, weight from debezium_source";
execInsertSql(streamTableEnv, insertInto);
final String expected = "["
+ "+I[101, 1000, scooter, 3.140000104904175], "
+ "+I[102, 2000, car battery, 8.100000381469727], "
+ "+I[103, 3000, 12-pack drill bits, 0.800000011920929], "
+ "+I[104, 4000, hammer, 0.75], "
+ "+I[105, 5000, hammer, 0.875], "
+ "+I[106, 10000, hammer, 1.0], "
+ "+I[107, 11000, rocks, 5.099999904632568], "
+ "+I[108, 8000, jacket, 0.10000000149011612], "
+ "+I[109, 9000, spare tire, 22.200000762939453], "
+ "+I[110, 14000, jacket, 0.5]]";
List<Row> result = execSelectSql(streamTableEnv, "select * from hoodie_sink", execMode);
assertRowsEquals(result, expected);
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
void testBulkInsert(boolean hiveStylePartitioning) {
TableEnvironment tableEnv = batchTableEnv;
// csv source
String csvSourceDDL = TestConfigurations.getCsvSourceDDL("csv_source", "test_source_5.data");
tableEnv.executeSql(csvSourceDDL);
String hoodieTableDDL = sql("hoodie_sink")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.OPERATION, "bulk_insert")
.option(FlinkOptions.WRITE_BULK_INSERT_SHUFFLE_BY_PARTITION, true)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
tableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into hoodie_sink select * from csv_source";
execInsertSql(tableEnv, insertInto);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from hoodie_sink").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// apply filters
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from hoodie_sink where uuid > 'id5'").execute().collect());
assertRowsEquals(result2, "["
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
@Test
void testBulkInsertNonPartitionedTable() {
TableEnvironment tableEnv = batchTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.OPERATION, "bulk_insert")
.noPartition()
.end();
tableEnv.executeSql(hoodieTableDDL);
final String insertInto1 = "insert into t1 values\n"
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1')";
execInsertSql(tableEnv, insertInto1);
final String insertInto2 = "insert into t1 values\n"
+ "('id1','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par2'),\n"
+ "('id1','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par3'),\n"
+ "('id1','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par4'),\n"
+ "('id1','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par5')";
execInsertSql(tableEnv, insertInto2);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, "["
+ "+I[id1, Danny, 23, 1970-01-01T00:00:01, par1], "
+ "+I[id1, Stephen, 33, 1970-01-01T00:00:02, par2], "
+ "+I[id1, Julian, 53, 1970-01-01T00:00:03, par3], "
+ "+I[id1, Fabian, 31, 1970-01-01T00:00:04, par4], "
+ "+I[id1, Sophia, 18, 1970-01-01T00:00:05, par5]]", 3);
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
void testAppendWrite(boolean clustering) {
TableEnvironment tableEnv = streamTableEnv;
// csv source
String csvSourceDDL = TestConfigurations.getCsvSourceDDL("csv_source", "test_source_5.data");
tableEnv.executeSql(csvSourceDDL);
String hoodieTableDDL = sql("hoodie_sink")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.OPERATION, "insert")
.option(FlinkOptions.INSERT_CLUSTER, clustering)
.end();
tableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into hoodie_sink select * from csv_source";
execInsertSql(tableEnv, insertInto);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from hoodie_sink").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// apply filters
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from hoodie_sink where uuid > 'id5'").execute().collect());
assertRowsEquals(result2, "["
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testWriteAndReadWithTimestampPartitioning(ExecMode execMode) {
// can not read the hive style and timestamp based partitioning table
// in batch mode, the code path in CopyOnWriteInputFormat relies on
// the value on the partition path to recover the partition value,
// but the date format has changed(milliseconds switch to hours).
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.partitionField("ts") // use timestamp as partition path field
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_T1);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// apply filters
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1 where uuid > 'id5'").execute().collect());
assertRowsEquals(result2, "["
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
@Test
void testWriteReadDecimals() {
TableEnvironment tableEnv = batchTableEnv;
String createTable = sql("decimals")
.field("f0 decimal(3, 2)")
.field("f1 decimal(10, 2)")
.field("f2 decimal(20, 2)")
.field("f3 decimal(38, 18)")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.OPERATION, "bulk_insert")
.option(FlinkOptions.PRECOMBINE_FIELD, "f1")
.pkField("f0")
.noPartition()