Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,9 @@ public class LazyFlinkSourceSplitEnumerator<T>
private final PipelineOptions pipelineOptions;
private final int numSplits;
private final List<FlinkSourceSplit<T>> pendingSplits;
private final List<Integer> pendingRequests;
private boolean splitsInitialized;
private boolean splitsReady;

public LazyFlinkSourceSplitEnumerator(
SplitEnumeratorContext<FlinkSourceSplit<T>> context,
Expand All @@ -66,7 +68,9 @@ public LazyFlinkSourceSplitEnumerator(
this.pipelineOptions = pipelineOptions;
this.numSplits = numSplits;
this.pendingSplits = new ArrayList<>(numSplits);
this.pendingRequests = new ArrayList<>();
this.splitsInitialized = splitInitialized;
this.splitsReady = false;
}

@Override
Expand Down Expand Up @@ -94,9 +98,13 @@ public void initializeSplits() {
},
(sourceSplits, error) -> {
if (error != null) {
pendingSplits.addAll(sourceSplits);
throw new RuntimeException("Failed to start source enumerator.", error);
}
splitsReady = true;
for (int subtask : pendingRequests) {
handleSplitRequest(subtask, null);
}
pendingRequests.clear();
});
}

Expand All @@ -113,6 +121,12 @@ public void handleSplitRequest(int subtask, @Nullable String hostname) {
LOG.info("Subtask {} {} is requesting a file source split", subtask, hostInfo);
}

if (!splitsReady) {
LOG.info("Subtask {} requested split before enumeration done, buffering", subtask);
pendingRequests.add(subtask);
return;
}

if (!pendingSplits.isEmpty()) {
final FlinkSourceSplit<T> split = pendingSplits.remove(pendingSplits.size() - 1);
context.assignSplit(split, subtask);
Expand Down
Loading
Loading