48
48
import org .junit .jupiter .api .AfterAll ;
49
49
import org .junit .jupiter .api .Assertions ;
50
50
import org .junit .jupiter .api .BeforeAll ;
51
- import org .junit .jupiter .api .BeforeEach ;
52
51
import org .junit .jupiter .api .Test ;
53
52
import org .junit .jupiter .api .io .TempDir ;
54
53
import org .junit .jupiter .params .ParameterizedTest ;
63
62
import org .apache .hudi .common .model .HoodieTableType ;
64
63
import org .apache .hudi .common .table .timeline .HoodieInstant ;
65
64
65
+ import org .apache .iceberg .Snapshot ;
66
66
import org .apache .iceberg .Table ;
67
67
import org .apache .iceberg .hadoop .HadoopTables ;
68
68
77
77
import io .onetable .hudi .HudiSourceClientProvider ;
78
78
import io .onetable .hudi .HudiSourceConfig ;
79
79
import io .onetable .hudi .HudiTestUtil ;
80
+ import io .onetable .iceberg .IcebergSourceClientProvider ;
80
81
import io .onetable .model .storage .TableFormat ;
81
82
import io .onetable .model .sync .SyncMode ;
82
83
@@ -87,8 +88,6 @@ public class ITOneTableClient {
87
88
88
89
private static JavaSparkContext jsc ;
89
90
private static SparkSession sparkSession ;
90
- private SourceClientProvider <HoodieInstant > hudiSourceClientProvider ;
91
- private SourceClientProvider <Long > deltaSourceClientProvider ;
92
91
93
92
@ BeforeAll
94
93
public static void setupOnce () {
@@ -102,14 +101,6 @@ public static void setupOnce() {
102
101
jsc = JavaSparkContext .fromSparkContext (sparkSession .sparkContext ());
103
102
}
104
103
105
- @ BeforeEach
106
- public void setup () {
107
- hudiSourceClientProvider = new HudiSourceClientProvider ();
108
- hudiSourceClientProvider .init (jsc .hadoopConfiguration (), Collections .emptyMap ());
109
- deltaSourceClientProvider = new DeltaSourceClientProvider ();
110
- deltaSourceClientProvider .init (jsc .hadoopConfiguration (), Collections .emptyMap ());
111
- }
112
-
113
104
@ AfterAll
114
105
public static void teardown () {
115
106
if (jsc != null ) {
@@ -126,7 +117,8 @@ private static Stream<Arguments> testCasesWithPartitioningAndSyncModes() {
126
117
127
118
private static Stream <Arguments > generateTestParametersForFormatsSyncModesAndPartitioning () {
128
119
List <Arguments > arguments = new ArrayList <>();
129
- for (TableFormat sourceTableFormat : Arrays .asList (TableFormat .HUDI , TableFormat .DELTA )) {
120
+ for (TableFormat sourceTableFormat :
121
+ Arrays .asList (TableFormat .HUDI , TableFormat .DELTA , TableFormat .ICEBERG )) {
130
122
for (SyncMode syncMode : SyncMode .values ()) {
131
123
for (boolean isPartitioned : new boolean [] {true , false }) {
132
124
arguments .add (Arguments .of (sourceTableFormat , syncMode , isPartitioned ));
@@ -142,9 +134,18 @@ private static Stream<Arguments> testCasesWithSyncModes() {
142
134
143
135
private SourceClientProvider <?> getSourceClientProvider (TableFormat sourceTableFormat ) {
144
136
if (sourceTableFormat == TableFormat .HUDI ) {
137
+ SourceClientProvider <HoodieInstant > hudiSourceClientProvider = new HudiSourceClientProvider ();
138
+ hudiSourceClientProvider .init (jsc .hadoopConfiguration (), Collections .emptyMap ());
145
139
return hudiSourceClientProvider ;
146
140
} else if (sourceTableFormat == TableFormat .DELTA ) {
141
+ SourceClientProvider <Long > deltaSourceClientProvider = new DeltaSourceClientProvider ();
142
+ deltaSourceClientProvider .init (jsc .hadoopConfiguration (), Collections .emptyMap ());
147
143
return deltaSourceClientProvider ;
144
+ } else if (sourceTableFormat == TableFormat .ICEBERG ) {
145
+ SourceClientProvider <Snapshot > icebergSourceClientProvider =
146
+ new IcebergSourceClientProvider ();
147
+ icebergSourceClientProvider .init (jsc .hadoopConfiguration (), Collections .emptyMap ());
148
+ return icebergSourceClientProvider ;
148
149
} else {
149
150
throw new IllegalArgumentException ("Unsupported source format: " + sourceTableFormat );
150
151
}
@@ -183,6 +184,7 @@ public void testVariousOperations(
183
184
.tableName (tableName )
184
185
.targetTableFormats (targetTableFormats )
185
186
.tableBasePath (table .getBasePath ())
187
+ .tableDataPath (table .getDataPath ())
186
188
.hudiSourceConfig (
187
189
HudiSourceConfig .builder ()
188
190
.partitionFieldSpecConfig (oneTablePartitionConfig )
@@ -215,6 +217,7 @@ public void testVariousOperations(
215
217
.tableName (tableName )
216
218
.targetTableFormats (targetTableFormats )
217
219
.tableBasePath (tableWithUpdatedSchema .getBasePath ())
220
+ .tableDataPath (tableWithUpdatedSchema .getDataPath ())
218
221
.hudiSourceConfig (
219
222
HudiSourceConfig .builder ()
220
223
.partitionFieldSpecConfig (oneTablePartitionConfig )
@@ -254,6 +257,7 @@ public void testVariousOperations(
254
257
public void testConcurrentInsertWritesInSource (
255
258
SyncMode syncMode , PartitionConfig partitionConfig ) {
256
259
String tableName = getTableName ();
260
+ SourceClientProvider <?> sourceClientProvider = getSourceClientProvider (TableFormat .HUDI );
257
261
List <TableFormat > targetTableFormats = getOtherFormats (TableFormat .HUDI );
258
262
try (TestJavaHudiTable table =
259
263
TestJavaHudiTable .forStandardSchema (
@@ -279,11 +283,11 @@ public void testConcurrentInsertWritesInSource(
279
283
.syncMode (syncMode )
280
284
.build ();
281
285
OneTableClient oneTableClient = new OneTableClient (jsc .hadoopConfiguration ());
282
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
286
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
283
287
284
288
checkDatasetEquivalence (TableFormat .HUDI , table , targetTableFormats , 50 );
285
289
table .insertRecordsWithCommitAlreadyStarted (insertsForCommit1 , commitInstant1 , true );
286
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
290
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
287
291
checkDatasetEquivalence (TableFormat .HUDI , table , targetTableFormats , 100 );
288
292
}
289
293
}
@@ -293,7 +297,7 @@ public void testConcurrentInsertWritesInSource(
293
297
public void testConcurrentInsertsAndTableServiceWrites (
294
298
SyncMode syncMode , PartitionConfig partitionConfig ) {
295
299
HoodieTableType tableType = HoodieTableType .MERGE_ON_READ ;
296
-
300
+ SourceClientProvider <?> sourceClientProvider = getSourceClientProvider ( TableFormat . HUDI );
297
301
List <TableFormat > targetTableFormats = getOtherFormats (TableFormat .HUDI );
298
302
String tableName = getTableName ();
299
303
try (TestSparkHudiTable table =
@@ -313,15 +317,15 @@ public void testConcurrentInsertsAndTableServiceWrites(
313
317
.syncMode (syncMode )
314
318
.build ();
315
319
OneTableClient oneTableClient = new OneTableClient (jsc .hadoopConfiguration ());
316
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
320
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
317
321
checkDatasetEquivalence (TableFormat .HUDI , table , targetTableFormats , 50 );
318
322
319
323
table .deleteRecords (insertedRecords1 .subList (0 , 20 ), true );
320
324
// At this point table should have 30 records but only after compaction.
321
325
String scheduledCompactionInstant = table .onlyScheduleCompaction ();
322
326
323
327
table .insertRecords (50 , true );
324
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
328
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
325
329
Map <String , String > sourceHudiOptions =
326
330
Collections .singletonMap ("hoodie.datasource.query.type" , "read_optimized" );
327
331
// Because compaction is not completed yet and read optimized query, there are 100 records.
@@ -334,7 +338,7 @@ public void testConcurrentInsertsAndTableServiceWrites(
334
338
100 );
335
339
336
340
table .insertRecords (50 , true );
337
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
341
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
338
342
// Because compaction is not completed yet and read optimized query, there are 150 records.
339
343
checkDatasetEquivalence (
340
344
TableFormat .HUDI ,
@@ -345,15 +349,15 @@ public void testConcurrentInsertsAndTableServiceWrites(
345
349
150 );
346
350
347
351
table .completeScheduledCompaction (scheduledCompactionInstant );
348
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
352
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
349
353
checkDatasetEquivalence (TableFormat .HUDI , table , targetTableFormats , 130 );
350
354
}
351
355
}
352
356
353
357
@ ParameterizedTest
354
358
@ EnumSource (
355
359
value = TableFormat .class ,
356
- names = {"HUDI" , "DELTA" })
360
+ names = {"HUDI" , "DELTA" , "ICEBERG" })
357
361
public void testTimeTravelQueries (TableFormat sourceTableFormat ) throws Exception {
358
362
String tableName = getTableName ();
359
363
try (GenericTable table =
@@ -365,6 +369,7 @@ public void testTimeTravelQueries(TableFormat sourceTableFormat) throws Exceptio
365
369
.tableName (tableName )
366
370
.targetTableFormats (targetTableFormats )
367
371
.tableBasePath (table .getBasePath ())
372
+ .tableDataPath (table .getDataPath ())
368
373
.syncMode (SyncMode .INCREMENTAL )
369
374
.build ();
370
375
SourceClientProvider <?> sourceClientProvider = getSourceClientProvider (sourceTableFormat );
@@ -462,6 +467,7 @@ public void testPartitionedData(
462
467
String hudiPartitionConfig ,
463
468
String filter ) {
464
469
String tableName = getTableName ();
470
+ SourceClientProvider <?> sourceClientProvider = getSourceClientProvider (TableFormat .HUDI );
465
471
try (TestJavaHudiTable table =
466
472
TestJavaHudiTable .forStandardSchema (
467
473
tableName , tempDir , hudiPartitionConfig , HoodieTableType .COPY_ON_WRITE )) {
@@ -478,10 +484,10 @@ public void testPartitionedData(
478
484
.build ();
479
485
table .insertRecords (100 , true );
480
486
OneTableClient oneTableClient = new OneTableClient (jsc .hadoopConfiguration ());
481
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
487
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
482
488
// Do a second sync to force the test to read back the metadata it wrote earlier
483
489
table .insertRecords (100 , true );
484
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
490
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
485
491
486
492
checkDatasetEquivalenceWithFilter (TableFormat .HUDI , table , targetTableFormats , filter );
487
493
}
@@ -491,6 +497,7 @@ public void testPartitionedData(
491
497
@ EnumSource (value = SyncMode .class )
492
498
public void testSyncWithSingleFormat (SyncMode syncMode ) {
493
499
String tableName = getTableName ();
500
+ SourceClientProvider <?> sourceClientProvider = getSourceClientProvider (TableFormat .HUDI );
494
501
try (TestJavaHudiTable table =
495
502
TestJavaHudiTable .forStandardSchema (
496
503
tableName , tempDir , null , HoodieTableType .COPY_ON_WRITE )) {
@@ -513,18 +520,18 @@ public void testSyncWithSingleFormat(SyncMode syncMode) {
513
520
.build ();
514
521
515
522
OneTableClient oneTableClient = new OneTableClient (jsc .hadoopConfiguration ());
516
- oneTableClient .sync (perTableConfigIceberg , hudiSourceClientProvider );
523
+ oneTableClient .sync (perTableConfigIceberg , sourceClientProvider );
517
524
checkDatasetEquivalence (
518
525
TableFormat .HUDI , table , Collections .singletonList (TableFormat .ICEBERG ), 100 );
519
- oneTableClient .sync (perTableConfigDelta , hudiSourceClientProvider );
526
+ oneTableClient .sync (perTableConfigDelta , sourceClientProvider );
520
527
checkDatasetEquivalence (
521
528
TableFormat .HUDI , table , Collections .singletonList (TableFormat .DELTA ), 100 );
522
529
523
530
table .insertRecords (100 , true );
524
- oneTableClient .sync (perTableConfigIceberg , hudiSourceClientProvider );
531
+ oneTableClient .sync (perTableConfigIceberg , sourceClientProvider );
525
532
checkDatasetEquivalence (
526
533
TableFormat .HUDI , table , Collections .singletonList (TableFormat .ICEBERG ), 200 );
527
- oneTableClient .sync (perTableConfigDelta , hudiSourceClientProvider );
534
+ oneTableClient .sync (perTableConfigDelta , sourceClientProvider );
528
535
checkDatasetEquivalence (
529
536
TableFormat .HUDI , table , Collections .singletonList (TableFormat .DELTA ), 200 );
530
537
}
@@ -533,6 +540,7 @@ public void testSyncWithSingleFormat(SyncMode syncMode) {
533
540
@ Test
534
541
public void testOutOfSyncIncrementalSyncs () {
535
542
String tableName = getTableName ();
543
+ SourceClientProvider <?> sourceClientProvider = getSourceClientProvider (TableFormat .HUDI );
536
544
try (TestJavaHudiTable table =
537
545
TestJavaHudiTable .forStandardSchema (
538
546
tableName , tempDir , null , HoodieTableType .COPY_ON_WRITE )) {
@@ -555,13 +563,13 @@ public void testOutOfSyncIncrementalSyncs() {
555
563
table .insertRecords (50 , true );
556
564
OneTableClient oneTableClient = new OneTableClient (jsc .hadoopConfiguration ());
557
565
// sync iceberg only
558
- oneTableClient .sync (singleTableConfig , hudiSourceClientProvider );
566
+ oneTableClient .sync (singleTableConfig , sourceClientProvider );
559
567
checkDatasetEquivalence (
560
568
TableFormat .HUDI , table , Collections .singletonList (TableFormat .ICEBERG ), 50 );
561
569
// insert more records
562
570
table .insertRecords (50 , true );
563
571
// iceberg will be an incremental sync and delta will need to bootstrap with snapshot sync
564
- oneTableClient .sync (dualTableConfig , hudiSourceClientProvider );
572
+ oneTableClient .sync (dualTableConfig , sourceClientProvider );
565
573
checkDatasetEquivalence (
566
574
TableFormat .HUDI , table , Arrays .asList (TableFormat .ICEBERG , TableFormat .DELTA ), 100 );
567
575
@@ -570,14 +578,14 @@ public void testOutOfSyncIncrementalSyncs() {
570
578
// insert more records
571
579
table .insertRecords (50 , true );
572
580
// incremental sync for two commits for iceberg only
573
- oneTableClient .sync (singleTableConfig , hudiSourceClientProvider );
581
+ oneTableClient .sync (singleTableConfig , sourceClientProvider );
574
582
checkDatasetEquivalence (
575
583
TableFormat .HUDI , table , Collections .singletonList (TableFormat .ICEBERG ), 200 );
576
584
577
585
// insert more records
578
586
table .insertRecords (50 , true );
579
587
// incremental sync for one commit for iceberg and three commits for delta
580
- oneTableClient .sync (dualTableConfig , hudiSourceClientProvider );
588
+ oneTableClient .sync (dualTableConfig , sourceClientProvider );
581
589
checkDatasetEquivalence (
582
590
TableFormat .HUDI , table , Arrays .asList (TableFormat .ICEBERG , TableFormat .DELTA ), 250 );
583
591
}
@@ -586,6 +594,7 @@ public void testOutOfSyncIncrementalSyncs() {
586
594
@ Test
587
595
public void testMetadataRetention () {
588
596
String tableName = getTableName ();
597
+ SourceClientProvider <?> sourceClientProvider = getSourceClientProvider (TableFormat .HUDI );
589
598
try (TestJavaHudiTable table =
590
599
TestJavaHudiTable .forStandardSchema (
591
600
tableName , tempDir , null , HoodieTableType .COPY_ON_WRITE )) {
@@ -599,7 +608,7 @@ public void testMetadataRetention() {
599
608
.build ();
600
609
OneTableClient oneTableClient = new OneTableClient (jsc .hadoopConfiguration ());
601
610
table .insertRecords (10 , true );
602
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
611
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
603
612
// later we will ensure we can still read the source table at this instant to ensure that
604
613
// neither target cleaned up the underlying parquet files in the table
605
614
Instant instantAfterFirstCommit = Instant .now ();
@@ -608,7 +617,7 @@ public void testMetadataRetention() {
608
617
.forEach (
609
618
unused -> {
610
619
table .insertRecords (10 , true );
611
- oneTableClient .sync (perTableConfig , hudiSourceClientProvider );
620
+ oneTableClient .sync (perTableConfig , sourceClientProvider );
612
621
});
613
622
// ensure that hudi rows can still be read and underlying files were not removed
614
623
List <Row > rows =
@@ -729,7 +738,7 @@ private void checkDatasetEquivalence(
729
738
.read ()
730
739
.options (finalTargetOptions )
731
740
.format (targetFormat .name ().toLowerCase ())
732
- .load (sourceTable .getBasePath ())
741
+ .load (sourceTable .getDataPath ())
733
742
.orderBy (sourceTable .getOrderByColumn ())
734
743
.filter (filterCondition );
735
744
}));
0 commit comments