zchovan commented on code in PR #8:
URL: 
https://github.com/apache/flink-connector-kudu/pull/8#discussion_r1973473991


##########
flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumerator.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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:
+ *
+ * <ul>
+ *   <li>Once a set of splits is enumerated for a time range, we track:
+ *       <ul>
+ *         <li><b>Unassigned splits</b>: Discovered but not yet assigned to 
readers.
+ *         <li><b>Pending splits</b>: Assigned but not yet fully processed.
+ *       </ul>
+ *   <li>A new set of splits is generated only when there are no remaining 
unassigned or pending
+ *       splits.
+ * </ul>
+ */
+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) {

Review Comment:
   done



##########
flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumerator.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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:
+ *
+ * <ul>
+ *   <li>Once a set of splits is enumerated for a time range, we track:
+ *       <ul>
+ *         <li><b>Unassigned splits</b>: Discovered but not yet assigned to 
readers.
+ *         <li><b>Pending splits</b>: Assigned but not yet fully processed.
+ *       </ul>
+ *   <li>A new set of splits is generated only when there are no remaining 
unassigned or pending
+ *       splits.
+ * </ul>
+ */
+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;

Review Comment:
   done



##########
flink-connector-kudu/src/main/java/org/apache/flink/connector/kudu/source/enumerator/KuduSourceEnumerator.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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:
+ *
+ * <ul>
+ *   <li>Once a set of splits is enumerated for a time range, we track:
+ *       <ul>
+ *         <li><b>Unassigned splits</b>: Discovered but not yet assigned to 
readers.
+ *         <li><b>Pending splits</b>: Assigned but not yet fully processed.
+ *       </ul>
+ *   <li>A new set of splits is generated only when there are no remaining 
unassigned or pending
+ *       splits.
+ * </ul>
+ */
+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 there are no pending splits, and no enumerated 
but not assigned
+    // splits for the
+    // current period.
+    private List<KuduSourceSplit> enumerateNewSplits(@Nonnull 
Supplier<Boolean> shouldGenerate) {
+        if (shouldGenerate.get() == Boolean.FALSE) {

Review Comment:
   done



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to