StephanEwen commented on a change in pull request #11554: 
[FLINK-15101][connector/common] Add SourceCoordinator implementation.
URL: https://github.com/apache/flink/pull/11554#discussion_r410746156
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java
 ##########
 @@ -0,0 +1,259 @@
+/*
+ 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.runtime.source.coordinator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+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.core.io.SimpleVersionedSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
+import org.apache.flink.runtime.source.event.AddSplitEvent;
+import org.apache.flink.runtime.source.event.SourceEventWrapper;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.function.BiConsumer;
+
+/**
+ * A context class for the {@link OperatorCoordinator}. Compared with {@link 
SplitEnumeratorContext} this class
+ * allows interaction with state and sending {@link OperatorEvent} to the 
SourceOperator while
+ * {@link SplitEnumeratorContext} only allows sending {@link SourceEvent}.
+ *
+ * <p>The context serves a few purposes:
+ * <ul>
+ *     <li>
+ *         Information provider - The context provides necessary information 
to the enumerator for it to
+ *         know what is the status of the source readers and their split 
assignments. These information
+ *         allows the split enumerator to do the coordination.
+ *     </li>
+ *     <li>
+ *         Action taker - The context also provides a few actions that the 
enumerator can take to carry
+ *         out the coordination. So far there are two actions: 1) assign 
splits to the source readers.
+ *         and 2) sens a custom {@link SourceEvent SourceEvents} to the source 
readers.
+ *     </li>
+ *     <li>
+ *         Thread model enforcement - The context ensures that all the 
manipulations to the coordinator state
+ *         are handled by the same thread.
+ *     </li>
+ * </ul>
+ * @param <SplitT> the type of the splits.
+ */
+@Internal
+public class SourceCoordinatorContext<SplitT extends SourceSplit> implements 
SplitEnumeratorContext<SplitT> {
+       private final ExecutorService coordinatorExecutor;
+       private final ExecutorNotifier notifier;
+       private final OperatorCoordinator.Context operatorCoordinatorContext;
+       private final ConcurrentMap<Integer, ReaderInfo> registeredReaders;
+       private final SplitAssignmentTracker<SplitT> assignmentTracker;
+       private final String coordinatorThreadName;
+
+       public SourceCoordinatorContext(
+                       ExecutorService coordinatorExecutor,
+                       String coordinatorThreadName,
+                       int numWorkerThreads,
+                       OperatorCoordinator.Context operatorCoordinatorContext) 
{
+               this(coordinatorExecutor, coordinatorThreadName, 
numWorkerThreads, operatorCoordinatorContext,
+                               new SplitAssignmentTracker<>());
+       }
+
+       // Package private method for unit test.
+       SourceCoordinatorContext(
+                       ExecutorService coordinatorExecutor,
+                       String coordinatorThreadName,
+                       int numWorkerThreads,
+                       OperatorCoordinator.Context operatorCoordinatorContext,
+                       SplitAssignmentTracker<SplitT> splitAssignmentTracker) {
+               this.coordinatorExecutor = coordinatorExecutor;
+               this.notifier = new ExecutorNotifier(
+                               
Executors.newScheduledThreadPool(numWorkerThreads, new ThreadFactory() {
+                                       private int index = 0;
+                                       @Override
+                                       public Thread newThread(Runnable r) {
+                                               return new Thread(r, 
coordinatorThreadName + "-worker-" + index++);
+                                       }
+                               }),
+                               coordinatorExecutor);
+               this.operatorCoordinatorContext = operatorCoordinatorContext;
+               this.registeredReaders = new ConcurrentHashMap<>();
+               this.assignmentTracker = splitAssignmentTracker;
+               this.coordinatorThreadName = coordinatorThreadName;
+       }
+
+       @Override
+       public MetricGroup metricGroup() {
+               return null;
+       }
+
+       @Override
+       public void sendEventToSourceReader(int subtaskId, SourceEvent event) {
+               try {
+                       operatorCoordinatorContext.sendEvent(new 
SourceEventWrapper(event), subtaskId);
+               } catch (TaskNotRunningException e) {
+                       throw new FlinkRuntimeException(String.format("Failed 
to send event %s to subtask %d",
+                                       event,
+                                       subtaskId), e);
+               }
+       }
+
+       @Override
+       public int currentParallelism() {
+               return operatorCoordinatorContext.currentParallelism();
+       }
+
+       @Override
+       public Map<Integer, ReaderInfo> registeredReaders() {
+               return Collections.unmodifiableMap(registeredReaders);
+       }
+
+       @Override
+       public void assignSplits(SplitsAssignment<SplitT> assignment) {
+               // Ensure the split assignment is done by the the coordinator 
executor.
+               if 
(!Thread.currentThread().getName().equals(coordinatorThreadName)) {
 
 Review comment:
   Going by thread name is very fragile. Why not go by reference equality of 
the Thread objects?

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to