Discussion thread | https://lists.apache.org/thread/b8f509878ptwl3kmmgg95tv8sb1j5987 |
---|---|
Vote thread | https://lists.apache.org/thread/t1zff21z440pvv48jyhm8pgtqsyplchn |
JIRA |
|
Release | 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:flink-connector-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 ofSingleThreadFetcherManager
andFutureCompletingBlockingQueue
. That means that theSingleThreadFetcherManager
andFutureCompletingBlockingQueue
have already been exposed to users for a long time and are widely used.As shown inFLINK-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.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 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.
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.
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 simplify the SourceReaderBase to consist of only SplitFetcherManager and RecordEmitter as major components.
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 few new methods to expose the functionality of the internal FutureCompletingBlockingQueue via the SplitFetcherManager.
@PublicEvolving public abstract class SplitFetcherManager<E, SplitT extends SourceSplit> { @Deprecated public SplitFetcherManager( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderFactory, Configuration configuration) { this(elementsQueue, splitReaderFactory, configuration, (ignore) -> { }); } @Deprecated @VisibleForTesting public SplitFetcherManager( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderFactory, Configuration configuration, Consumer<Collection<String>> splitFinishedHook) { } // todo: provide a new constructor without FutureCompletingBlockingQueue. public SplitFetcherManager( Supplier<SplitReader<E, SplitT>> splitReaderFactory, Configuration configuration) { this(splitReaderFactory, configuration, (ignore) -> { }); } // todo: provide a new constructor without FutureCompletingBlockingQueue. public SplitFetcherManager( Supplier<SplitReader<E, SplitT>> splitReaderFactory, Configuration configuration, Consumer<Collection<String>> splitFinishedHook) { this.elementsQueue = new FutureCompletingBlockingQueue<>( configuration.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY)); // ...... } /** * returns the RecordsWithSplitIds produced by SplitReader. **/ public RecordsWithSplitIds<E> poll(){ return elementsQueue.poll(); } /** * Returns the availability future. If the queue is non-empty, then this future will already be * complete. Otherwise the obtained future is guaranteed to get completed the next time the * queue becomes non-empty, or a notification happens via {@link #notifyAvailable()}. * * <p>It is important that a completed future is no guarantee that the next call to {@link * #poll()} will return a non-null element. If there are concurrent consumer, another consumer * may have taken the available element. Or there was no element in the first place, because the * future was completed through a call to {@link #notifyAvailable()}. * * <p>For that reason, it is important to call this method (to obtain a new future) every time * again after {@link #poll()} returned null and you want to wait for data. */ public CompletableFuture<Void> getAvailabilityFuture(){ return elementsQueue.getAvailabilityFuture(); } /** * Makes sure the availability future is complete, if it is not complete already. All futures * returned by previous calls to {@link #getAvailabilityFuture()} are guaranteed to be * completed. * * <p>All future calls to the method will return a completed future, until the point that the * availability is reset via calls to {@link #poll()} that leave the queue empty. */ public void notifyAvailable(){ elementsQueue.notifyAvailable(); } /** Checks whether is no data available. */ public boolean noAvailableElement(){ return elementsQueue.isEmpty(); } }
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.
@PublicEvolving public class SingleThreadFetcherManager<E, SplitT extends SourceSplit> extends SplitFetcherManager<E, SplitT> { @Deprecated public SingleThreadFetcherManager( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderSupplier) { this(elementsQueue, splitReaderSupplier, new Configuration()); } @Deprecated public SingleThreadFetcherManager( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderSupplier, Configuration configuration) { super(elementsQueue, splitReaderSupplier, configuration); } @Deprecated @VisibleForTesting public SingleThreadFetcherManager( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderSupplier, Configuration configuration, Consumer<Collection<String>> splitFinishedHook) { super(elementsQueue, splitReaderSupplier, configuration, splitFinishedHook); } // todo: provide a new constructor without FutureCompletingBlockingQueue. public SingleThreadFetcherManager( Supplier<SplitReader<E, SplitT>> splitReaderSupplier) { this(splitReaderSupplier, new Configuration()); } // todo: provide a new constructor without FutureCompletingBlockingQueue. public SingleThreadFetcherManager( Supplier<SplitReader<E, SplitT>> splitReaderSupplier, Configuration configuration) { super(splitReaderSupplier, configuration); } // todo: provide a new constructor without FutureCompletingBlockingQueue. public SingleThreadFetcherManager( Supplier<SplitReader<E, SplitT>> splitReaderSupplier, Configuration configuration, Consumer<Collection<String>> splitFinishedHook) { super(splitReaderSupplier, configuration, splitFinishedHook); } }
SplitFetcherTask
Change SplitFetcherTask from Internal to PublicEvolving.
@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
@PublicEvolving public class SplitFetcher<E, SplitT extends SourceSplit> implements Runnable { SplitFetcher( int id, FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, SplitReader<E, SplitT> splitReader, Consumer<Throwable> errorHandler, Runnable shutdownHook, Consumer<Collection<String>> splitFinishedHook, 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.
- Remove the direct operations to FutureCompletingBlockingQueue and utilize the functionality of SplitFetcherManager as an alternative.
@PublicEvolving public abstract class SourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> implements SourceReader<T, SplitT> { @Deprecated public SourceReaderBase( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, SplitFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration config, SourceReaderContext context) { this(elementsQueue, splitFetcherManager, recordEmitter, null, config, context); } @Deprecated public SourceReaderBase( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, SplitFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, @Nullable RecordEvaluator<T> eofRecordEvaluator, Configuration config, SourceReaderContext context) { //this.elementsQueue = elementsQueue; this.splitFetcherManager = splitFetcherManager; this.recordEmitter = recordEmitter; this.splitStates = new HashMap<>(); this.options = new SourceReaderOptions(config); this.config = config; this.context = context; this.noMoreSplitsAssignment = false; this.eofRecordEvaluator = eofRecordEvaluator; numRecordsInCounter = context.metricGroup().getIOMetricGroup().getNumRecordsInCounter(); } // todo: provide a new constructor without FutureCompletingBlockingQueue public SourceReaderBase( SplitFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration config, SourceReaderContext context) { this(splitFetcherManager, recordEmitter, null, config, context); } // todo: provide a new constructor without FutureCompletingBlockingQueue public SourceReaderBase( SplitFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, @Nullable RecordEvaluator<T> eofRecordEvaluator, Configuration config, SourceReaderContext context) { this.splitFetcherManager = splitFetcherManager; this.recordEmitter = recordEmitter; this.splitStates = new HashMap<>(); this.options = new SourceReaderOptions(config); this.config = config; this.context = context; this.noMoreSplitsAssignment = false; this.eofRecordEvaluator = eofRecordEvaluator; numRecordsInCounter = 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.
@PublicEvolving public abstract class SingleThreadMultiplexSourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> extends SourceReaderBase<E, T, SplitT, SplitStateT> { @Depricated 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); } @Depricated public SingleThreadMultiplexSourceReaderBase( FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue, Supplier<SplitReader<E, SplitT>> splitReaderSupplier, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration config, SourceReaderContext context) { super( elementsQueue, new SingleThreadFetcherManager<>(elementsQueue, splitReaderSupplier, config), recordEmitter, config, context); } // todo: Add new constructors without FutureCompletingBlockingQueue public SingleThreadMultiplexSourceReaderBase( Supplier<SplitReader<E, SplitT>> splitReaderSupplier, RecordEmitter<E, T, SplitStateT> recordEmitter, Configuration config, SourceReaderContext context) { super( new SingleThreadFetcherManager<>(splitReaderSupplier, config),, recordEmitter, config, context); } // todo: provide a new constructor without FutureCompletingBlockingQueue public SingleThreadMultiplexSourceReaderBase( SingleThreadFetcherManager<E, SplitT> splitFetcherManager, RecordEmitter<E, T, SplitStateT> recordEmitter, @Nullable RecordEvaluator<T> eofRecordEvaluator, Configuration config, SourceReaderContext context) { super( splitFetcherManager, recordEmitter, eofRecordEvaluator, config, context); } }
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
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.