cshuo commented on code in PR #17562: URL: https://github.com/apache/hudi/pull/17562#discussion_r2617776449
########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java: ########## @@ -0,0 +1,134 @@ +/* + * 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.source.enumerator; + +import org.apache.hudi.source.ScanContext; +import org.apache.hudi.source.split.HoodieContinuousSplitBatch; +import org.apache.hudi.source.split.HoodieContinuousSplitDiscover; +import org.apache.hudi.source.split.HoodieSourceSplit; +import org.apache.hudi.source.split.HoodieSplitProvider; + +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi commits of upstream Hoodie table. + */ +public class HoodieContinuousSplitEnumerator extends AbstractHoodieSplitEnumerator { + private static final Logger LOG = LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class); + + private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext; + private final HoodieSplitProvider splitProvider; + private final HoodieContinuousSplitDiscover splitDiscover; + private final ScanContext scanContext; + + /** + * Instant for the last enumerated commit. Next incremental enumeration should be based off + * this as the starting position. + */ + private final AtomicReference<HoodieEnumeratorPosition> position; + + public HoodieContinuousSplitEnumerator( + SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext, + HoodieSplitProvider splitProvider, + HoodieContinuousSplitDiscover splitDiscover, + ScanContext scanContext, + @Nullable HoodieSplitEnumeratorState enumState) { + super(enumeratorContext, splitProvider); + this.enumeratorContext = enumeratorContext; + this.splitProvider = splitProvider; + this.splitDiscover = splitDiscover; + this.scanContext = scanContext; + this.position = new AtomicReference<>(); + + if (enumState != null) { + this.position.set(HoodieEnumeratorPosition.of(enumState.getLastEnumeratedInstant(), enumState.getLastEnumeratedInstantCompletionTime())); + } else { + this.position.set(HoodieEnumeratorPosition.of(scanContext.getStartInstant(), scanContext.getStartInstant())); + } + } + + @Override + public void start() { + super.start(); + enumeratorContext.callAsync( + this::discoverSplits, + this::processDiscoveredSplits, + 0L, + scanContext.getScanInterval().toMillis()); + } + + @Override + boolean shouldWaitForMoreSplits() { + return true; + } + + @Override + public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws Exception { + return new HoodieSplitEnumeratorState(splitProvider.state(), position.get().lastInstant(), position.get().lastInstantCompletionTime()); + } + + private HoodieContinuousSplitBatch discoverSplits() { + int pendingSplitNumber = splitProvider.pendingSplitCount(); + if (pendingSplitNumber > scanContext.getMaxPendingSplits()) { + LOG.info( + "Pause split discovery as the assigner already has too many pending splits: {}", + pendingSplitNumber); + return HoodieContinuousSplitBatch.EMPTY; + } + return splitDiscover.discoverSplits(position.get().lastInstantCompletionTime()); + } + + private void processDiscoveredSplits(HoodieContinuousSplitBatch result, Throwable throwable) { + if (throwable == null) { + if (!Objects.equals(result.getFromInstant(), position.get().lastInstantCompletionTime())) { + LOG.info( + "Skip {} discovered splits because the scan starting position doesn't match " + + "the current enumerator position: enumerator position = {}, scan starting position = {}", + result.getSplits().size(), + position.get().lastInstantCompletionTime(), + result.getFromInstant()); + } else { + if (!result.getSplits().isEmpty()) { + splitProvider.onDiscoveredSplits(result.getSplits()); + LOG.info( + "Added {} splits discovered between ({}, {}] to the assigner", + result.getSplits().size(), + result.getFromInstant(), + result.getToInstant()); + } else { + LOG.info( + "No new splits discovered between ({}, {}]", + result.getFromInstant(), + result.getToInstant()); + } + position.set(HoodieEnumeratorPosition.of(position.get().lastInstantCompletionTime(), result.getToInstant())); Review Comment: The constructor expects `(lastInstant, lastInstantCompletionTime)`, but the code provides `(lastInstantCompletionTime, instant)`, should be reversed? ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java: ########## @@ -0,0 +1,134 @@ +/* + * 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.source.enumerator; + +import org.apache.hudi.source.ScanContext; +import org.apache.hudi.source.split.HoodieContinuousSplitBatch; +import org.apache.hudi.source.split.HoodieContinuousSplitDiscover; +import org.apache.hudi.source.split.HoodieSourceSplit; +import org.apache.hudi.source.split.HoodieSplitProvider; + +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi commits of upstream Hoodie table. + */ +public class HoodieContinuousSplitEnumerator extends AbstractHoodieSplitEnumerator { + private static final Logger LOG = LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class); + + private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext; + private final HoodieSplitProvider splitProvider; + private final HoodieContinuousSplitDiscover splitDiscover; + private final ScanContext scanContext; + + /** + * Instant for the last enumerated commit. Next incremental enumeration should be based off + * this as the starting position. + */ + private final AtomicReference<HoodieEnumeratorPosition> position; + + public HoodieContinuousSplitEnumerator( + SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext, + HoodieSplitProvider splitProvider, + HoodieContinuousSplitDiscover splitDiscover, + ScanContext scanContext, + @Nullable HoodieSplitEnumeratorState enumState) { + super(enumeratorContext, splitProvider); + this.enumeratorContext = enumeratorContext; + this.splitProvider = splitProvider; + this.splitDiscover = splitDiscover; + this.scanContext = scanContext; + this.position = new AtomicReference<>(); + + if (enumState != null) { + this.position.set(HoodieEnumeratorPosition.of(enumState.getLastEnumeratedInstant(), enumState.getLastEnumeratedInstantCompletionTime())); + } else { + this.position.set(HoodieEnumeratorPosition.of(scanContext.getStartInstant(), scanContext.getStartInstant())); + } + } + + @Override + public void start() { + super.start(); + enumeratorContext.callAsync( + this::discoverSplits, + this::processDiscoveredSplits, + 0L, + scanContext.getScanInterval().toMillis()); + } + + @Override + boolean shouldWaitForMoreSplits() { + return true; + } + + @Override + public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws Exception { + return new HoodieSplitEnumeratorState(splitProvider.state(), position.get().lastInstant(), position.get().lastInstantCompletionTime()); + } + + private HoodieContinuousSplitBatch discoverSplits() { + int pendingSplitNumber = splitProvider.pendingSplitCount(); + if (pendingSplitNumber > scanContext.getMaxPendingSplits()) { + LOG.info( + "Pause split discovery as the assigner already has too many pending splits: {}", + pendingSplitNumber); + return HoodieContinuousSplitBatch.EMPTY; + } + return splitDiscover.discoverSplits(position.get().lastInstantCompletionTime()); + } + + private void processDiscoveredSplits(HoodieContinuousSplitBatch result, Throwable throwable) { + if (throwable == null) { + if (!Objects.equals(result.getFromInstant(), position.get().lastInstantCompletionTime())) { + LOG.info( + "Skip {} discovered splits because the scan starting position doesn't match " Review Comment: Curious about when will this happen, I notice the `fromInstant` of `result` is just assigned as `position.get().lastInstantCompletionTime()` in `discoverSplits`, so this checking will never happen? ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/ScanContext.java: ########## @@ -0,0 +1,197 @@ +/* + * 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.source; + +import org.apache.hudi.configuration.FlinkOptions; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.types.logical.RowType; + +import java.io.Serializable; +import java.time.Duration; + +/** + * Hudi source scan context. + */ +@Internal +public class ScanContext implements Serializable { + private final Configuration conf; + private final Path path; + private final RowType rowType; + private final String startInstant; + private final long maxCompactionMemoryInBytes; + private final long maxPendingSplits; + // skip compaction + private final boolean skipCompaction; + // skip clustering + private final boolean skipClustering; + // skip insert overwrite + private final boolean skipInsertOverwrite; + // cdc enabled + private final boolean cdcEnabled; + + public ScanContext( + Configuration conf, + Path path, + RowType rowType, + String startInstant, + long maxCompactionMemoryInBytes, + long maxPendingSplits, + boolean skipCompaction, + boolean skipClustering, + boolean skipInsertOverwrite, + boolean cdcEnabled) { + this.conf = conf; + this.path = path; + this.rowType = rowType; + this.startInstant = startInstant; + this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes; + this.maxPendingSplits = maxPendingSplits; + this.skipCompaction = skipCompaction; + this.skipClustering = skipClustering; + this.skipInsertOverwrite = skipInsertOverwrite; + this.cdcEnabled = cdcEnabled; + } + + public Configuration getConf() { + return conf; + } + + public Path getPath() { + return path; + } + + public RowType getRowType() { + return rowType; + } + + public String getStartInstant() { + return startInstant; + } + + public long getMaxCompactionMemoryInBytes() { + return maxCompactionMemoryInBytes; + } + + public long getMaxPendingSplits() { + return maxPendingSplits; + } + + public boolean skipCompaction() { + return skipCompaction; + } + + public boolean skipClustering() { + return skipClustering; + } + + public boolean skipInsertOverwrite() { + return skipInsertOverwrite; + } + + public boolean cdcEnabled() { + return cdcEnabled; + } + + public Duration getScanInterval() { + return Duration.ofMinutes(conf.get(FlinkOptions.READ_STREAMING_CHECK_INTERVAL)); Review Comment: should be `ofSeconds` ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/split/HoodieContinuousSplitBatch.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.source.split; + +import org.apache.flink.util.Preconditions; + +import java.util.Collection; +import java.util.Collections; + +/** + * Result from continuous enumerator. + */ +public class HoodieContinuousSplitBatch { + public static HoodieContinuousSplitBatch EMPTY = new HoodieContinuousSplitBatch(Collections.emptyList(), "", ""); Review Comment: shoud be `final`? ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieEnumeratorPosition.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.source.enumerator; + +import java.io.Serializable; +import java.util.Objects; + +/*** + * The position of {@link HoodieContinuousSplitEnumerator}. + */ +public class HoodieEnumeratorPosition implements Serializable { + private final String lastInstant; Review Comment: `lastInstant` maybe not needed now, because we now using completion time based incremental reading, and `lastInstant` in `StreamReadMonitoringFunction` is only used for backward compatibility for flink state, which should be a problem for the new flip-27 source. -- 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]
