/
SplitFetcherManager.java
343 lines (308 loc) · 14.3 KB
/
SplitFetcherManager.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
/*
* 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.reader.fetcher;
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.connector.source.SourceSplit;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.SourceReaderBase;
import org.apache.flink.connector.base.source.reader.SourceReaderOptions;
import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.Supplier;
import static org.apache.flink.configuration.PipelineOptions.ALLOW_UNALIGNED_SOURCE_SPLITS;
/**
* A class responsible for starting the {@link SplitFetcher} and manage the life cycles of them.
* This class works with the {@link SourceReaderBase}.
*
* <p>The split fetcher manager could be used to support different threading models by implementing
* the {@link #addSplits(List)} method differently. For example, a single thread split fetcher
* manager would only start a single fetcher and assign all the splits to it. A one-thread-per-split
* fetcher may spawn a new thread every time a new split is assigned.
*/
@PublicEvolving
public abstract class SplitFetcherManager<E, SplitT extends SourceSplit> {
private static final Logger LOG = LoggerFactory.getLogger(SplitFetcherManager.class);
private final Consumer<Throwable> errorHandler;
/** An atomic integer to generate monotonically increasing fetcher ids. */
private final AtomicInteger fetcherIdGenerator;
/** A supplier to provide split readers. */
private final Supplier<SplitReader<E, SplitT>> splitReaderFactory;
/** Uncaught exception in the split fetchers. */
private final AtomicReference<Throwable> uncaughtFetcherException;
/** The element queue that the split fetchers will put elements into. */
private final FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue;
/** A map keeping track of all the split fetchers. */
protected final Map<Integer, SplitFetcher<E, SplitT>> fetchers;
/**
* An executor service with two threads. One for the fetcher and one for the future completing
* thread.
*/
private final ExecutorService executors;
/** Indicating the split fetcher manager has closed or not. */
private volatile boolean closed;
/**
* Hook for handling finished splits in {@link SplitFetcher}, usually used for testing split
* finishing behavior of {@link SplitFetcher} and {@link SplitReader}.
*/
private final Consumer<Collection<String>> splitFinishedHook;
private final boolean allowUnalignedSourceSplits;
/**
* Create a split fetcher manager.
*
* @param elementsQueue the queue that split readers will put elements into.
* @param splitReaderFactory a supplier that could be used to create split readers.
* @param configuration the configuration of this fetcher manager.
* @deprecated Please use {@link #SplitFetcherManager(Supplier, Configuration)} instead.
*/
@Deprecated
public SplitFetcherManager(
FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
Supplier<SplitReader<E, SplitT>> splitReaderFactory,
Configuration configuration) {
this(elementsQueue, splitReaderFactory, configuration, (ignore) -> {});
}
/**
* Create a split fetcher manager.
*
* @param elementsQueue the queue that split readers will put elements into.
* @param splitReaderFactory a supplier that could be used to create split readers.
* @param configuration the configuration of this fetcher manager.
* @param splitFinishedHook Hook for handling finished splits in split fetchers.
* @deprecated Please use {@link #SplitFetcherManager(Supplier, Configuration, Consumer)}
* instead.
*/
@Deprecated
@VisibleForTesting
public SplitFetcherManager(
FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
Supplier<SplitReader<E, SplitT>> splitReaderFactory,
Configuration configuration,
Consumer<Collection<String>> splitFinishedHook) {
this.elementsQueue = elementsQueue;
this.errorHandler =
new Consumer<Throwable>() {
@Override
public void accept(Throwable t) {
LOG.error("Received uncaught exception.", t);
if (!uncaughtFetcherException.compareAndSet(null, t)) {
// Add the exception to the exception list.
uncaughtFetcherException.get().addSuppressed(t);
}
// Wake up the main thread to let it know the exception.
elementsQueue.notifyAvailable();
}
};
this.splitReaderFactory = splitReaderFactory;
this.splitFinishedHook = splitFinishedHook;
this.uncaughtFetcherException = new AtomicReference<>(null);
this.fetcherIdGenerator = new AtomicInteger(0);
this.fetchers = new ConcurrentHashMap<>();
this.allowUnalignedSourceSplits = configuration.get(ALLOW_UNALIGNED_SOURCE_SPLITS);
// Create the executor with a thread factory that fails the source reader if one of
// the fetcher thread exits abnormally.
final String taskThreadName = Thread.currentThread().getName();
this.executors =
Executors.newCachedThreadPool(
r -> new Thread(r, "Source Data Fetcher for " + taskThreadName));
this.closed = false;
}
/**
* Create a split fetcher manager.
*
* @param splitReaderFactory a supplier that could be used to create split readers.
* @param configuration the configuration of this fetcher manager.
*/
public SplitFetcherManager(
Supplier<SplitReader<E, SplitT>> splitReaderFactory, Configuration configuration) {
this(splitReaderFactory, configuration, (ignore) -> {});
}
/**
* Create a split fetcher manager.
*
* @param splitReaderFactory a supplier that could be used to create split readers.
* @param configuration the configuration of this fetcher manager.
* @param splitFinishedHook Hook for handling finished splits in split fetchers.
*/
public SplitFetcherManager(
Supplier<SplitReader<E, SplitT>> splitReaderFactory,
Configuration configuration,
Consumer<Collection<String>> splitFinishedHook) {
this.elementsQueue =
new FutureCompletingBlockingQueue<>(
configuration.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY));
this.errorHandler =
new Consumer<Throwable>() {
@Override
public void accept(Throwable t) {
LOG.error("Received uncaught exception.", t);
if (!uncaughtFetcherException.compareAndSet(null, t)) {
// Add the exception to the exception list.
uncaughtFetcherException.get().addSuppressed(t);
}
// Wake up the main thread to let it know the exception.
elementsQueue.notifyAvailable();
}
};
this.splitReaderFactory = splitReaderFactory;
this.splitFinishedHook = splitFinishedHook;
this.uncaughtFetcherException = new AtomicReference<>(null);
this.fetcherIdGenerator = new AtomicInteger(0);
this.fetchers = new ConcurrentHashMap<>();
this.allowUnalignedSourceSplits = configuration.get(ALLOW_UNALIGNED_SOURCE_SPLITS);
// Create the executor with a thread factory that fails the source reader if one of
// the fetcher thread exits abnormally.
final String taskThreadName = Thread.currentThread().getName();
this.executors =
Executors.newCachedThreadPool(
r -> new Thread(r, "Source Data Fetcher for " + taskThreadName));
this.closed = false;
}
public abstract void addSplits(List<SplitT> splitsToAdd);
public abstract void removeSplits(List<SplitT> splitsToRemove);
public void pauseOrResumeSplits(
Collection<String> splitIdsToPause, Collection<String> splitIdsToResume) {
for (SplitFetcher<E, SplitT> fetcher : fetchers.values()) {
Map<String, SplitT> idToSplit = fetcher.assignedSplits();
List<SplitT> splitsToPause = lookupInAssignment(splitIdsToPause, idToSplit);
List<SplitT> splitsToResume = lookupInAssignment(splitIdsToResume, idToSplit);
if (!splitsToPause.isEmpty() || !splitsToResume.isEmpty()) {
fetcher.pauseOrResumeSplits(splitsToPause, splitsToResume);
}
}
}
private List<SplitT> lookupInAssignment(
Collection<String> splitIds, Map<String, SplitT> assignment) {
List<SplitT> splits = new ArrayList<>();
for (String s : splitIds) {
SplitT split = assignment.get(s);
if (split != null) {
splits.add(split);
}
}
return splits;
}
protected void startFetcher(SplitFetcher<E, SplitT> fetcher) {
executors.submit(fetcher);
}
/**
* Synchronize method to ensure no fetcher is created after the split fetcher manager has
* closed.
*
* @return the created split fetcher.
* @throws IllegalStateException if the split fetcher manager has closed.
*/
protected synchronized SplitFetcher<E, SplitT> createSplitFetcher() {
if (closed) {
throw new IllegalStateException("The split fetcher manager has closed.");
}
// Create SplitReader.
SplitReader<E, SplitT> splitReader = splitReaderFactory.get();
int fetcherId = fetcherIdGenerator.getAndIncrement();
SplitFetcher<E, SplitT> splitFetcher =
new SplitFetcher<>(
fetcherId,
elementsQueue,
splitReader,
errorHandler,
() -> {
fetchers.remove(fetcherId);
// We need this to synchronize status of fetchers to concurrent partners
// as
// ConcurrentHashMap's aggregate status methods including size, isEmpty,
// and
// containsValue are not designed for program control.
elementsQueue.notifyAvailable();
},
this.splitFinishedHook,
allowUnalignedSourceSplits);
fetchers.put(fetcherId, splitFetcher);
return splitFetcher;
}
/**
* Check and shutdown the fetchers that have completed their work.
*
* @return true if all the fetchers have completed the work, false otherwise.
*/
public boolean maybeShutdownFinishedFetchers() {
Iterator<Map.Entry<Integer, SplitFetcher<E, SplitT>>> iter = fetchers.entrySet().iterator();
while (iter.hasNext()) {
Map.Entry<Integer, SplitFetcher<E, SplitT>> entry = iter.next();
SplitFetcher<E, SplitT> fetcher = entry.getValue();
if (fetcher.isIdle()) {
LOG.info("Closing splitFetcher {} because it is idle.", entry.getKey());
fetcher.shutdown();
iter.remove();
}
}
return fetchers.isEmpty();
}
/**
* Return the queue contains data produced by split fetchers.This method is Internal and only
* used in {@link SourceReaderBase}.
*/
@Internal
public FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> getQueue() {
return elementsQueue;
}
/**
* Close the split fetcher manager.
*
* @param timeoutMs the max time in milliseconds to wait.
* @throws Exception when failed to close the split fetcher manager.
*/
public synchronized void close(long timeoutMs) throws Exception {
closed = true;
fetchers.values().forEach(SplitFetcher::shutdown);
executors.shutdown();
if (!executors.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS)) {
LOG.warn(
"Failed to close the source reader in {} ms. There are still {} split fetchers running",
timeoutMs,
fetchers.size());
}
}
public void checkErrors() {
if (uncaughtFetcherException.get() != null) {
throw new RuntimeException(
"One or more fetchers have encountered exception",
uncaughtFetcherException.get());
}
}
// -----------------------
@VisibleForTesting
public int getNumAliveFetchers() {
return fetchers.size();
}
}