stevenzwu commented on a change in pull request #4329: URL: https://github.com/apache/iceberg/pull/4329#discussion_r835403234
########## File path: flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java ########## @@ -0,0 +1,160 @@ +/* + * 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.iceberg.flink.source.enumerator; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.source.FlinkSplitPlanner; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { + private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); + + private final Table table; + private final ScanContext scanContext; + private final ExecutorService workerPool; + + public ContinuousSplitPlannerImpl(Table table, ScanContext scanContext, String threadPoolName) { + this.table = table; + this.scanContext = scanContext; + this.workerPool = ThreadPools.newWorkerPool( + "iceberg-plan-worker-pool-" + threadPoolName, scanContext.planParallelism()); + } + + @Override + public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { + table.refresh(); + if (lastPosition != null) { + return discoverIncrementalSplits(lastPosition); + } else { + return discoverInitialSplits(); + } + } + + /** + * Discover incremental changes between @{code lastPosition} and current table snapshot + */ + private ContinuousEnumerationResult discoverIncrementalSplits(IcebergEnumeratorPosition lastPosition) { + // incremental discovery mode + Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot.snapshotId() == lastPosition.endSnapshotId()) { + LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition); + } else { + ScanContext incrementalScan = scanContext + .copyWithAppendsBetween(lastPosition.endSnapshotId(), currentSnapshot.snapshotId()); + LOG.info("Incremental scan: startSnapshotId = {}, endSnapshotId = {}", + incrementalScan.startSnapshotId(), incrementalScan.endSnapshotId()); + List<IcebergSourceSplit> splits = FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); + IcebergEnumeratorPosition position = IcebergEnumeratorPosition.builder() + .startSnapshotId(lastPosition.endSnapshotId()) + .startSnapshotTimestampMs(lastPosition.endSnapshotTimestampMs()) + .endSnapshotId(currentSnapshot.snapshotId()) + .endSnapshotTimestampMs(currentSnapshot.timestampMillis()) + .build(); + LOG.info("Discovered {} splits from incremental scan: {}", splits.size(), position); + return new ContinuousEnumerationResult(splits, position); + } + } + + /** + * Discovery initial set of splits based on {@link StreamingStartingStrategy}. + * + * <li>{@link ContinuousEnumerationResult#splits()} should contain initial splits + * discovered from table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. + * For all other strategies, splits collection should be empty. + * <li>{@link ContinuousEnumerationResult#position()} points to the starting position + * for the next incremental split discovery with exclusive behavior. Meaning files committed + * by the snapshot from the position in {@code ContinuousEnumerationResult} won't be included + * in the next incremental scan. + */ + private ContinuousEnumerationResult discoverInitialSplits() { + Optional<Snapshot> startSnapshotOptional = getStartSnapshot(table, scanContext); + if (!startSnapshotOptional.isPresent()) { + // set IcebergEnumeratorPosition to null in this case + return new ContinuousEnumerationResult(Collections.emptyList(), null); + } Review comment: Yes. I also thought about this. This is only applicable for latest or earliest snapshot. With an empty table, the behavior is that the first snapshot (committed after the Flink source started) will be discovered as the starting snapshot. Due to the exclusive behavior, files appended by the first snapshot will be ignored. Only later files will be discovered. It is also arguable that latest or earliest snapshot (exclusive) needs to be applied to a valid snapshot. Then this behavior is correct. For specific starting snapshot id or timestamp, empty table will result in an exception since there is no snapshot matching the id or timestamp. @rdblue @kbendick what's your take? is the behavior for latest or earliest snapshot acceptable? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
