zchovan commented on code in PR #8: URL: https://github.com/apache/flink-connector-kudu/pull/8#discussion_r1965721784
########## flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSplitGenerator.java: ########## @@ -0,0 +1,88 @@ +/* + * 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.flink.connector.kudu.source.enumerator; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.source.split.KuduSourceSplit; + +import org.apache.kudu.client.AsyncKuduScanner; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduTable; + +import java.util.ArrayList; +import java.util.List; + +/** + * The class responsible for producing scan tokens for given timestamps and returning them in the + * form of {@link KuduSourceSplit}. + */ +public class KuduSplitGenerator { + private final KuduTableInfo tableInfo; + private final KuduClient kuduClient; + + public KuduSplitGenerator(KuduReaderConfig readerConfig, KuduTableInfo tableInfo) { + this.tableInfo = tableInfo; + this.kuduClient = new KuduClient.KuduClientBuilder(readerConfig.getMasters()).build(); + } + + public List<KuduSourceSplit> generateFullScanSplits(long snapshotTimestamp) { Review Comment: are there any preconditions for snapshotTimeStamp that could be checked and failing early? (e.g. null/incorrect/negative values) ########## flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSplitGenerator.java: ########## @@ -0,0 +1,88 @@ +/* + * 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.flink.connector.kudu.source.enumerator; + +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.source.split.KuduSourceSplit; + +import org.apache.kudu.client.AsyncKuduScanner; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduTable; + +import java.util.ArrayList; +import java.util.List; + +/** + * The class responsible for producing scan tokens for given timestamps and returning them in the + * form of {@link KuduSourceSplit}. + */ +public class KuduSplitGenerator { + private final KuduTableInfo tableInfo; + private final KuduClient kuduClient; + + public KuduSplitGenerator(KuduReaderConfig readerConfig, KuduTableInfo tableInfo) { + this.tableInfo = tableInfo; + this.kuduClient = new KuduClient.KuduClientBuilder(readerConfig.getMasters()).build(); + } + + public List<KuduSourceSplit> generateFullScanSplits(long snapshotTimestamp) { + try { + KuduTable table = kuduClient.openTable(tableInfo.getName()); + List<KuduScanToken> tokens = + kuduClient + .newScanTokenBuilder(table) + .snapshotTimestampRaw(snapshotTimestamp) + .readMode(AsyncKuduScanner.ReadMode.READ_AT_SNAPSHOT) + .build(); + + return serializeTokens(tokens); + } catch (Exception e) { + throw new RuntimeException("Error during full snapshot scan", e); + } + } + + public List<KuduSourceSplit> generateIncrementalSplits(long startHT, long endHT) { Review Comment: preconditions check for start/endHT? e.g. start<end, etc. ########## flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumerator.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.flink.connector.kudu.source.enumerator; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.source.split.KuduSourceSplit; +import org.apache.flink.connector.kudu.source.split.SplitFinishedEvent; + +import org.apache.kudu.util.HybridTimeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * The Kudu source enumerator is responsible for periodically discovering and assigning new splits + * when possible. + * + * <p>To provide CDC-like functionality, the enumeration works as follows: Initially, we perform a + * snapshot read of the table and mark the snapshot time as t0. From that point onward, we perform + * differential scans in the time intervals t0 - t1, t1 - t2, and so on. + * + * <p>This approach means that new splits can only be enumerated once the current time range is + * fully processed. + * + * <p>The process is controlled as follows: Once a set of splits is enumerated for a time range, we + * track: Unassigned splits that have been discovered but not yet assigned to readers. Pending + * splits that are assigned but not yet fully processed. A new set of splits is generated only when + * there are no remaining unassigned or pending splits. + */ +public class KuduSourceEnumerator + implements SplitEnumerator<KuduSourceSplit, KuduSourceEnumeratorState> { + private static final Logger LOG = LoggerFactory.getLogger(KuduSourceEnumerator.class); + + private final SplitEnumeratorContext<KuduSourceSplit> context; + private final List<Integer> readersAwaitingSplit; + private final List<KuduSourceSplit> unassigned; + private final List<KuduSourceSplit> pending; + private final Duration period; + private final KuduSplitGenerator splitGenerator; + private long lastEndTimestamp; + + public KuduSourceEnumerator( + KuduTableInfo tableInfo, + KuduReaderConfig readerConfig, + Duration period, + SplitEnumeratorContext<KuduSourceSplit> context, + KuduSourceEnumeratorState enumState) { + this.period = period; + this.context = context; + this.readersAwaitingSplit = new ArrayList<>(); + this.unassigned = enumState == null ? new ArrayList<>() : enumState.getUnassigned(); + this.pending = enumState == null ? new ArrayList<>() : enumState.getPending(); + this.splitGenerator = new KuduSplitGenerator(readerConfig, tableInfo); + this.lastEndTimestamp = enumState == null ? -1L : enumState.getLastEndTimestamp(); + } + + @Override + public void start() { + context.callAsync( + () -> this.enumerateNewSplits(() -> pending.isEmpty() && unassigned.isEmpty()), + this::assignSplits, + 0, + this.period.toMillis()); + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + readersAwaitingSplit.add(subtaskId); + assignSplitsToReaders(); + } + + @Override + public void addSplitsBack(List<KuduSourceSplit> splits, int subtaskId) { + LOG.debug("File Source Enumerator adds splits back: {}", splits); + unassigned.addAll(splits); + if (context.registeredReaders().containsKey(subtaskId)) { + readersAwaitingSplit.add(subtaskId); + } + assignSplitsToReaders(); + } + + @Override + public void addReader(int subtaskId) { + // The source is purely lazy-pull-based, nothing to do upon registration + } + + @Override + public KuduSourceEnumeratorState snapshotState(long checkpointId) throws Exception { + return new KuduSourceEnumeratorState(lastEndTimestamp, unassigned, pending); + } + + @Override + public void close() throws IOException { + try { + splitGenerator.close(); + } catch (Exception e) { + throw new IOException("Error closing split generator", e); + } + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + if (sourceEvent instanceof SplitFinishedEvent) { + SplitFinishedEvent splitFinishedEvent = (SplitFinishedEvent) sourceEvent; + LOG.debug( + "Received SplitFinishedEvent from subtask {} for splits: {}", + subtaskId, + splitFinishedEvent.getFinishedSplits()); + pending.removeAll(splitFinishedEvent.getFinishedSplits()); + readersAwaitingSplit.add(subtaskId); + } + } + + // This function is invoked repeatedly according to this.period if there are no outstanding + // splits. + // Outstanding meaning that no pending splits, and no enumerated but not assigned splits for the Review Comment: nit: that there are no pending splints and there are no enumerated... ########## flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumerator.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.flink.connector.kudu.source.enumerator; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.kudu.connector.KuduTableInfo; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.source.split.KuduSourceSplit; +import org.apache.flink.connector.kudu.source.split.SplitFinishedEvent; + +import org.apache.kudu.util.HybridTimeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * The Kudu source enumerator is responsible for periodically discovering and assigning new splits + * when possible. + * + * <p>To provide CDC-like functionality, the enumeration works as follows: Initially, we perform a + * snapshot read of the table and mark the snapshot time as t0. From that point onward, we perform + * differential scans in the time intervals t0 - t1, t1 - t2, and so on. + * + * <p>This approach means that new splits can only be enumerated once the current time range is + * fully processed. + * + * <p>The process is controlled as follows: Once a set of splits is enumerated for a time range, we + * track: Unassigned splits that have been discovered but not yet assigned to readers. Pending + * splits that are assigned but not yet fully processed. A new set of splits is generated only when Review Comment: nit: make this an itemised list? e.g. ....we track: * Unassigned splits that have been discovered but not yet assigned to readers. * Pending splits that are assigned but not yet fully processed. A new set of splits is generated only.... ########## flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/reader/KuduSourceSplitReader.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.flink.connector.kudu.source.reader; + +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.kudu.connector.reader.KuduReaderConfig; +import org.apache.flink.connector.kudu.source.split.KuduSourceSplit; + +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.RowResult; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; + +/** The Kudu source reader that reads data for corresponding splits. */ +public class KuduSourceSplitReader implements SplitReader<RowResult, KuduSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(KuduSourceSplitReader.class); + private final KuduClient kuduClient; + private final List<KuduSourceSplit> splits; + private final AtomicBoolean wakeUpFlag = new AtomicBoolean(false); + + public KuduSourceSplitReader(KuduReaderConfig readerConfig) { + this.kuduClient = new KuduClient.KuduClientBuilder(readerConfig.getMasters()).build(); + this.splits = new ArrayList<>(); + } + + @Override + public RecordsWithSplitIds<RowResult> fetch() throws IOException { + wakeUpFlag.compareAndSet(true, false); + + final Optional<KuduSourceSplit> currentSplitOpt = getNextSplit(); + if (!currentSplitOpt.isPresent()) { + return new RecordsBySplits.Builder<RowResult>().build(); + } + + KuduSourceSplit currentSplit = currentSplitOpt.get(); + byte[] serializedToken = currentSplit.getSerializedScanToken(); + KuduScanner scanner = KuduScanToken.deserializeIntoScanner(serializedToken, kuduClient); + RecordsBySplits.Builder<RowResult> builder = new RecordsBySplits.Builder<>(); + + try { + while (scanner.hasMoreRows()) { + for (RowResult row : scanner.nextRows()) { + if (wakeUpFlag.get()) { + LOG.debug("Wakeup signal received inside row iteration, stopping fetch."); + scanner.close(); // Close the scanner + splits.add(currentSplit); // Put the split back + return new RecordsBySplits.Builder<RowResult>() + .build(); // Return empty result + } + builder.add(currentSplit.splitId(), row); + } + } + builder.addFinishedSplit( + currentSplit.splitId()); // Mark split as completed only after the loop + + } finally { Review Comment: shoudln't there be a catch here to handle/recover problems? -- 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]
