FLIP-27:重构Source接口
由Aljoscha Krettek创建
状态
当前状态: 讨论中
讨论线索: https://lists.apache.org/thread.html/70484d6aa4b8e7121181ed8d5857a94bfb7d5a76334b9c8fcc59700c@%3Cdev.flink.apache.org%3E
JIRA: FLINK-10740 - Refactor Source Interface IN PROGRESS
Released: Flink Version
请将讨论保留在邮件列表上,而不要在wiki上发表评论(wiki讨论显得比较慢)
动机
这个FLIP旨在解决当前source接口(SourceFunction)的一些问题或缺点,同时统一批和流的source接口API。我们想要解决的缺点或要点是:
- 当前有一个缺点是批和流执行实现不同的source接口
- 用于"work发现"(splits, partitions等)和实际"读取" 数据的逻辑混合在sourcefunction接口和datastream api中,导致像kafka和kinisis源这样的复杂实现。
- Partitions/shards/splits在接口中不是显示的。这使得很难以独立源的方式实现某些功能,例如事件时间对齐,每个分区的watermarks,动态的分片分配,work stealing(待翻译)。例如kafka和Kinesis消费者支持每个分区的watermarks,但从Flink1.8.1开始,只有kinisis消费者支持事件时间对齐(有选择地读取splits,以确保我们在事件时间中均匀前进)。
- checkpoint锁在source function接口中持有。它的实现必须确保在拿到锁的情况下进行元素发射和状态更新。Flink没有办法优化它如何处理那把锁。这把锁不是公平锁。在锁争用下,某些线程可能无法获得锁(检查点线程)。这也阻碍了operators使用actor/mailbox样式的无锁线程模型。
- 缺少通用的代码块,这意味着每个source单独实现一个复杂的线程模型,使得实现和测试一个新的source很困难,并且增加了对现有源代码的贡献的高门槛。
总体设计
在设计中有几个关键的方面,在每个部分都会讨论。
Work发现与读取的分离
source有两个核心的组件:
- SplitEnumerator:发现和分配splits(文件,分区等)
- Reader:从splits读取实际数据
SplitEnumerator 类似于旧的批处理source接口的splits创建和splits分配的功能。它只会运行一次,没有并行(如果有必要的话,可以在将来支持并行)。
它可以在在Jobmanager中运行,也可以在TaskManager的一个单独线程中运行(参见"Where to run the Enumerator")。
例如:
- 在File source中,SplitEnumerator负责列出所有文件(可能将它们细分为块/范围)
- 在Kafka source中,SplitEnumerator负责发现source应该读取的Kafka分区。
Reader从分配到的splits读取数据。reader包含了当前source接口的大部分功能。一些reader可能读取一系列有界的splits,一些可能并发的读取多个(无界)的splits。
enumerator和reader之间的分离允许将不同的enumeration策略与split readers混合和适配。例如,当前Kafka connector有不同的策略去发现分区,这些策略与其余代码混合在一起。有了新的接口,我们只需要一个split reader实现,并且可以有多个split enumerators用于不同的分区发现策略。
这两个组件封装了核心功能,Source接口本身只是用来创建split enumerators和readers
批流统一
每个source都应该能够作为有界(批)和无界(流)的source来运行
目前要讨论的是,在创造source stream时,它在 DataStream API中成为有界还是无界的
有界性是在创建SplitEnumerator时传递给源的属性。readers对这个属性并不可知,它们只是从分配好的splits读取。无论源是有界的还是无界的,都会在创建时传递给splitenumerator。
这样,当我们可以明确有界流模型时,我们还可以使api类型在将来是安全的
// DataStream API
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
FileSource<MyType> theSource = new ParquetFileSource("fs:///path/to/dir", AvroParquet.forSpecific(MyType.class));
DataStream<MyType> stream = env.continuousSource(theSource);
DataStream<MyType> boundedStream = env.boundedSource(theSource);
// 一旦我们将有界流添加到datastream api中,这将是一个选项。
BoundedDataStream<MyType> batch = env.boundedSource(theSource);
例子
FileSource
- 对于“bounded input”,它使用一个SplitEnumerator枚举给定路径下的所有文件
- 对于"continuous input",它使用一个SplitEnumerator定期枚举给定路径下的所有文件并分配新文件
KafkaSource
- 对于“bounded input”,它使用一个SplitEnumerator列出所有partition同时获取每个partition最新的offset,并将其作为结束的offset附到split上
- 对于“continuous input”,它使用一个SplitEnumerator列出所有partition同时将LONG_MAX作为结束offset附到split上
- 这个source可能又另外的选项去周期的发现新的分区。这仅仅可以应用于“continuous input”。
Reader接口和多线程模型
reader需要实现以下几点
- 不停止的循环,所以不需要管理锁
- 非阻塞的方法,它支持在actor/mailbox/dispatcher样式的operator中运行
- 所有的方法通过单线程调用,所以实现不需要处理并发性
- Watermark/事件时间处理抽象为可扩展的感知split和对齐(请参见下面的 “每个split的事件时间"和"事件时间对齐”)
- 所有的reader应该很自然的支持状态和checkpoint
- 批量执行时应避免生成Watermark
以下核心方面为我们提供了这些特性:
- splits既可以是work分配的类型,也可以是source持有的状态类型。对于reader来说split分配或从checkpoint恢复split是相同的。
- 推进reader是一个无阻塞的调用,它返回一个future
- 我们在主要的接口上面建立高级的原语(参见下面的“高级Readers”)
- 我们在SourceOutput中隐藏事件时间/watermarks并为批处理(没有watermark)和流处理(有watermark)传递不同的source上下文
// SourceReader
interface SourceReader {
void start() throws IOException;
CompletableFuture<?> available() throws IOException;
ReaderStatus emitNext(SourceOutput<E> output) throws IOException;
void addSplits(List<SplitT> splits) throws IOException;
List<SplitT> snapshotState();
}
该实现假设有一个单独的线程驱动source,当数据可用时,此单线程调用emitNext(…),当没有可用数据时,则放弃执行。它还处理checkpoint触发,计时器回调等,从而使任务可以摆脱锁。这个线程预计会在某种形式的mailbox上工作,其中数据发射循环可能是一个线程(与计时器、checkpoint等类似)。下面是一个非常简单的驱动循环伪代码
这种类型的循环确保我们不会使用future,除非reader暂时没有数据,并且我们会绕过mailbox以提高性能(mailbox将需要一定量的同步)
// reader循环伪代码
final BlockingQueue<Runnable> mailbox = new LinkedBlockingQueue<>();
final SourceReader<T> reader = ...;
final Runnable readerLoop = new Runnable() {
public void run() {
while (true) {
ReaderStatus status = reader.emitNext(output);
if (status == MORE_AVAILABLE) {
if (mailbox.isEmpty()) {
continue;
}
else {
addReaderLoopToMailbox();
break;
}
}
else if (status == NOTHING_AVAILABLE) {
reader.available().thenAccept((ignored) -> addReaderLoopToMailbox());
break;
}
else if (status == END_OF_SPLIT_DATA) {
break;
}
}
};
void addReaderLoopToMailbox() {
mailbox.add(readerLoop);
}
void triggerCheckpoint(CheckpointOptions options) {
mailbox.add(new CheckpointAction(options));
}
void taskMainLoop() {
while (taskAlive) {
mailbox.take().run();
}
}
}
高级Readers
核心source接口(低级接口)非常普通。这使得它很灵活,但很难为贡献者实现,特别是对于足够复杂的reader模式,如Kafka或Kinesis。
通常,用于连接器的大多数I/O库不是异步的,需要生成一个I/O线程使得他们对主线程是非阻塞的。
我们建议通过构建更高级别的source抽象来解决这个问题,这些抽象提供了更简单的接口,允许阻塞调用。
这些更高级别的抽象还将解决并发处理多个splits的source和每个split事件时间逻辑的问题。
大多数readers分为以下几类:
- 单个split(Sequential Single Split)(文件,数据库查询,大多数有界的splits)
- 多个split多路复用(Multi-split multiplexed)(Kafka,Pulser,Pravega…)
- 多个split多线程(Multi-split multi-threaded)
大多数针对这些高级构建块实现的reader只需要实现与此类似的接口,约定还将是除wakeup()之外的所有方法都将由同一线程调用,从而消除了连接器中任何并发处理的需要。
// SourceReader reading methods
interface SplitReader<RecordsT> extends Closeable {
@Nullable
RecordsT fetchNextRecords(Duration timeout) throws IOException;
void wakeup();
}
每个split事件时间
待决定
事件时间对齐
待决定
Enumerator运行在哪儿
Enumerator和SourceReader之间的split的通信有特殊的要求:
- Lazy/基于拉取的分配:只有当reader请求下一个split时enumerator才发送一个split。这样可以实现更好的负载均衡。
- "pull"消息上的有效负载,用于将诸如“location”之类的信息从SourceReader传递到SplitEnumerator,因此支持像位置感知的split分配这样的features。
- 基于checkpoint的仅有一次容错:一次将split发送给reader。split要么仍然是enumerator(及其检查点)的一部分,要么是reader的一部分,要么已经完成。
- checkpoint(或者没有checkpoint)之间的仅有一次:在检查点之间(或者没有检查点),分配给reader的split必须在失败/恢复时重新添加到enumerator中。
- 通信通道不能将任务连接到单个故障区域
鉴于这些要求,将有两个选项来实现这种通信。
Option1:Enumerator运行在TaskManager上
SplitEnumerator运行在并行度为1的Task上,enumerator的下游是SourceReader任务,SourceReader任务是并行运行的。通过常规的数据流来通信。
readers通过发送“backwards events”来请求splits,类似于在批量迭代中发送“request partition”或“superstep synchronization”。这些不对operators公开,但是tasks可以访问他们。
Task对backwards events做出反应:只有在一个事件发生时才会发送一个split。这就给了我们laze/基于拉取的分配方式。请求backwards event消息(例如位置感应)上的有效负载是有可能的。
Checkpoints和splits是自然对齐的,因为splits通过数据通道。enumerator实际上是source中唯一的入口task,也是唯一接收“trigger checkpoint” RPC调用的入口task。
调度器将enumerator和split reader之间的网络连接视为故障区域的边界。
要解除enumerator和reader启动的耦合,我们需要以下机制之一:
- Pipelined persistent channels(流水线持久通道):通道的内容在两次checkpoint之间是持久的。接收任务请求“检查点x之后”的数据。检查点x+1完成时,数据将被清理。
当一个reader失败,从checkpoint重新恢复reader任务之后可以重新连接到的流,并将获得之前分配的splits。批处理是一个特例,如果没有checkpoint,通道将保留从一开始的所有数据。
- Pro:“Pipelined persistent channels”不仅限于应用在enumerator到reader连接上
- Con:split总是指向同一个reader,恢复时不能跨多个reader分发。尤其是批处理程序,这可能会在恢复中造成严重的滞后。
- Reconnects and task notifications on failures(失败时重连和任务通知):enumerator task需要记住每一个结果分区的splits分配直到下一次checkpoint完成。enumerator task必须在失败时通知下游task,并将split添加回enumerator。恢复的reader任务只需要重新连接并获取新的流。
- Pro:在失败/恢复时跨所有reader重新分配split(无分离)。
- Con:中断分离任务和网络栈的抽象
Option2:Enumerator运行在JobManager上
和目前的批(DataSet)输入split分配类似,SplitEnumerator代码运行在JobManager上,作为ExecutionJobVertex的一部分。为了支持周期split发现,必须有一个额外的线程来周期调用enumerator。
reader通过RPC消息请求split,enumerator通过RPC响应。RPC消息携带负载信息比如location。
我需要格外小心,以便将split分配消息与checkpoint barrier对齐,如果我们开始支持基于元数据的watermark(在处理有界的split时一致的处理事件时间),我们还需要通过RPC来支持,并将其与输入split分配对齐。
enumerator在触发检查点时创建自己的检查点状态。
这里关键是在master(ExecutionGraph)和checkpoint上增加了复杂性。当通过现在的单线程执行图dispatcher线程时,可以将它们与rpc消息正确地对齐,但是要支持异步检查点写入需要更高的复杂性。
Option3:引入一个叫SourceCoordinator的独立组件,Enumerator运行在SourceCoordinator上
SourceCoordinator是一个独立的组件,不是ExecutionGraph的一部分。SourceCoordinator可以是运行在JobMaster上或者作为一个独立进程。这里不受设计限制。SourceCoordinator (Enumerator)通过RPC通信。Split分配通过RPC支持pull-based。SourceReader需要向SourceCoordinator注册(地址位于taskDeploymentDescriptor中或由jobmaster通过rpc更新)以及发送split请求携带负载信息。
每个job至多有一个由JobMaster启动的SourceCoordinator。一个job可能有多个Enumerator,因为可能有多个不同的source,所有的Enumerator都运行在这个SourceCoordinator。
Split分配需要满足checkpoint模式语义。Enumerator有自己的状态(split分配),他们是全局checkpoint的一部分。当一个新的checkpoing被触发,CheckpointCoordinator首先发送barrier到SourceCoordinator。SourceCoordinator保存所有Enumerator的快照状态。SourceCoordinator通过RPC发送barrier到SourceReader。split和barrier通过RPC是先进先出的,所以Flink可以自然地将split分配与checkpoint对齐。
如果用户指定RestartAllStrategy为故障恢复策略,当Task失败时Flink重启所有的Task和SourceCoordinator。所有的Task和Enumerator都被重启并从上一次成功的checkpoint恢复。
如果用户指定RestartPipelinedRegionStrategy 为故障恢复策略,这就有点复杂。这个模型下不存在故障区域的问题,因为Enumerator和SourceReader(SourceCoordinator不是ExecutionGraph的一点)之间不存在执行边。我们需要单独来阐述。
- 当一个SourceReader task失败,JobMaster不会重启SourceCoordinator或上面的Enumerator。JobMaster会像往常一样cancel同一故障区域中的其他task。然后JobMaster通知Enumerator SourceReader任务的失败或取消(同一故障区域中可能有多个SourceReader task),以及将从哪个checkpoint版本恢复。通知在重启新任务之前。当Enumerator知道task失败时,它从特定checkpoint版本恢复与状态相关的失败task。那意味着SourceCoordinator需要支持部分恢复。Enumerator还保留着SourceReader的两级map,在内存中保存checkpoint版本和split分配。此map有助于找到应重新分配或添加回Enumerator的split。有不同的策略来处理这些失败的split。在一些基于事件时间的job中,将失败的split重新分配给其他task可能会打破watermark的语义。恢复split分配状态后,在内存中重构map并处理失败的split,Enumerator将确认返回给JobMaster,然后JobMaster重启失败区域的task。可能存在这样一种优化:Enumerator立即返回确认,而不等待恢复。因此调度失败区域的task和恢复Enumerator 可以在同一时间进行。另一个重要的事情是当Enumerator正在恢复,其它运行的SourceReader应该正常工作,包括拉取下一个split。
- 当Enumerator或SourceCoordinator失败了,如果WAL可用(下面会提到),JobMaster会重启Enumerator或SourceCoordinator但不会重启SourceReader任务。重启之后,Enumerator恢复状态,重放WAL,然后启动开始工作。同时,SourceReader等待重新连接,在成功重新注册之前不再临时分配split。重新注册是有必要的。在Enumerator和SourceReader之间重放WAL后,应进行对齐,因为Enumerator无法确保对每个SourceReader的上次split分配是否成功。如果需要(进程挂掉),JobMaster将更新重新连接的信息。如果没有可用的WAL,故障将变成全局故障,所有task和Enumerator会被重启并重最后一次成功的checkpoint恢复。
CheckpointCoordinator应通知Enumerator检查点已完成。因此Enumerator可以删除内存中保存的map和WAL。
开放的问题
在这两种情况下,enumerator都是需要重新启动整个数据流的故障点。
为了避免这种情况,我们可能需要一种额外的机制,比如用于split分配的WAL。
Option之间的比较
标准 | 运行在Task | 运行在JobManager | 运行在SourceCoordinator |
---|---|---|---|
Checkpoint对齐 | 没有必要改动(split是数据消息,和barrier自然的对齐) | split分配和checkpoint触发之间的谨慎协调。如果两个操作都在单线程ExecutionGraph线程中运行,则可能很简单。 | 没有必要改动(split通过RPC,和barrier自然对齐) |
Checkpoint状态 | 没有额外的机制(仅常规的任务状态) | 需要在JobMaster/ExecutionGraph上增加对异步无元数据状态的支持 | 需要在SourceCoordinator上添加对异步状态的支持 |
Enumerator的封装 | 封装在单独的task中 | ExecutionGraph额外的复杂性 | 新的组件SourceCoordinator |
网络栈改变 | 重大变化。有些更清晰,比如重新连接。有些可能打破了抽象,比如通知下游失败的任务 | 没必要改动 | 没必要改动 |
调度/故障区域 | 小改动 | 没必要改动 | 小改动 |
支持优雅的Enumerator恢复(避免整个重启) | 网络重连(如上),在checkpoint之间的split分配增加WAL。 | 跟踪两次checkpoint之间的split分配,在checkpoint之间的split分配增加WAL。 | 跟踪两次checkpoint之间的split分配,在checkpoint之间的split分配增加WAL。 |
Watermark | 没有必要改动(split是数据消息,watermark自然流动) | Watermark将通过ExecutionGraph和RPC | Watermark将通过RPC |
来自Stephan的个人观点:如果我们找到一种优雅的方法来抽象网络堆栈更改,我更倾向于Enumerator运行在Task,而不是JobManager
核心的公开接口
Source
// SplitEnumerator
public interface Source<T, SplitT extends SourceSplit, EnumChkT> extends Serializable {
/**
* Checks whether the source supports the given boundedness.
*
* <p>Some sources might only support either continuous unbounded streams, or
* bounded streams.
*/
boolean supportsBoundedness(Boundedness boundedness);
/**
* Creates a new reader to read data from the spits it gets assigned.
* The reader starts fresh and does not have any state to resume.
*/
SourceReader<T, SplitT> createReader(SourceContext ctx) throws IOException;
/**
* Creates a new SplitEnumerator for this source, starting a new input.
*/
SplitEnumerator<SplitT, EnumChkT> createEnumerator(Boundedness mode) throws IOException;
/**
* Restores an enumerator from a checkpoint.
*/
SplitEnumerator<SplitT, EnumChkT> restoreEnumerator(Boundedness mode, EnumChkT checkpoint) throws IOException;
// ------------------------------------------------------------------------
// serializers for the metadata
// ------------------------------------------------------------------------
/**
* Creates a serializer for the input splits. Splits are serialized when sending them
* from enumerator to reader, and when checkpointing the reader's current state.
*/
SimpleVersionedSerializer<SplitT> getSplitSerializer();
/**
* Creates the serializer for the {@link SplitEnumerator} checkpoint.
* The serializer is used for the result of the {@link SplitEnumerator#snapshotState()}
* method.
*/
SimpleVersionedSerializer<EnumChkT> getEnumeratorCheckpointSerializer();
}
/**
* The boundedness of the source: "bounded" for the currently available data (batch style),
* "continuous unbounded" for a continuous streaming style source.
*/
public enum Boundedness {
/**
* A bounded source processes the data that is currently available and will end after that.
*
* <p>When a source produces a bounded stream, the runtime may activate additional optimizations
* that are suitable only for bounded input. Incorrectly producing unbounded data when the source
* is set to produce a bounded stream will often result in programs that do not output any results
* and may eventually fail due to runtime errors (out of memory or storage).
*/
BOUNDED,
/**
* A continuous unbounded source continuously processes all data as it comes.
*
* <p>The source may run forever (until the program is terminated) or might actually end at some point,
* based on some source-specific conditions. Because that is not transparent to the runtime,
* the runtime will use an execution mode for continuous unbounded streams whenever this mode
* is chosen.
*/
CONTINUOUS_UNBOUNDED
}
Reader
(如上)
Split Enumerator
// SplitEnumerator
public interface SplitEnumerator<SplitT, CheckpointT> extends Closeable {
/**
* Returns true when the input is bounded and no more splits are available.
* False means that the definite end of input has been reached, and is only possible
* in bounded sources.
*/
boolean isEndOfInput();
/**
* Returns the next split, if it is available. If nothing is currently available, this returns
* an empty Optional.
* More may be available later, if the {@link #isEndOfInput()} is false.
*/
Optional<SplitT> nextSplit(ReaderLocation reader);
/**
* Adds splits back to the enumerator. This happens when a reader failed and restarted,
* and the splits assigned to that reader since the last checkpoint need to be made
* available again.
*/
void addSplitsBack(List<SplitT> splits);
/**
* Checkpoints the state of this split enumerator.
*/
CheckpointT snapshotState();
/**
* Called to close the enumerator, in case it holds on to any resources, like threads or
* network connections.
*/
@Override
void close() throws IOException;
}
public interface PeriodicSplitEnumerator<SplitT, CheckpointT> extends SplitEnumerator<SplitT, CheckpointT> {
/**
* Called periodically to discover further splits.
*
* @return Returns true if further splits were discovered, false if not.
*/
boolean discoverMoreSplits() throws IOException;
/**
* Continuous enumeration is only applicable to unbounded sources.
*/
default boolean isEndOfInput() {
return false;
}
}
StreamExecutionEnvironment
// StreamExecutionEnvironment
public class StreamExecutionEnvironment {
...
public <T> DataStream<T> continuousSource(Source<T, ?, ?> source) {...}
public <T> DataStream<T> continuousSource(Source<T, ?, ?> source, TypeInformation<T> type) {...}
public <T> DataStream<T> boundedSource(Source<T, ?, ?> source) {...}
public <T> DataStream<T> boundedSource(Source<T, ?, ?> source, TypeInformation<T> type) {...}
...
}
SourceOutput and Watermarking
// SourceOutput and Watermarking
public interface SourceOutput<E> extends WatermarkOutput {
void emitRecord(E record);
void emitRecord(E record, long timestamp);
}
/**
* An output for watermarks. The output accepts watermarks and idleness (inactivity) status.
*/
public interface WatermarkOutput {
/**
* Emits the given watermark.
*
* <p>Emitting a watermark also ends previously marked idleness.
*/
void emitWatermark(Watermark watermark);
/**
* Marks this output as idle, meaning that downstream operations do not
* wait for watermarks from this output.
*
* <p>An output becomes active again as soon as the next watermark is emitted.
*/
void markIdle();
}
实施计划
实施应该分为以下几个步骤进行,其中有一些可以同时进行。
- 通过实现不同模式的流行连接器来验证接口提议:
a.FileSource
(1)对于行格式(文件内的splittable,split内的的checkpoint offset)
(2)对于bulk格式如 Parquet / Orc。
(3)有界和无界的split enumerator
b.KafkaSource
(1)无界不包含动态分区发现
(2)无界包含动态分区发现
(3)有界
c.Kinesis
(1)无界 - 为高级reader模式实现测试工具
- 测试(1)中实现的reader的功能
- 实现一个新的SourceReaderTask并实现单线程的mailbox逻辑
- 实现SourceEnumeratorTask
- 实现对网络通道和调度程序或rpc服务和checkpoint的更改,以处理split分配和checkpoint并重新添加split。
兼容性、弃用和迁移计划
在DataStream API中,我们标记已经存在的source接口为deprecated但是会在后面几个版本保留。
新的source接口由不同的stream operators支持,所以这两个source接口可以很容易共存一段时间。
我们不会触及DataSet API,它最终无论如何都将被DataStream api包含
测试计划
待决定
====================================================================
以前的版本
公开的接口
我们提出了一个新的source接口以及两个伴随接口SplitEnumerator和SplitReader:
// Source
public interface Source<T, SplitT, EnumeratorCheckpointT> extends Serializable {
TypeSerializer<SplitT> getSplitSerializer();
TypeSerializer<T> getElementSerializer();
TypeSerializer<EnumeratorCheckpointT> getEnumeratorCheckpointSerializer();
EnumeratorCheckpointT createInitialEnumeratorCheckpoint();
SplitEnumerator<SplitT, EnumeratorCheckpointT> createSplitEnumerator(EnumeratorCheckpointT checkpoint);
SplitReader<T, SplitT> createSplitReader(SplitT split);
}
// SplitEnumerator
public interface SplitEnumerator<SplitT, CheckpointT> {
Iterable<SplitT> discoverNewSplits();
CheckpointT checkpoint();
}
// SplitReader
public interface SplitReader<T, SplitT> {
/**
* Initializes the reader and advances to the first record. Returns true
* if a record was read. If no record was read, records might still be
* available for reading in the future.
*/
boolean start() throws IOException;
/**
* Advances to the next record. Returns true if a record was read. If no
* record was read, records might still be available for reading in the future.
*
* <p>This method must return as fast as possible and not block if no records
* are available.
*/
boolean advance() throws IOException;
/**
* Returns the current record.
*/
T getCurrent() throws NoSuchElementException;
long getCurrentTimestamp() throws NoSuchElementException;
long getWatermark();
/**
* Returns a checkpoint that represents the current reader state. The current
* record is not the responsibility of the reader, it is assumed that the
* component that uses the reader is responsible for that.
*/
SplitT checkpoint();
/**
* Returns true if reading of this split is done, i.e. there will never be
* any available records in the future.
*/
boolean isDone() throws IOException;
/**
* Shuts down the reader.
*/
void close() throws IOException;
}
Source接口只是用来创建split enumerator和split reader的工厂。split enumerator负责检测新的partitions/shards/splits,而split reader负责从split中读取。这将分离关注点,并允许将enumeration放在并行度为1的操作中或执行图之外。同时也给了Flink更多的选择来决定如何调度split的处理。
这也潜在地允许混合和适配不同的enumeration策略与split reader。例如,当前的kafka连接器具有不同的分区发现策略,这些策略与其余代码混合在一起。有了新的接口,我们只需要一个split reader实现,并且可以有多个split enumerator用于不同的分区发现策略。
这里给出了一个实现:https://github.com/aljoscha/flink/commits/refactor-source-interface。这还附带了一个完整的Kafka source实现,它已经支持checkpointing。
提议的变更
作为mvp,我们建议添加新的接口和运行时实现,使用现有的SourceFunction 来运行enumerator,并使用特殊的operator实现来运行split reader。下一步,我们可以为enumerator和reader添加一个专用的StreamTask实现,以利用额外的优化空间。例如,更有效地处理检查点锁。
接下来的步骤是实现事件时间对齐。
兼容性、弃用和迁移计划
- 新的接口和新的source实现将与现有的source一起提供,因此不会破坏现有的编程。
- 我们可以考虑允许将现有job/savepoints平滑地迁移到新接口,但这是次要的问题。