Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.


Discussion threadhttps://lists.apache.org/thread/b8f509878ptwl3kmmgg95tv8sb1j5987
Vote threadhttps://lists.apache.org/thread/t1zff21z440pvv48jyhm8pgtqsyplchn

JIRA

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-3346532417

Release1.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?

  1. Though the SingleThreadFetcherManager is annotated as Internal, it actually acts as some-degree public API, which is widely used in many connector projects:

...

  1. flink-

...

  1. connector-

...

  1. kafka,  flink-connector-mongodb, flink-cdc-connectors and soon. In flink-connector-kafka, KafkaSourceFetcherManager even extends SingleThreadFetcherManager.
  2. More over, even the constructor

...

  1. of SingleThreadMultiplexSourceReaderBase

...

  1. (which is PublicEvolving) includes the params of SingleThreadFetcherManager 

...

  1. and FutureCompletingBlockingQueue

...

  1. . That means that the SingleThreadFetcherManager

...

  1. and FutureCompletingBlockingQueue

...

  1. have already been exposed to users for a long time and are widely used.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.

    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); }

...

  1. 
    
    



  2. The original design of SplitFetcherManager and its subclasses was to make them public to the Source developers. The

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: 

...

  1. 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 .  This will make the code slightly cleaner.

Public Interfaces

Change SingleThreadFetcherManager, SplitFetcherManager,  SplitFetcher and SplitFetcherTask from Internal  to PublicEvolving .

Proposed Changes

...


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 SourceReaderBase<E, TSplitFetcherManager<E, SplitT extends SourceSplit, SplitStateT>
        implements SourceReader<T, SplitT> SourceSplit> {   
 
 
    @Deprecated
    	public SourceReaderBaseSingleThreadMultiplexSourceReaderBase(
            FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueueSupplier<SplitReader<E,
            SplitFetcherManager<E, SplitT> splitFetcherManager,SplitT>> splitReaderSupplier,
            RecordEmitter<E, T, SplitStateT> recordEmitter,
            Configuration config,
            SourceReaderContext context) {
        thissuper(elementsQueue,
  splitFetcherManager, recordEmitter, null, config, context);
    }
 
 
    @Deprecated
  new SingleThreadFetcherManager<>(splitReaderSupplier, config),
  public SourceReaderBase(
            FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueuerecordEmitter,
              SplitFetcherManager<E, SplitT> splitFetcherManagerconfig,
             RecordEmitter<E, T, SplitStateT> recordEmitter,context);
    }

    @Deprecated
      @Nullable RecordEvaluator<T> eofRecordEvaluator,public SplitFetcherManager(
            ConfigurationFutureCompletingBlockingQueue<RecordsWithSplitIds<E>> configelementsQueue,
            SourceReaderContextSupplier<SplitReader<E, context)SplitT>> {
splitReaderFactory,
            //this.elementsQueue = elementsQueue;Configuration configuration) {
        this.splitFetcherManager = splitFetcherManager;
  (elementsQueue, splitReaderFactory, configuration, (ignore) -> {
      this.recordEmitter = recordEmitter});
    }      this.splitStates = new HashMap<>();  

    @Deprecated
    @VisibleForTesting
    this.options = new SourceReaderOptions(config);public SplitFetcherManager(
        this.config = config;
  FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
     this.context = context;
     Supplier<SplitReader<E, SplitT>> splitReaderFactory,
 this.noMoreSplitsAssignment = false;
        this.eofRecordEvaluator =Configuration eofRecordEvaluator;configuration,
 
        numRecordsInCounter = context.metricGroup().getIOMetricGroup().getNumRecordsInCounter();
    } 
 
     Consumer<Collection<String>> splitFinishedHook) {

	}



 // todo: provide a new constructor without FutureCompletingBlockingQueue
.     
 public SourceReaderBaseSplitFetcherManager(
            SplitFetcherManager<ESupplier<SplitReader<E, SplitT>SplitT>> splitFetcherManagersplitReaderFactory,
            RecordEmitter<E, T, SplitStateT> recordEmitter,Configuration configuration) {
        this(splitReaderFactory, configuration, (ignore)  Configuration config,-> {
        });
    SourceReaderContext context) {
  
  }

  //  this(splitFetcherManager, recordEmitter, null, config, context);
    }
 
 
    public SourceReaderBasetodo: provide a new constructor without FutureCompletingBlockingQueue.
  public SplitFetcherManager(
            SplitFetcherManager<ESupplier<SplitReader<E, SplitT>SplitT>> splitFetcherManagersplitReaderFactory,
            RecordEmitter<E,Configuration Tconfiguration, SplitStateT> recordEmitter,
            @NullableConsumer<Collection<String>> RecordEvaluator<T>splitFinishedHook) eofRecordEvaluator,{
        this.elementsQueue = new FutureCompletingBlockingQueue<>(
 Configuration  config,
            SourceReaderContext context) {
        this.splitFetcherManager = splitFetcherManager;
        this.recordEmitter = recordEmitter;
        this.splitStates = new HashMap<>();
        this.options = new SourceReaderOptions(config); 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>
        extends SplitFetcherManager<E, SplitT> {  

   	@Deprecated
    public SingleThreadFetcherManager(
        this.config = config;
  FutureCompletingBlockingQueue<RecordsWithSplitIds<E>>   elementsQueue,
   this.context = context;
       Supplier<SplitReader<E, this.noMoreSplitsAssignment = false;SplitT>> splitReaderSupplier) {
        this.eofRecordEvaluator = eofRecordEvaluator;
 (elementsQueue, splitReaderSupplier, new Configuration());
    }

  	@Deprecated
  numRecordsInCounter  =public context.metricGroup().getIOMetricGroup().getNumRecordsInCounter();SingleThreadFetcherManager(
    }
}
Code Block
@PublicEvolving
public abstract class SingleThreadMultiplexSourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> extends SourceReaderBase<E,FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> TelementsQueue,
 SplitT, SplitStateT> { 

	@Depricated
	public SingleThreadMultiplexSourceReaderBase(
       Supplier<SplitReader<E, SplitT>> splitReaderSupplier,
   FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
         Configuration configuration) {
        SingleThreadFetcherManager<Esuper(elementsQueue, SplitT> splitFetcherManager,
  splitReaderSupplier, configuration);
    }

    @Deprecated
    @VisibleForTesting
  RecordEmitter<E, T, SplitStateT>public recordEmitter,SingleThreadFetcherManager(
            ConfigurationFutureCompletingBlockingQueue<RecordsWithSplitIds<E>> configelementsQueue,
            SourceReaderContextSupplier<SplitReader<E, context)SplitT>> {splitReaderSupplier,
        super(elementsQueue, splitFetcherManager, recordEmitter, config, context);
 	}

	@Depricated
	public SingleThreadMultiplexSourceReaderBase(
    Configuration configuration,
            Consumer<Collection<String>>  FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,splitFinishedHook) {
            Supplier<SplitReader<Esuper(elementsQueue, SplitT>> splitReaderSupplier,
 configuration, splitFinishedHook);
    }

  // todo: provide a RecordEmitter<E,new T,constructor SplitStateT> recordEmitter,
 without FutureCompletingBlockingQueue.     
  public SingleThreadFetcherManager(
     Configuration config,
      Supplier<SplitReader<E,      SourceReaderContext contextSplitT>> splitReaderSupplier) {
        super(
splitReaderSupplier, new Configuration());
    }  

 // todo: provide a new constructor without FutureCompletingBlockingQueue. elementsQueue,
     public SingleThreadFetcherManager(
            new SingleThreadFetcherManager<>(elementsQueue,Supplier<SplitReader<E, SplitT>> splitReaderSupplier,
 config),
           Configuration configuration) {
   recordEmitter,
                config,super(splitReaderSupplier, configuration);
                context);
	}


	 // todo: provide Adda new constructorsconstructor without FutureCompletingBlockingQueue. 
 	    public SingleThreadMultiplexSourceReaderBaseSingleThreadFetcherManager(
            Supplier<SplitReader<E, SplitT>> splitReaderSupplier,
            RecordEmitter<E,Configuration Tconfiguration, SplitStateT> recordEmitter,
            Configuration config,
            SourceReaderContext context) Consumer<Collection<String>> splitFinishedHook) {
        	super(
                new SingleThreadFetcherManager<>(splitReaderSupplier, config),,splitReaderSupplier, configuration, splitFinishedHook);
    }
}


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 recordEmitter,
{    
	SplitFetcher(
            int  configid,
                context);FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
    }

    public SingleThreadMultiplexSourceReaderBase(
            SingleThreadFetcherManager<ESplitReader<E, SplitT> splitFetcherManagersplitReader,
            RecordEmitter<E,Consumer<Throwable> TerrorHandler, SplitStateT> recordEmitter,
            @NullableRunnable RecordEvaluator<T> eofRecordEvaluatorshutdownHook,
            ConfigurationConsumer<Collection<String>> configsplitFinishedHook,
            SourceReaderContextboolean contextallowUnalignedSourceSplits);
}


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>
 {
        super(
             implements SourceReader<T, SplitT> splitFetcherManager,{  
 
 
    @Deprecated
    public SourceReaderBase(
     recordEmitter       FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
            SplitFetcherManager<E, SplitT> splitFetcherManager,
  eofRecordEvaluator,
             RecordEmitter<E, T, SplitStateT> recordEmitter,
            Configuration config,
            SourceReaderContext context) {
        this(elementsQueue, splitFetcherManager, recordEmitter, null, config, context);
    }
}

3. Mark constructor of SplitFetcherManager andSingleThreadFetcherManager as  @Depricated and provide a new constructor without FutureCompletingBlockingQueue. Mark SplitFetcherManager andSingleThreadFetcherManager as `@PublicEvolving`.

Code Block
titleSplitFetcherManager
@PublicEvolving
public abstract class SplitFetcherManager<E, SplitT extends SourceSplit> {        
	
 
 
    @Deprecated
    public SplitFetcherManagerSourceReaderBase(
            FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
            Supplier<SplitReader<ESplitFetcherManager<E, SplitT>>SplitT> splitReaderFactorysplitFetcherManager,
            Configuration configuration) {
 RecordEmitter<E, T, SplitStateT> recordEmitter,
       this(elementsQueue, splitReaderFactory, configuration, (ignore) -> {
@Nullable RecordEvaluator<T> eofRecordEvaluator,
      });
    }    Configuration  config,
 

    @Deprecated
    @VisibleForTesting
   SourceReaderContext publiccontext) SplitFetcherManager({
        //this.elementsQueue = elementsQueue;
  FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
     this.splitFetcherManager = splitFetcherManager;
     Supplier<SplitReader<E, SplitT>> splitReaderFactory,
    this.recordEmitter = recordEmitter;
        this.splitStates = Configuration configuration,new HashMap<>();
        this.options =   Consumer<Collection<String>> splitFinishedHook) {

	}



 // todo: provide a new constructor without FutureCompletingBlockingQueue.     
 public SplitFetcherManager(
new SourceReaderOptions(config);
        this.config = config;
        this.context = context;
   Supplier<SplitReader<E, SplitT>> splitReaderFactory,
   this.noMoreSplitsAssignment = false;
       Configuration configuration) { this.eofRecordEvaluator = eofRecordEvaluator;
 
        this(splitReaderFactory, configuration, (ignore) -> {numRecordsInCounter = context.metricGroup().getIOMetricGroup().getNumRecordsInCounter();
    }  
  });
    // todo: provide a new constructor without FutureCompletingBlockingQueue
   }

 public SplitFetcherManagerSourceReaderBase(
            Supplier<SplitReader<ESplitFetcherManager<E, SplitT>>SplitT> splitReaderFactorysplitFetcherManager,
            Configuration configurationRecordEmitter<E, T, SplitStateT> recordEmitter,
            Configuration config,
 Consumer<Collection<String>> splitFinishedHook) {
        this.elementsQueue =SourceReaderContext newcontext) FutureCompletingBlockingQueue<>({
        this(splitFetcherManager, recordEmitter, null, config, context);
    configuration.getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY));
		// ......
}
Code Block
titleSingleThreadFetcherManager
@PublicEvolving
public class SingleThreadFetcherManager<E, SplitT extends SourceSplit>
        extends SplitFetcherManager<E, SplitT> {  
   	@Deprecated
    public SingleThreadFetcherManager(
     }
 
 
     // todo: provide a new constructor without FutureCompletingBlockingQueue
     public SourceReaderBase(
            SplitFetcherManager<E, SplitT> splitFetcherManager,
       FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
    RecordEmitter<E, T, SplitStateT> recordEmitter,
     Supplier<SplitReader<E, SplitT>> splitReaderSupplier) {
    @Nullable RecordEvaluator<T>   this(elementsQueueeofRecordEvaluator,
 splitReaderSupplier, new Configuration());
    }

  	@Deprecated
   Configuration public SingleThreadFetcherManager(
config,
            SourceReaderContext FutureCompletingBlockingQueue<RecordsWithSplitIds<E>>context) elementsQueue,{
        this.splitFetcherManager    Supplier<SplitReader<E, SplitT>> splitReaderSupplier,= splitFetcherManager;
        this.recordEmitter = recordEmitter;
  Configuration configuration) {
    this.splitStates = new  super(elementsQueue, splitReaderSupplier, configurationHashMap<>();
    }

    @Deprecated
this.options =   @VisibleForTestingnew SourceReaderOptions(config);
    public SingleThreadFetcherManager(
   this.config = config;
       FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
   this.context = context;
        this.noMoreSplitsAssignment  Supplier<SplitReader<E, SplitT>> splitReaderSupplier,= false;
        this.eofRecordEvaluator =   Configuration configuration,eofRecordEvaluator;
 
        numRecordsInCounter    Consumer<Collection<String>> splitFinishedHook) {
        super(= 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 SingleThreadMultiplexSourceReaderBase<E, T, SplitT extends SourceSplit, SplitStateT> extends SourceReaderBase<E, T, SplitT, SplitStateT> { 

	@Depricated
	public SingleThreadMultiplexSourceReaderBase(
 elementsQueue, splitReaderSupplier, configuration, splitFinishedHook);
    }

 	public SingleThreadFetcherManager(
           FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> Supplier<SplitReader<EelementsQueue,
 SplitT>>  splitReaderSupplier) {
        this(splitReaderSupplierSingleThreadFetcherManager<E, new Configuration());SplitT> splitFetcherManager,
    }


        RecordEmitter<E, T, publicSplitStateT> SingleThreadFetcherManager(recordEmitter,
            Supplier<SplitReader<E, SplitT>> splitReaderSupplierConfiguration config,
            ConfigurationSourceReaderContext configurationcontext) {
        super(splitReaderSupplierelementsQueue, splitFetcherManager, recordEmitter, config, configurationcontext);
    	}

 	@Depricated
	public SingleThreadMultiplexSourceReaderBase(
      public SingleThreadFetcherManager(      FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueue,
            Supplier<SplitReader<E, SplitT>> splitReaderSupplier,
            RecordEmitter<E, T, SplitStateT> recordEmitter,
            Configuration configurationconfig,
            Consumer<Collection<String>>SourceReaderContext splitFinishedHookcontext) {
        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();
	}

 	/**

                elementsQueue,
                new SingleThreadFetcherManager<>(elementsQueue, splitReaderSupplier, config),
       * Returns the availability future. If the queue is non-emptyrecordEmitter,
 then this future will already be
     * complete. Otherwise the obtained futureconfig,
 is guaranteed to get completed the next time the
     * queue becomes non-empty, or a notification happens via {@link #notifyAvailable()}. context);
	}


	// todo: Add new constructors without FutureCompletingBlockingQueue
 	public SingleThreadMultiplexSourceReaderBase(
     *
     * <p>It isSupplier<SplitReader<E, importantSplitT>> thatsplitReaderSupplier,
 a completed future is no guarantee that the next call to {@link
   RecordEmitter<E, T, SplitStateT> recordEmitter,
  * #poll()} will return a non-null element. If there are concurrentConfiguration consumerconfig, another consumer
     * may have taken the available element. OrSourceReaderContext therecontext) was{
 no element in the first place, because the	super(
     * future was completed through a call to {@link #notifyAvailable()}.
  new SingleThreadFetcherManager<>(splitReaderSupplier, config),,
 *
     * <p>For that reason, it is important to call this methodrecordEmitter,
 (to obtain a new future) every time
     * again after {@link #poll()} returned null and you want to wait for data.
 config,
              */
	public CompletableFuture<Void> getAvailabilityFuture(context){;
 		return elementsQueue.getAvailabilityFuture();
	   }

      /**
/ todo: provide a new * 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 beconstructor without FutureCompletingBlockingQueue
     public SingleThreadMultiplexSourceReaderBase(
            SingleThreadFetcherManager<E, SplitT> splitFetcherManager,
            RecordEmitter<E, T, SplitStateT> recordEmitter,
     * completed.
     *
 @Nullable RecordEvaluator<T> eofRecordEvaluator,
  * <p>All future calls to the method will return a completedConfiguration futureconfig,
 until the point that the
     * availability isSourceReaderContext resetcontext) via{
 calls to {@link #poll()} that leave the queue empty.
  super(
                */
	public void notifyAvailable(){
		elementsQueue.notifyAvailable();
	}

splitFetcherManager,
      /** Checks whether is no data available. */
 	public boolean noAvailableElement(){
		return elementsQueue.isEmpty();
 	}
}

5. Mark SplitFetcher and SplitFetcherTask as PublicEvolving.

Code Block
@PublicEvolving
public class SplitFetcher<E, SplitT extends SourceSplit> implements Runnable {
    FetchTask(
   recordEmitter,
                eofRecordEvaluator,
          SplitReader<E, SplitT> splitReader,
            FutureCompletingBlockingQueue<RecordsWithSplitIds<E>> elementsQueueconfig,
            Consumer<Collection<String>> splitFinishedCallback,
   context);
         int fetcherIndex) {
        this.splitReader = splitReader;
        this.elementsQueue = elementsQueue;
        this.splitFinishedCallback = splitFinishedCallback;
        this.lastRecords = null;
        this.fetcherIndex = fetcherIndex;
        this.wakeup = false;
    }
}
Code Block
@PublicEvolving
public interface SplitFetcherTask {}

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

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

}
}


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.


 @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) -> {
        });
         
  }
 
 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();
    }
}

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