StephanEwen commented on a change in pull request #13401:
URL: https://github.com/apache/flink/pull/13401#discussion_r491699660



##########
File path: 
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.file.src.impl;
+
+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.base.source.event.RequestSplitEvent;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.PendingSplitsCheckpoint;
+import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
+import org.apache.flink.connector.file.src.enumerate.FileEnumerator;
+import org.apache.flink.core.fs.Path;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A continuously monitoring enumerator.
+ */
+public class ContinuousFileSplitEnumerator implements 
SplitEnumerator<FileSourceSplit, PendingSplitsCheckpoint> {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(ContinuousFileSplitEnumerator.class);
+
+       private final SplitEnumeratorContext<FileSourceSplit> context;
+
+       private final FileSplitAssigner splitAssigner;
+
+       private final FileEnumerator enumerator;
+
+       private final HashSet<Path> pathsAlreadyProcessed;
+
+       private final HashSet<Integer> readersAwaitingSplit;
+
+       private final Path[] paths;
+
+       private final long discoveryInterval;
+
+       // 
------------------------------------------------------------------------
+
+       public ContinuousFileSplitEnumerator(
+                       SplitEnumeratorContext<FileSourceSplit> context,
+                       FileEnumerator enumerator,
+                       FileSplitAssigner splitAssigner,
+                       Path[] paths,
+                       Collection<Path> alreadyDiscoveredPaths,
+                       long discoveryInterval) {
+
+               checkArgument(discoveryInterval > 0L);
+               this.context = checkNotNull(context);
+               this.enumerator = checkNotNull(enumerator);
+               this.splitAssigner = checkNotNull(splitAssigner);
+               this.paths = paths;
+               this.discoveryInterval = discoveryInterval;
+               this.pathsAlreadyProcessed = new 
HashSet<>(alreadyDiscoveredPaths);
+               this.readersAwaitingSplit = new HashSet<>();
+       }
+
+       @Override
+       public void start() {
+               context.callAsync(
+                       () -> enumerator.enumerateSplits(paths, 1),
+                       this::processDiscoveredSplits,
+                       discoveryInterval, discoveryInterval);
+       }
+
+       @Override
+       public void close() throws IOException {
+               // no resources to close
+       }
+
+       @Override
+       public void addReader(int subtaskId) {
+               // this source is purely lazy-pull-based, nothing to do upon 
registration
+       }
+
+       @Override
+       public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+               if (sourceEvent instanceof RequestSplitEvent) {
+                       readersAwaitingSplit.add(subtaskId);
+                       assignSplits();
+               }
+               else {
+                       LOG.error("Received unrecognized event: {}", 
sourceEvent);
+               }
+       }
+
+       @Override
+       public void addSplitsBack(List<FileSourceSplit> splits, int subtaskId) {
+               LOG.debug("File Source Enumerator adds splits back: {}", 
splits);
+               splitAssigner.addSplits(splits);
+       }
+
+       @Override
+       public PendingSplitsCheckpoint snapshotState() throws Exception {
+               return 
PendingSplitsCheckpoint.fromCollectionSnapshot(splitAssigner.remainingSplits(), 
pathsAlreadyProcessed);
+       }
+
+       // 
------------------------------------------------------------------------
+
+       private void processDiscoveredSplits(Collection<FileSourceSplit> 
splits, Throwable error) {
+               if (error != null) {
+                       LOG.error("Failed to enumerate files", error);
+                       return;
+               }
+
+               final Collection<FileSourceSplit> newSplits = splits.stream()
+                               .filter((split) -> 
pathsAlreadyProcessed.add(split.path()))
+                               .collect(Collectors.toList());
+               splitAssigner.addSplits(newSplits);
+
+               assignSplits();
+       }
+
+       private void assignSplits() {
+               final Iterator<Integer> awaitingReader = 
readersAwaitingSplit.iterator();
+
+               while (awaitingReader.hasNext()) {
+                       final int awaitingSubtask = awaitingReader.next();
+                       final Optional<FileSourceSplit> nextSplit = 
splitAssigner.getNext(null);

Review comment:
       Good catch.




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