stevenzwu commented on a change in pull request #15924:
URL: https://github.com/apache/flink/pull/15924#discussion_r641778102



##########
File path: 
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java
##########
@@ -0,0 +1,406 @@
+/*
+ * 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.base.source.hybrid;
+
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SourceSplit;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.function.BiConsumer;
+
+/**
+ * Wraps the actual split enumerators and facilitates source switching. 
Enumerators are created
+ * lazily when source switch occurs to support runtime position conversion.
+ *
+ * <p>This enumerator delegates to the current underlying split enumerator and 
transitions to the
+ * next source once all readers have indicated via {@link 
SourceReaderFinishedEvent} that all input
+ * was consumed.
+ *
+ * <p>Switching between enumerators occurs by either creating the new 
enumerator with a fixed start
+ * position via {@link Source#createEnumerator(SplitEnumeratorContext)} or by 
using the state of the
+ * previous enumerator and the optional user supplied checkpoint state 
converter to create the next
+ * enumerator with a runtime determined start position via {@link
+ * Source#restoreEnumerator(SplitEnumeratorContext, Object)}.
+ *
+ * <p>During subtask recovery, splits that have been assigned since the last 
checkpoint will be

Review comment:
       there are a lot of complexities to take care of this scenario. I am 
wondering if we can do the source switch only at  `notifyCheckpointComplete`. 
That might help avoid those complexities and the HybridSourceSplitEnumerator 
doesn't need to track any assignment or pendingSplits.
   
   It has its own drawbacks
   - delay the switch up to a checkpoint interval
   - it requires checkpoint enabled. it won't work for batch sources that don't 
support or enable checkpoint. is it a valid concern?




-- 
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.

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


Reply via email to