Discussion thread | https://lists.apache.org/thread/b8f509878ptwl3kmmgg95tv8sb1j5987 | ||||||||
---|---|---|---|---|---|---|---|---|---|
Vote thread | https://lists.apache.org/thread/t1zff21z440pvv48jyhm8pgtqsyplchn | ||||||||
JIRA J IRA |
| ||||||||
Release | <Flink Version>1.19 |
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Why is it necessary to make SingleThreadFetcherManager PublicEvolving?
- Though the
...
- SingleThreadFetcherManager is annotated as Internal, it actually acts as some-degree public API, which is widely used in many connector projects:
...
...
...
- kafka, flink-connector-mongodb, flink-cdc-connectors and soon. In flink-connector-kafka, KafkaSourceFetcherManager even extends SingleThreadFetcherManager.
More over, even the constructor
...
of
SingleThreadMultiplexSourceReaderBase
...
(which is PublicEvolving) includes the params of
SingleThreadFetcherManager
...
and
FutureCompletingBlockingQueue
...
. That means that the
SingleThreadFetcherManager
...
and
FutureCompletingBlockingQueue
...
have already been exposed to users for a long time and are widely used.As shown in
FLINK-31324,FLINK-28853used to change the default constructor of SingleThreadFetcherManager.However, it influenced a lot. Finally, the former constructor was added back and marked asDeprecated.Code Block public SingleThreadMultiplexSourceReaderBase( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, SingleThreadFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration config, SourceReaderContext context) { super(elementsQueue, splitFetcherManager, recordEmitter, config, context); }
...
The original design of SplitFetcherManager and
As shown in FLINK-31324, FLINK-28853 used to change the default constructor of SingleThreadFetcherManager.However, it influenced a lot. Finally, the former constructor was added back and marked asDeprecated。
Therefore, why not make SingleThreadFetcherManager PublicEvolving?
More contexts from the origin design:
...
- its subclasses was to make them public to the Source developers. The goal is to let us take care of the threading model, while the Source developers can just focus on the SplitReader implementation. Therefore, I think making SplitFetcherManater / SingleThreadFetcherManager public aligns with the original design. That is also why these classes are exposed in the constructor of SourceReaderBase.
2. For FutureCompletingBlockingQueue, as a hindsight, it might be better to not expose it to the Source developers. They are unlikely to use it anywhere other than just constructing it. The reason that FutureCompletingBlockingQueue is currently exposed in the SourceReaderBase constructor is because both the SplitFetcherManager and SourceReaderBase need it. One way to hide the FutureCompletingBlockingQueue from the public API is to make SplitFetcherManager the only owner class of the queue, and expose some of its methods via SplitFetcherManager. This way, the SourceReaderBase can invoke the methods via SplitFetcherManager. I believe this also makes the code slightly cleaner.
Public Interfaces
Change SingleThreadFetcherManager
, FutureCompletingBlockingQueue
and parent class SplitFetcherManager
from Internal
to PublicEvolving
.
Proposed Changes
- Mark constructor of SingleThreadMultiplexSourceReaderBase as
@Depricated
. Add new constructors without FutureCompletingBlockingQueue
. This will make the code slightly cleaner.
In summary, this flip has 2 goals:
- To expose the SplitFetcherManager / SingleThreadFetcheManager as Public, allowing connector developers to easily create their own threading models in the SourceReaderBase.
- To hide the element queue from the connector developers and make SplitFetcherManager the only owner class of the queue
Public Interfaces
SplitFetcherManager
- Change SplitFetcherManager from Internal to PublicEvolving.
- Deprecate the old constructor exposing the FutureCompletingBlockingQueue, and add new constructors as replacements which creates the FutureCompletingBlockingQueue instance internally.
- Add a new internal method SplitFetcherManager#getQueue() for SourceReaderBase usage.
Code Block |
---|
@PublicEvolving
public abstract class SplitFetcherManager<E, SplitT extends SourceSplit> {
public SingleThreadMultiplexSourceReaderBase( |
Code Block |
// SingleThreadMultiplexSourceReaderBase @Depricated public SingleThreadMultiplexSourceReaderBase( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, SingleThreadFetcherManager<ESupplier<SplitReader<E, SplitT>SplitT>> splitFetcherManagersplitReaderSupplier, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration config, SourceReaderContext context) { super(elementsQueue, splitFetcherManager, recordEmitter, config, context); } @Depricated public SingleThreadMultiplexSourceReaderBase( new FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueueSingleThreadFetcherManager<>(splitReaderSupplier, config), Supplier<SplitReader<E, SplitT>> splitReaderSupplierrecordEmitter, RecordEmitter<E, T, SplitStateT> recordEmitterconfig, Configuration config, context); } @Deprecated SourceReaderContext context)public {SplitFetcherManager( super( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, elementsQueueSupplier<SplitReader<E, SplitT>> splitReaderFactory, Configuration configuration) { new SingleThreadFetcherManager<> this(elementsQueue, splitReaderSuppliersplitReaderFactory, config)configuration, (ignore) -> { }); } recordEmitter, config,@Deprecated context); } // todo: Add new constructors without FutureCompletingBlockingQueue public SingleThreadMultiplexSourceReaderBase@VisibleForTesting public SplitFetcherManager( Supplier<SplitReader<E, SplitT>> splitReaderSupplierFutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, RecordEmitter<ESupplier<SplitReader<E, T, SplitStateT> recordEmitterSplitT>> splitReaderFactory, Configuration configconfiguration, SourceReaderContextConsumer<Collection<String>> contextsplitFinishedHook) { } // todo: provide a new constructor without super( FutureCompletingBlockingQueue. public SplitFetcherManager( new SingleThreadFetcherManager<>(splitReaderSupplierSupplier<SplitReader<E, config)SplitT>> splitReaderFactory,, Configuration recordEmitter,configuration) { this(splitReaderFactory, configuration, (ignore) -> { config, }); } context); } public SingleThreadMultiplexSourceReaderBase// todo: provide a new constructor without FutureCompletingBlockingQueue. public SplitFetcherManager( SingleThreadFetcherManager<ESupplier<SplitReader<E, SplitT>SplitT>> splitFetcherManagersplitReaderFactory, RecordEmitter<E,Configuration Tconfiguration, SplitStateT> recordEmitter, @NullableConsumer<Collection<String>> RecordEvaluator<T>splitFinishedHook) eofRecordEvaluator,{ this.elementsQueue = Configuration config,new FutureCompletingBlockingQueue<>( SourceReaderContext context) { super( splitFetcherManager, configuration.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY)); // ...... } @Internal public FutureCompletingBlockingQueue getQueue(); } |
SingleThreadFetcherManager
- Change SingleThreadFetcherManager from Internal to PublicEvolving.
- Deprecate the old constructor exposing the FutureCompletingBlockingQueue, and add new constructors without FutureCompletingBlockingQueue as replacements which creates the FutureCompletingBlockingQueue instance by its parent class(SplitFetcherManager) internally.
Code Block |
---|
@PublicEvolving public class SingleThreadFetcherManager<E, SplitT extends SourceSplit> recordEmitter, extends SplitFetcherManager<E, SplitT> { @Deprecated public SingleThreadFetcherManager( eofRecordEvaluator, FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, config, Supplier<SplitReader<E, SplitT>> splitReaderSupplier) { context this(elementsQueue, splitReaderSupplier, new Configuration()); } |
2. Mark constructor of SourceReaderBase as @Depricated
and provide a new constructor without
FutureCompletingBlockingQueue
Code Block |
---|
@PublicEvolving public abstract class SourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> @Deprecated public SingleThreadFetcherManager( implements SourceReader<T, SplitT> { FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, @Deprecated public SourceReaderBase( Supplier<SplitReader<E, FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueueSplitT>> splitReaderSupplier, SplitFetcherManager<E,Configuration SplitT>configuration) splitFetcherManager,{ super(elementsQueue, splitReaderSupplier, configuration); RecordEmitter<E, T, SplitStateT> recordEmitter, } @Deprecated @VisibleForTesting Configurationpublic config,SingleThreadFetcherManager( SourceReaderContext context) { FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, this(elementsQueue, splitFetcherManager, recordEmitterSupplier<SplitReader<E, null,SplitT>> configsplitReaderSupplier, context); } @Deprecated publicConfiguration SourceReaderBase(configuration, FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,Consumer<Collection<String>> splitFinishedHook) { super(elementsQueue, splitReaderSupplier, SplitFetcherManager<Econfiguration, SplitT> splitFetcherManager,splitFinishedHook); } // todo: provide a new constructor without FutureCompletingBlockingQueue. RecordEmitter<E, T, SplitStateT> recordEmitter, public SingleThreadFetcherManager( @Nullable RecordEvaluator<T> eofRecordEvaluatorSupplier<SplitReader<E, SplitT>> splitReaderSupplier) { super(splitReaderSupplier, new Configuration config,()); } // todo: provide a new constructor SourceReaderContextwithout context)FutureCompletingBlockingQueue. { public SingleThreadFetcherManager( //this.elementsQueue = elementsQueue; Supplier<SplitReader<E, this.splitFetcherManager = splitFetcherManager; SplitT>> splitReaderSupplier, this.recordEmitter = recordEmitter; Configuration configuration) { this.splitStates = new HashMap<>(super(splitReaderSupplier, configuration); } // todo: provide this.options =a new SourceReaderOptions(config);constructor without FutureCompletingBlockingQueue. public SingleThreadFetcherManager( this.config = config; Supplier<SplitReader<E, SplitT>> splitReaderSupplier, this.context = context; this.noMoreSplitsAssignment =Configuration false;configuration, this.eofRecordEvaluator = eofRecordEvaluator; Consumer<Collection<String>> splitFinishedHook) { numRecordsInCounter = context.metricGroup().getIOMetricGroup().getNumRecordsInCounter()super(splitReaderSupplier, configuration, splitFinishedHook); } // todo: provide a new constructor without FutureCompletingBlockingQueue public SourceReaderBase( SplitFetcherManager<E, SplitT> splitFetcherManager, } |
SplitFetcherTask
Change SplitFetcherTask from Internal to PublicEvolving.
Code Block |
---|
@PublicEvolving
public interface SplitFetcherTask {} |
SplitFetcher
Change SplitFetcher from Internal to PublicEvolving, but still not to expose the construct of SplitFetcher, so it can only created by org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager#createSplitFetcher
Code Block |
---|
@PublicEvolving public class SplitFetcher<E, SplitT extends SourceSplit> implements Runnable { SplitFetcher( RecordEmitter<E, T, SplitStateT> recordEmitter int id, ConfigurationFutureCompletingBlockingQueue<RecordsWithSplitIds<E>> configelementsQueue, SourceReaderContextSplitReader<E, context)SplitT> {splitReader, this(splitFetcherManager, recordEmitter, null, config, context); } public SourceReaderBase( Consumer<Throwable> errorHandler, SplitFetcherManager<E, SplitT> splitFetcherManagerRunnable shutdownHook, RecordEmitter<E,Consumer<Collection<String>> TsplitFinishedHook, SplitStateT> recordEmitter, boolean allowUnalignedSourceSplits); } |
the constructor of SplitFetcher to the end users?
SourceReaderBase
Deprecate the old constructor exposing the FutureCompletingBlockingQueue, and add new constructors as replacements which creates the FutureCompletingBlockingQueue instance in SplitFetcherManager internally.
Code Block |
---|
@PublicEvolving public abstract class SourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> @Nullable RecordEvaluator<T> eofRecordEvaluator, Configuration config, implements SourceReader<T, SplitT> { SourceReaderContext context) {@Deprecated public SourceReaderBase( this.splitFetcherManager = splitFetcherManager; FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, this.recordEmitter = recordEmitter; this.splitStates =SplitFetcherManager<E, new HashMap<>();SplitT> splitFetcherManager, this.options = new SourceReaderOptions(config); RecordEmitter<E, T, SplitStateT> recordEmitter, this.config = Configuration config;, this.context =SourceReaderContext context;) { this.noMoreSplitsAssignment = false(elementsQueue, splitFetcherManager, recordEmitter, null, config, context); } this.eofRecordEvaluator = eofRecordEvaluator; @Deprecated public SourceReaderBase( numRecordsInCounter = context.metricGroup().getIOMetricGroup().getNumRecordsInCounter();FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, } } |
3. Mark constructor of SplitFetcherManager andSingleThreadFetcherManager as @Depricated
and provide a new constructor without FutureCompletingBlockingQueue. Mark SplitFetcherManager andSingleThreadFetcherManager as `@PublicEvolving`.
Code Block | ||
---|---|---|
| ||
@PublicEvolving public abstract class SplitFetcherManager<E, SplitTSplitT> extendssplitFetcherManager, SourceSplit> { @Deprecated public SplitFetcherManager(RecordEmitter<E, T, SplitStateT> recordEmitter, FutureCompletingBlockingQueue<RecordsWithSplitIds<E>>@Nullable RecordEvaluator<T> elementsQueueeofRecordEvaluator, Supplier<SplitReader<E, SplitT>> splitReaderFactoryConfiguration config, ConfigurationSourceReaderContext configurationcontext) { //this(.elementsQueue, splitReaderFactory, configuration, (ignore) -> { = elementsQueue; this.splitFetcherManager = })splitFetcherManager; } this.recordEmitter = @Deprecated recordEmitter; @VisibleForTesting this.splitStates = publicnew SplitFetcherManagerHashMap<>(); this.options = FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,new SourceReaderOptions(config); this.config = config; Supplier<SplitReader<E, SplitT>> splitReaderFactory, this.context = context; Configuration configuration, this.noMoreSplitsAssignment = false; this.eofRecordEvaluator Consumer<Collection<String>> splitFinishedHook) { } = eofRecordEvaluator; numRecordsInCounter = context.metricGroup().getIOMetricGroup().getNumRecordsInCounter(); } // todo: provide a new constructor without FutureCompletingBlockingQueue.without FutureCompletingBlockingQueue public SplitFetcherManagerSourceReaderBase( Supplier<SplitReader<ESplitFetcherManager<E, SplitT>>SplitT> splitReaderFactorysplitFetcherManager, Configuration configuration) { RecordEmitter<E, T, SplitStateT> recordEmitter, this(splitReaderFactory, configuration, (ignore) -> { Configuration config, }); SourceReaderContext context) { } public SplitFetcherManager( this(splitFetcherManager, recordEmitter, null, config, context); Supplier<SplitReader<E,} SplitT>> splitReaderFactory, // todo: provide a new constructor without Configuration configurationFutureCompletingBlockingQueue public SourceReaderBase( SplitFetcherManager<E, SplitT> splitFetcherManager, Consumer<Collection<String>> splitFinishedHook) {RecordEmitter<E, T, SplitStateT> recordEmitter, this.elementsQueue = new FutureCompletingBlockingQueue<>( @Nullable RecordEvaluator<T> eofRecordEvaluator, configuration.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY)); // ...... } | ||
Code Block | ||
| ||
@PublicEvolving public class SingleThreadFetcherManager<E, SplitT extends SourceSplit> Configuration config, SourceReaderContext extends SplitFetcherManager<E, SplitT> { context) { @Deprecated public SingleThreadFetcherManager( this.splitFetcherManager = splitFetcherManager; this.recordEmitter FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,= recordEmitter; this.splitStates = Supplier<SplitReader<E, SplitT>> splitReaderSupplier) {new HashMap<>(); this(elementsQueue, splitReaderSupplier,.options = new ConfigurationSourceReaderOptions()config); } @Deprecated this.config public SingleThreadFetcherManager(= config; this.context FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,= context; this.noMoreSplitsAssignment = false; Supplier<SplitReader<E, SplitT>> splitReaderSupplier, this.eofRecordEvaluator = eofRecordEvaluator; Configuration configuration) { numRecordsInCounter super(elementsQueue, splitReaderSupplier, configuration= context.metricGroup().getIOMetricGroup().getNumRecordsInCounter(); } } |
SingleThreadMultiplexSourceReaderBase
Deprecate the old constructor exposing the FutureCompletingBlockingQueue, and add new constructors as replacements which creates the FutureCompletingBlockingQueue instance in SplitFetcherManager internally.
Code Block |
---|
@PublicEvolving public abstract class @Deprecated @VisibleForTesting public SingleThreadFetcherManagerSingleThreadMultiplexSourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> extends SourceReaderBase<E, T, SplitT, SplitStateT> { @Depricated public SingleThreadMultiplexSourceReaderBase( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderSupplierSingleThreadFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration configurationconfig, Consumer<Collection<String>>SourceReaderContext splitFinishedHookcontext) { super(elementsQueue, splitReaderSuppliersplitFetcherManager, recordEmitter, configurationconfig, splitFinishedHookcontext); } @Depricated public SingleThreadFetcherManagerSingleThreadMultiplexSourceReaderBase( Supplier<SplitReader<EFutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, SplitT>> splitReaderSupplier) { Supplier<SplitReader<E, SplitT>> this(splitReaderSupplier, new Configuration()); } RecordEmitter<E, T, publicSplitStateT> SingleThreadFetcherManager(recordEmitter, Supplier<SplitReader<E, SplitT>> splitReaderSupplierConfiguration config, ConfigurationSourceReaderContext configurationcontext) { super(splitReaderSupplier, configuration); } elementsQueue, public SingleThreadFetcherManager( new Supplier<SplitReader<ESingleThreadFetcherManager<>(elementsQueue, SplitT>>splitReaderSupplier, splitReaderSupplierconfig), Configuration configurationrecordEmitter, Consumer<Collection<String>> splitFinishedHook) {config, super(splitReaderSupplier, configuration, splitFinishedHook); } } |
4. SplitFetcherManager provides wrapper methods for FutureCompletingBlockingQueue to replace its usage in SourceReaderBase.
Code Block |
---|
@PublicEvolving public abstract class SplitFetcherManager<E, SplitT extends SourceSplit> { /** * returns the RecordsWithSplitIds produced by SplitReader. **/ public RecordsWithSplitIds<E> poll(){ return elementsQueue.poll(); } /** context); } // todo: Add new constructors without FutureCompletingBlockingQueue public SingleThreadMultiplexSourceReaderBase( Supplier<SplitReader<E, SplitT>> splitReaderSupplier, * Returns the availability future. If theRecordEmitter<E, queueT, isSplitStateT> non-emptyrecordEmitter, then this future will already be * complete. Otherwise the obtained future is guaranteed to get completed the next time the Configuration config, SourceReaderContext context) { * queue becomes non-empty, or a notification happens via {@link #notifyAvailable()}. super( * new * <p>It is important that a completed future is no guarantee that the next call to {@linkSingleThreadFetcherManager<>(splitReaderSupplier, config),, recordEmitter, * #poll()} will return a non-null element. If there are concurrent consumerconfig, another consumer * may have taken the available element. Or there was no elementcontext); in the first place,} because the // todo: provide *a futurenew wasconstructor completedwithout throughFutureCompletingBlockingQueue a call to {@link #notifyAvailable()}.public SingleThreadMultiplexSourceReaderBase( * * <p>ForSingleThreadFetcherManager<E, thatSplitT> reasonsplitFetcherManager, it is important to call this method (to obtain a new future) every timeRecordEmitter<E, T, SplitStateT> recordEmitter, * again after {@link #poll()} returned null and@Nullable youRecordEvaluator<T> wanteofRecordEvaluator, to wait for data. */ public CompletableFuture<Void> getAvailabilityFuture(){ return elementsQueue.getAvailabilityFuture(); } /** Configuration config, * Makes sure the availabilitySourceReaderContext futurecontext) is{ complete, if it is not complete already. All futures super( * returned by previous calls to {@link #getAvailabilityFuture()} are guaranteed to besplitFetcherManager, * completed. * *recordEmitter, <p>All future calls to the method will return a completed future, until the point that theeofRecordEvaluator, * availability is reset via calls to {@link #poll()} that leave the queue empty. config, */ public void notifyAvailable(){ elementsQueue.notifyAvailable(); } /** Checks whether is no data available. */ public boolean noAvailableElement(){ return elementsQueue.isEmpty(); context); } } |
Compatibility, Deprecation, and Migration Plan
The Connectors that utilize constructors (including param elementsQueue) of SingleThreadMultiplexSourceReaderBase, SourceReaderBase, and SingleThreadFetcherManager should be migrated to the new constructor that does not have elementsQueue parameter.
Test Plan
Proposed Changes
- By exposing the SplitFetcherManager / SingleThreadFetcheManager, connector developers can easily create their own threading models in the SourceReaderBase, by implementing addSplits(), removeSplits() and
maybeShutdownFinishedFetchers() functions.
- Note that the SplitFetcher constructor is package private, so users can only create SplitFetchers via SplitFetcherManager.createSplitFetcher(). This ensures each SplitFetcher is always owned by the SplitFetcherManager.
- This FLIP essentially embedded the element queue (a FutureCompletingBlockingQueue) instance into the SplitFetcherManager. This hides the element queue from the connector developers and simplifies the SourceReaderBase to consist of only SplitFetcherManager and RecordEmitter as major components.
Compatibility, Deprecation, and Migration Plan
The Connectors that utilize constructors (including param elementsQueue) of SingleThreadMultiplexSourceReaderBase, SourceReaderBase, and SingleThreadFetcherManager should be migrated to the new constructor that does not have elementsQueue parameter.
Mark the impacted constructors as deprecated in 1.19, and remove them in release of 2.0.
Test Plan
nothing else to do.
Rejected Alternatives
change SplitFetcher to a public Interface
If SplitFetcherManager becomes PublicEvolving, that also means SplitFetcher needs to be PublicEvolving, because it is returned by the protected method SplitFetcherManager.createSplitFetcher()
However, SplitFetcher requires FutureCompletingBlockingQueue as a constructor parameter. SplitFetcher is a class rather than Interface. Therefore, I want to change SplitFetcher to a public Interface and moving its implementation details to an implement subclass .
Reject Reason
The constructor of the SplitFetcher is already package private. So it can only be accessed from the classes in the package org.apache.flink.connector.base.source.reader.fetcher. And apparently, user classes should not be in this package. Therefore, even if we mark the
SplitFetcher class as PublicEvolving, the constructor is not available to the users. Only the public and protected methods are considered public API
in this case. Private / package private methods and fields are still internal.
SplitFetcherManager expose poll / getAvailabilityFuture / notifyAvailable / noAvailableElement
- Change SplitFetcherManager from Internal to PublicEvolving.
- Deprecate the old constructor exposing the FutureCompletingBlockingQueue, and add new constructors as replacements which creates the FutureCompletingBlockingQueue instance internally.
- Add a new internal method SplitFetcherManager#getQueue() for SourceReaderBase usage.
|
Reject Reason
SplitFetcherManager exposes 4 more methods, poll / getAvailabilityFuture / notifyAvailable / noAvailableElement, which are tightly coupled with the implementation of the elementQueue. The naming of these methods look weird, like what does it mean to "poll from a SplitFetcherManager" / "notify a SplitFetcherManager available"? To clarify these methods we have to explain to developers that "well we hide a queue inside SplitFetcherMamager and the poll method is actually polling from the queue". I'm afraid these methods will implicitly expose the concept and the implementation of the queue to developers.
A cleaner solution would be having a new interface that extracts SplitFetcher creating and managing logic from the current SplitFetcherManager, but having too many concepts might make the entire Source API even harder to understand. To make a compromise, only expose constructors of SplitFetcherManager as public APIs, and adding a new internal method SplitFetcherManager#getQueue() for SourceReaderBase (well it's a bit hacky , more worthy than exposing methods like poll and notifyAvailable on SplitFetcherManager )nothing else to do.