1919
2020import com .alibaba .fluss .client .admin .Admin ;
2121import com .alibaba .fluss .client .metadata .LakeSnapshot ;
22+ import com .alibaba .fluss .flink .lake .split .LakeSnapshotAndFlussLogSplit ;
2223import com .alibaba .fluss .flink .lake .split .LakeSnapshotSplit ;
23- import com .alibaba .fluss .flink .lakehouse .paimon .split .PaimonSnapshotAndFlussLogSplit ;
2424import com .alibaba .fluss .flink .source .enumerator .initializer .OffsetsInitializer ;
2525import com .alibaba .fluss .flink .source .split .LogSplit ;
2626import com .alibaba .fluss .flink .source .split .SourceSplitBase ;
3030import com .alibaba .fluss .metadata .TableBucket ;
3131import com .alibaba .fluss .metadata .TableInfo ;
3232
33- import org .apache .paimon .CoreOptions ;
34- import org .apache .paimon .catalog .Catalog ;
35- import org .apache .paimon .catalog .Identifier ;
36- import org .apache .paimon .flink .FlinkCatalogFactory ;
37- import org .apache .paimon .flink .source .FileStoreSourceSplit ;
38- import org .apache .paimon .flink .source .FileStoreSourceSplitGenerator ;
39- import org .apache .paimon .options .MemorySize ;
40- import org .apache .paimon .options .Options ;
41- import org .apache .paimon .table .FileStoreTable ;
42- import org .apache .paimon .table .source .InnerTableScan ;
43-
4433import javax .annotation .Nullable ;
4534
4635import java .util .ArrayList ;
5342import java .util .stream .IntStream ;
5443
5544import static com .alibaba .fluss .client .table .scanner .log .LogScanner .EARLIEST_OFFSET ;
56- import static com .alibaba .fluss .flink .utils .DataLakeUtils .extractLakeCatalogProperties ;
57- import static com .alibaba .fluss .utils .Preconditions .checkState ;
5845
5946/** A generator for lake splits. */
6047public class LakeSplitGenerator {
@@ -86,10 +73,6 @@ public List<SourceSplitBase> generateHybridLakeSplits() throws Exception {
8673 // get the file store
8774 LakeSnapshot lakeSnapshotInfo =
8875 flussAdmin .getLatestLakeSnapshot (tableInfo .getTablePath ()).get ();
89- FileStoreTable fileStoreTable =
90- getTable (
91- lakeSnapshotInfo .getSnapshotId (),
92- extractLakeCatalogProperties (tableInfo .getProperties ()));
9376
9477 boolean isLogTable = !tableInfo .hasPrimaryKey ();
9578 boolean isPartitioned = tableInfo .isPartitioned ();
@@ -113,17 +96,13 @@ public List<SourceSplitBase> generateHybridLakeSplits() throws Exception {
11396 lakeSplits ,
11497 isLogTable ,
11598 lakeSnapshotInfo .getTableBucketsOffset (),
116- partitionNameById ,
117- fileStoreTable );
99+ partitionNameById );
118100 } else {
119101 Map <Integer , List <LakeSplit >> nonPartitionLakeSplits =
120102 lakeSplits .values ().iterator ().next ();
121103 // non-partitioned table
122104 return generateNoPartitionedTableSplit (
123- nonPartitionLakeSplits ,
124- isLogTable ,
125- lakeSnapshotInfo .getTableBucketsOffset (),
126- fileStoreTable );
105+ nonPartitionLakeSplits , isLogTable , lakeSnapshotInfo .getTableBucketsOffset ());
127106 }
128107 }
129108
@@ -145,8 +124,7 @@ private List<SourceSplitBase> generatePartitionTableSplit(
145124 Map <String , Map <Integer , List <LakeSplit >>> lakeSplits ,
146125 boolean isLogTable ,
147126 Map <TableBucket , Long > tableBucketSnapshotLogOffset ,
148- Map <Long , String > partitionNameById ,
149- @ Nullable FileStoreTable fileStoreTable )
127+ Map <Long , String > partitionNameById )
150128 throws Exception {
151129 List <SourceSplitBase > splits = new ArrayList <>();
152130 Map <String , Long > flussPartitionIdByName =
@@ -181,8 +159,7 @@ private List<SourceSplitBase> generatePartitionTableSplit(
181159 partitionName ,
182160 isLogTable ,
183161 tableBucketSnapshotLogOffset ,
184- bucketEndOffset ,
185- fileStoreTable ));
162+ bucketEndOffset ));
186163
187164 } else {
188165 // only lake data
@@ -216,8 +193,7 @@ private List<SourceSplitBase> generatePartitionTableSplit(
216193 isLogTable ,
217194 // pass empty map since we won't read lake splits
218195 Collections .emptyMap (),
219- bucketEndOffset ,
220- fileStoreTable ));
196+ bucketEndOffset ));
221197 }
222198 return splits ;
223199 }
@@ -228,8 +204,7 @@ private List<SourceSplitBase> generateSplit(
228204 @ Nullable String partitionName ,
229205 boolean isLogTable ,
230206 Map <TableBucket , Long > tableBucketSnapshotLogOffset ,
231- Map <Integer , Long > bucketEndOffset ,
232- @ Nullable FileStoreTable fileStoreTable ) {
207+ Map <Integer , Long > bucketEndOffset ) {
233208 List <SourceSplitBase > splits = new ArrayList <>();
234209 if (isLogTable ) {
235210 if (lakeSplits != null ) {
@@ -264,12 +239,9 @@ private List<SourceSplitBase> generateSplit(
264239 new TableBucket (tableInfo .getTableId (), partitionId , bucket );
265240 Long snapshotLogOffset = tableBucketSnapshotLogOffset .get (tableBucket );
266241 long stoppingOffset = bucketEndOffset .get (bucket );
267- FileStoreSourceSplitGenerator splitGenerator = new FileStoreSourceSplitGenerator ();
268-
269242 splits .add (
270243 generateSplitForPrimaryKeyTableBucket (
271- fileStoreTable ,
272- splitGenerator ,
244+ lakeSplits != null ? lakeSplits .get (bucket ) : null ,
273245 tableBucket ,
274246 partitionName ,
275247 snapshotLogOffset ,
@@ -295,83 +267,26 @@ private List<SourceSplitBase> toLakeSnapshotSplits(
295267 }
296268
297269 private SourceSplitBase generateSplitForPrimaryKeyTableBucket (
298- FileStoreTable fileStoreTable ,
299- FileStoreSourceSplitGenerator splitGenerator ,
270+ @ Nullable List <LakeSplit > lakeSplits ,
300271 TableBucket tableBucket ,
301272 @ Nullable String partitionName ,
302273 @ Nullable Long snapshotLogOffset ,
303274 long stoppingOffset ) {
304-
305275 // no snapshot data for this bucket or no a corresponding log offset in this bucket,
306276 // can only scan from change log
307277 if (snapshotLogOffset == null || snapshotLogOffset < 0 ) {
308- return new PaimonSnapshotAndFlussLogSplit (
278+ return new LakeSnapshotAndFlussLogSplit (
309279 tableBucket , partitionName , null , EARLIEST_OFFSET , stoppingOffset );
310280 }
311281
312- // then, generate a split contains
313- // snapshot and change log so that we can merge change log and snapshot
314- // to get the full data
315- fileStoreTable =
316- fileStoreTable .copy (
317- Collections .singletonMap (
318- CoreOptions .SOURCE_SPLIT_TARGET_SIZE .key (),
319- // we set a max size to make sure only one splits
320- MemorySize .MAX_VALUE .toString ()));
321- InnerTableScan tableScan =
322- fileStoreTable .newScan ().withBucketFilter ((b ) -> b == tableBucket .getBucket ());
323-
324- if (partitionName != null ) {
325- tableScan =
326- tableScan .withPartitionFilter (getPartitionSpec (fileStoreTable , partitionName ));
327- }
328-
329- List <FileStoreSourceSplit > fileStoreSourceSplits =
330- splitGenerator .createSplits (tableScan .plan ());
331-
332- checkState (fileStoreSourceSplits .size () == 1 , "Splits for primary key table must be 1." );
333- FileStoreSourceSplit fileStoreSourceSplit = fileStoreSourceSplits .get (0 );
334- return new PaimonSnapshotAndFlussLogSplit (
335- tableBucket ,
336- partitionName ,
337- fileStoreSourceSplit ,
338- snapshotLogOffset ,
339- stoppingOffset );
340- }
341-
342- private Map <String , String > getPartitionSpec (
343- FileStoreTable fileStoreTable , String partitionName ) {
344- List <String > partitionKeys = fileStoreTable .partitionKeys ();
345- checkState (
346- partitionKeys .size () == 1 ,
347- "Must only one partition key for paimon table %, but got %s, the partition keys are: " ,
348- tableInfo .getTablePath (),
349- partitionKeys .size (),
350- partitionKeys );
351- return Collections .singletonMap (partitionKeys .get (0 ), partitionName );
352- }
353-
354- private FileStoreTable getTable (long snapshotId , Map <String , String > catalogProperties )
355- throws Exception {
356- try (Catalog catalog =
357- FlinkCatalogFactory .createPaimonCatalog (Options .fromMap (catalogProperties ))) {
358- return (FileStoreTable )
359- catalog .getTable (
360- Identifier .create (
361- tableInfo .getTablePath ().getDatabaseName (),
362- tableInfo .getTablePath ().getTableName ()))
363- .copy (
364- Collections .singletonMap (
365- CoreOptions .SCAN_SNAPSHOT_ID .key (),
366- String .valueOf (snapshotId )));
367- }
282+ return new LakeSnapshotAndFlussLogSplit (
283+ tableBucket , partitionName , lakeSplits , snapshotLogOffset , stoppingOffset );
368284 }
369285
370286 private List <SourceSplitBase > generateNoPartitionedTableSplit (
371287 Map <Integer , List <LakeSplit >> lakeSplits ,
372288 boolean isLogTable ,
373- Map <TableBucket , Long > tableBucketSnapshotLogOffset ,
374- FileStoreTable fileStoreTable ) {
289+ Map <TableBucket , Long > tableBucketSnapshotLogOffset ) {
375290 // iterate all bucket
376291 // assume bucket is from 0 to bucket count
377292 Map <Integer , Long > bucketEndOffset =
@@ -380,12 +295,6 @@ private List<SourceSplitBase> generateNoPartitionedTableSplit(
380295 IntStream .range (0 , bucketCount ).boxed ().collect (Collectors .toList ()),
381296 bucketOffsetsRetriever );
382297 return generateSplit (
383- lakeSplits ,
384- null ,
385- null ,
386- isLogTable ,
387- tableBucketSnapshotLogOffset ,
388- bucketEndOffset ,
389- fileStoreTable );
298+ lakeSplits , null , null , isLogTable , tableBucketSnapshotLogOffset , bucketEndOffset );
390299 }
391300}
0 commit comments