Flink 的 Timer 机制是流处理中实现基于时间操作的核心功能,例如窗口操作、超时检测、状态清理等。它主要分为两种类型:
- Processing Time Timer (处理时间定时器):基于处理机器的系统时钟。当机器的系统时间达到定时器设定的时间时,定时器触发。
- Event Time Timer (事件时间定时器):基于数据流中事件本身携带的时间戳。当 Watermark(水位线)越过定时器设定的时间时,定时器触发。
下面主要讨论 Processing Time Timer,因为Event Time Timer只是根据水位线单线程触发。
调用流程追踪
回调链:
-
用户在
KeyedProcessFunction
中调用ctx.timerService().registerProcessingTimeTimer(...)
。 -
KeyedProcessOperator 通过 context 联系
KeyedProcessFunction
,KeyedProcessFunction
调用ctx.timerService()
实际转发 KeyedProcessOperator 注入的 SimpleTimerService -
SimpleTimerService
将调用转发给internalTimerService.registerProcessingTimeTimer(...)
。 -
InternalTimerService
向StreamTask
的ProcessingTimeService
注册一个回调。
KeyedProcessOperator 内部open
public void open() throws Exception {
super.open();
collector = new TimestampedCollector<>(output);
InternalTimerService<VoidNamespace> internalTimerService =
getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this);
TimerService timerService = new SimpleTimerService(internalTimerService);
context = new ContextImpl(userFunction, timerService);
onTimerContext = new OnTimerContextImpl(userFunction, timerService);
}
实际上调用 AbstractStreamOperator的方法
public <K, N> InternalTimerService<N> getInternalTimerService(
String name, TypeSerializer<N> namespaceSerializer, Triggerable<K, N> triggerable) {
if (timeServiceManager == null) {
throw new RuntimeException("The timer service has not been initialized.");
}
@SuppressWarnings("unchecked")
InternalTimeServiceManager<K> keyedTimeServiceHandler =
(InternalTimeServiceManager<K>) timeServiceManager;
TypeSerializer<K> keySerializer = stateHandler.getKeySerializer();
checkState(keySerializer != null, "Timers can only be used on keyed operators.");
return keyedTimeServiceHandler.getInternalTimerService(
name, keySerializer, namespaceSerializer, triggerable);
}
Triggerable 接口有两个方法:onEventTime(InternalTimer<K, N> timer) 和 onProcessingTime(InternalTimer<K, N> timer)。当 InternalTimerService 检测到有定时器到期时,就会调用实现了这个接口的对象的相应方法。
这个方法根据 InternalTimeServiceManagerImpl 获取 TimerService
public <N> InternalTimerService<N> getInternalTimerService(
String name,
TypeSerializer<K> keySerializer,
TypeSerializer<N> namespaceSerializer,
Triggerable<K, N> triggerable) {
checkNotNull(keySerializer, "Timers can only be used on keyed operators.");
// the following casting is to overcome type restrictions.
TimerSerializer<K, N> timerSerializer =
new TimerSerializer<>(keySerializer, namespaceSerializer);
InternalTimerServiceImpl<K, N> timerService =
registerOrGetTimerService(name, timerSerializer);
timerService.startTimerService(
timerSerializer.getKeySerializer(),
timerSerializer.getNamespaceSerializer(),
triggerable);
return timerService;
}
register中保证每个名字只有一个 TimerService
<N> InternalTimerServiceImpl<K, N> registerOrGetTimerService(
String name, TimerSerializer<K, N> timerSerializer) {
InternalTimerServiceImpl<K, N> timerService =
(InternalTimerServiceImpl<K, N>) timerServices.get(name);
if (timerService == null) {
if (priorityQueueSetFactory instanceof AsyncKeyedStateBackend) {
timerService =
new InternalTimerServiceAsyncImpl<>(
taskIOMetricGroup,
localKeyGroupRange,
keyContext,
processingTimeService,
createTimerPriorityQueue(
PROCESSING_TIMER_PREFIX + name, timerSerializer),
createTimerPriorityQueue(
EVENT_TIMER_PREFIX + name, timerSerializer),
cancellationContext);
} else {
timerService =
new InternalTimerServiceImpl<>(
taskIOMetricGroup,
localKeyGroupRange,
keyContext,
processingTimeService,
createTimerPriorityQueue(
PROCESSING_TIMER_PREFIX + name, timerSerializer),
createTimerPriorityQueue(
EVENT_TIMER_PREFIX + name, timerSerializer),
cancellationContext);
}
timerServices.put(name, timerService);
}
return timerService;
}
startTimerService
方法是 InternalTimerServiceImpl
的初始化入口。它负责设置必要的序列化器、触发目标(通常是算子自身),并且在从故障恢复时处理已保存的定时器。
与处理时间定时器的联系点:
// ... existing code ...
this.triggerTarget = Preconditions.checkNotNull(triggerTarget);
// re-register the restored timers (if any)
// 关键点:检查处理时间定时器队列 (processingTimeTimersQueue) 的头部是否有定时器
final InternalTimer<K, N> headTimer = processingTimeTimersQueue.peek();
if (headTimer != null) {
// 如果存在(通常意味着是从快照恢复的),
// 则调用 processingTimeService.registerTimer 来重新注册这个最早到期的处理时间定时器。
// this::onProcessingTime 是回调方法,当定时器触发时,会调用 InternalTimerServiceImpl 的 onProcessingTime 方法。
nextTimer =
processingTimeService.registerTimer(
headTimer.getTimestamp(), this::onProcessingTime);
}
this.isInitialized = true;
} else {
// ... existing code ...
- 恢复处理时间定时器:
- 在
if (restoredTimersSnapshot != null)
的逻辑块之后(或者如果restoredTimersSnapshot
为null
),代码会检查processingTimeTimersQueue
。这个队列存储了当前算子实例负责的所有处理时间定时器。 - 如果
processingTimeTimersQueue.peek()
返回一个非null
的headTimer
,这通常意味着在任务启动时,状态后端已经恢复了之前保存的定时器到这个队列中。 - 此时,
InternalTimerServiceImpl
需要告诉底层的ProcessingTimeService
(由StreamTask
提供,通常是基于 JVM 的ScheduledExecutorService
):“嘿,我这里最早有一个处理时间定时器需要在headTimer.getTimestamp()
这个时间点触发,到时请调用我的onProcessingTime
方法。” processingTimeService.registerTimer(headTimer.getTimestamp(), this::onProcessingTime)
就是在执行这个注册操作。this::onProcessingTime
是一个方法引用,指向InternalTimerServiceImpl
自己的onProcessingTime
方法。当ProcessingTimeService
确定时间到达后,会通过 Mailbox 机制回调这个方法。nextTimer
字段保存了ProcessingTimeService
返回的ScheduledFuture<?>
,允许后续取消或管理这个已注册的系统级定时器。
- 在
所以,startTimerService
在初始化阶段确保了从状态恢复的处理时间定时器能够被正确地重新调度。
registerProcessingTimeTimer
方法是用户(通过 KeyedProcessFunction
-> SimpleTimerService
)实际注册一个新的处理时间定时器时调用的核心逻辑。
与处理时间定时器的联系点:
// ... existing code ...
@Override
public void registerProcessingTimeTimer(N namespace, long time) {
// 获取当前处理时间定时器队列中最早的定时器 (如果存在)
InternalTimer<K, N> oldHead = processingTimeTimersQueue.peek();
// 将新的定时器添加到处理时间定时器队列中
// TimerHeapInternalTimer 包含了时间戳、key 和 namespace
// keyContext.getCurrentKey() 获取当前正在处理的元素的 key
if (processingTimeTimersQueue.add(
new TimerHeapInternalTimer<>(time, (K) keyContext.getCurrentKey(), namespace))) {
// 如果添加成功 (通常意味着队列状态改变了,比如新定时器成了新的头部,或者队列之前是空的)
// 获取之前队列头部的触发时间,如果队列之前为空,则认为是 Long.MAX_VALUE
long nextTriggerTime = oldHead != null ? oldHead.getTimestamp() : Long.MAX_VALUE;
// 检查新注册的定时器是否比当前已调度的系统级定时器更早
if (time < nextTriggerTime) {
// 如果新定时器更早,说明需要重新调度
if (nextTimer != null) {
// 取消之前已注册的系统级定时器 (nextTimer)
// false 表示不中断正在执行的任务 (如果回调已经在执行中)
nextTimer.cancel(false);
}
// 使用 processingTimeService 注册新的、更早的定时器
// 当这个新的时间点到达时,会回调 this::onProcessingTime
nextTimer = processingTimeService.registerTimer(time, this::onProcessingTime);
}
}
}
// ... existing code ...
- 添加定时器到内部队列:
- 首先,
new TimerHeapInternalTimer<>(time, (K) keyContext.getCurrentKey(), namespace)
创建了一个新的处理时间定时器对象。 processingTimeTimersQueue.add(...)
将这个新定时器添加到内部的优先队列中。这个队列会根据时间戳对定时器进行排序。
- 首先,
- 与
ProcessingTimeService
交互以优化调度:InternalTimerServiceImpl
只会向底层的ProcessingTimeService
注册一个系统级的定时器,即其内部队列中最早到期的那个处理时间定时器。这样做是为了避免向系统注册过多的定时器回调,提高效率。InternalTimer<K, N> oldHead = processingTimeTimersQueue.peek();
获取在添加新定时器之前队列中最早的定时器。long nextTriggerTime = oldHead != null ? oldHead.getTimestamp() : Long.MAX_VALUE;
获取之前需要触发的时间。if (time < nextTriggerTime)
: 这个判断至关重要。它检查新注册的定时器time
是否比当前已在ProcessingTimeService
中注册的下一个触发时间nextTriggerTime
更早。- 如果新定时器确实更早,那么之前向
ProcessingTimeService
注册的那个nextTimer
就作废了(因为它不再是最早的了)。 nextTimer.cancel(false);
取消旧的系统级定时器。nextTimer = processingTimeService.registerTimer(time, this::onProcessingTime);
然后向ProcessingTimeService
注册这个新的、更早的定时器。
- 如果新定时器确实更早,那么之前向
- 如果新注册的定时器并不比当前已调度的
nextTimer
更早,那么就不需要做任何操作,因为当前的nextTimer
仍然是有效的,它会在其预定时间触发,届时onProcessingTime
方法会处理所有到期的定时器(包括这个新加入但不是最早的定时器)。
processingTimeService
的具体实现类
InternalTimerServiceImpl
中的 processingTimeService
字段的类型是 org.apache.flink.streaming.runtime.tasks.ProcessingTimeService
。这是一个具体的类,而不是一个接口。
这个 org.apache.flink.streaming.runtime.tasks.ProcessingTimeService
类本身是如何实现定时调度功能的呢?它内部通常会进一步依赖另一个服务。我们来看一下它的 registerTimer
方法:
ProcessingTimeServiceImpl.java
// 注意:这是 org.apache.flink.streaming.runtime.tasks.ProcessingTimeServiceImpl
// 而不是我们正在分析的 org.apache.flink.streaming.api.operators.InternalTimerServiceImpl
package org.apache.flink.streaming.runtime.tasks;
// ... imports ...
public class ProcessingTimeServiceImpl implements ProcessingTimeService { // 实现了同名接口
private final TimerService timerService; // 关键:内部持有一个 TimerService
private ProcessingTimeCallback
addQuiesceProcessingToCallback(ProcessingTimeCallback callback) {
return timestamp -> {
if (isQuiesced()) {
return;
}
numRunningTimers.incrementAndGet();
try {
// double check to deal with the race condition:
// before executing the previous line to increase the number of running timers,
// the quiesce-completed future is already completed as the number of running
// timers is 0 and "quiesced" is true
if (!isQuiesced()) {
callback.onProcessingTime(timestamp);
}
} finally {
if (numRunningTimers.decrementAndGet() == 0 && isQuiesced()) {
quiesceCompletedFuture.complete(null);
}
}
};
}
@Override
public ScheduledFuture<?> registerTimer(long timestamp, ProcessingTimeCallback target) {
if (isQuiesced()) {
return new NeverCompleteFuture(
ProcessingTimeServiceUtil.getProcessingTimeDelay(
timestamp, getCurrentProcessingTime()));
}
return timerService.registerTimer(
timestamp,
addQuiesceProcessingToCallback(processingTimeCallbackWrapper.apply(target)));
}
// ...
}
从上面代码可以看出,ProcessingTimeService
将 registerTimer
的调用委托给了它内部持有的一个 timerService
成员。这个 timerService
的类型是TimerService
(这是一个接口)。
TimerService
接口的一个默认的实现是 SystemProcessingTimeService
。 SystemProcessingTimeService
内部使用标准的 JUC.ScheduledExecutorService
来实际执行定时任务的调度。
【实际代码封装了task,task.run() 内 回调,但是逻辑是一样的】
// 简化版的 SystemProcessingTimeService 逻辑示意
public class SystemProcessingTimeService implements TimerService {
private final ScheduledExecutorService executor; // Java的定时任务执行器
public SystemProcessingTimeService(AsyncExceptionHandler asyncExceptionHandler) {
this.executor = Executors.newSingleThreadScheduledExecutor(...);
}
@Override
public ScheduledFuture<?> registerTimer(long timestamp, ProcessingTimeCallback callback) {
long delay = Math.max(0L, timestamp - getCurrentProcessingTime());
return executor.schedule(() -> {
try {
callback.onProcessingTime(timestamp); // 当时间到达,执行回调
} catch (Throwable t) {
// handle error
}
}, delay, TimeUnit.MILLISECONDS);
}
// ...
}
所以,调用链大致如下:
InternalTimerServiceImpl
调用其成员processingTimeService
(类型为ProcessingTimeService
) 的registerTimer
方法。ProcessingTimeService
将调用委托给其内部的timerService
成员 (类型为接口flink.runtime.jobgraph.tasks.TimerService
) 的registerTimer
方法。- 这个
timerService
的一个典型实现是flink.runtime.jobgraph.tasks.SystemProcessingTimeService
。 SystemProcessingTimeService
使用java.util.concurrent.ScheduledExecutorService
来调度一个任务,该任务在指定的delay
后执行。- 当这个被调度的任务执行时,它会调用最初传递的
ProcessingTimeCallback
,也就是InternalTimerServiceImpl
实例的onProcessingTime
方法。
这样,通过层层委托,最终利用 Java 的 ScheduledExecutorService
实现了在特定处理时间点触发 Flink 操作符内部定时器逻辑的目标。
处理时间 的Timer还是交给线程池处理?
整个调用链路总结:
- 用户在 KeyedProcessFunction 中调用 ctx.timerService().registerProcessingTimeTimer(...)。
- KeyedProcessOperator 通过 context 联系 KeyedProcessFunction,在 open() 方法中创建了 ContextImpl 和 SimpleTimerService,并将后者注入前者。
- SimpleTimerService 将调用转发给 internalTimerService.registerProcessingTimeTimer(...)。
- InternalTimerServiceImpl 会调用 processingTimeService.registerTimer(time, this::onProcessingTime)。这里的 processingTimeService 是由 StreamTask 提供的 ProcessingTimeServiceImpl 实例。
- ProcessingTimeServiceImpl 会使用 SystemProcessingTimeService来调度这个回调。SystemProcessingTimeService 内部有一个 ScheduledThreadPoolExecutor(一个线程池)。当定时器到期时,这个线程池中的某个 Timer 线程会执行 注册的回调,即 InternalTimerServiceImpl.onProcessingTime()。
- 到此为止,我们离开了 StreamTask 的主 Mailbox 线程,进入了SystemProcessingTimeService 的 Timer 线程。
问题在于StreamTask.getProcessingTimeServiceFactory() 执行了一个包装:
// ... existing code ...
public ProcessingTimeServiceFactory getProcessingTimeServiceFactory() {
return mailboxExecutor -> // mailboxExecutor 是 StreamTask 的 MailboxExecutor
new ProcessingTimeServiceImpl(
timerService, // 这是 SystemProcessingTimeService
// 关键:这个 lambda 包装了原始回调
callback -> deferCallbackToMailbox(mailboxExecutor, callback));
}
private static void deferCallbackToMailbox(
MailboxExecutor mailboxExecutor, ProcessingTimeCallback callback) {
try {
// 将 callback 的执行提交到 mailboxExecutor
mailboxExecutor.execute(
() -> {
try {
// 这里的 callback.onProcessingTime() 最终会调用
// InternalTimerServiceImpl.onProcessingTime()
// 但这个调用现在是在 Mailbox 线程中执行的
callback.onProcessingTime(Long.MAX_VALUE);
} catch (Throwable t) {
// ... error handling ...
}
},
"Timer callback for %s",
callback);
} catch (RejectedExecutionException e) {
// ... error handling ...
}
}
// ... existing code ...
ProcessingTimeServiceImpl 实际上做了一层封装(processingTimeCallbackWrapper):
class ProcessingTimeServiceImpl implements ProcessingTimeService {
private final TimerService timerService;
private final Function<ProcessingTimeCallback, ProcessingTimeCallback>
processingTimeCallbackWrapper;
ProcessingTimeServiceImpl(
TimerService timerService,
Function<ProcessingTimeCallback, ProcessingTimeCallback>
processingTimeCallbackWrapper) {
this.timerService = timerService;
this.processingTimeCallbackWrapper = processingTimeCallbackWrapper;
// ...
}
@Override
public ScheduledFuture<?> registerTimer(long timestamp, ProcessingTimeCallback target) {
if (isQuiesced()) {
return new NeverCompleteFuture(
ProcessingTimeServiceUtil.getProcessingTimeDelay(
timestamp, getCurrentProcessingTime()));
}
return timerService.registerTimer(
timestamp,
addQuiesceProcessingToCallback(processingTimeCallbackWrapper.apply(target)));
}
因此最终还是单线程执行。
Timer 机制补充
核心的 Timer 服务由 InternalTimerService
接口定义,其主要实现之一是 InternalTimerServiceImpl
。
关键组件和流程:
-
InternalTimerServiceImpl
:- 持有处理时间定时器队列 (
processingTimeTimersQueue
) 和事件时间定时器队列 (eventTimeTimersQueue
)。这些通常是优先队列(例如TimerHeapInternalTimer
存储在基于堆的优先队列中),按时间戳排序。 keyContext
: 用于设置当前处理的 key,因为 Timer 通常是 KeyedStream 上下文中的。triggerTarget
: 一个Triggerable
接口实例,当 Timer 触发时,会调用其onProcessingTime
或onEventTime
方法。例如,KeyedProcessOperator
会实现Triggerable
接口。processingTimeService
: 一个外部服务,用于获取当前处理时间并注册实际的物理定时器回调。
- 持有处理时间定时器队列 (
-
注册定时器:
registerProcessingTimeTimer(namespace, time)
:- 创建一个
TimerHeapInternalTimer
对象,包含时间戳、当前 key 和 namespace。 - 将其添加到
processingTimeTimersQueue
。 - 关键:如果新注册的 Timer 是队列中最早的(或者队列之前为空),它会通过
processingTimeService.registerTimer(time, this::onProcessingTime)
注册一个回调。this::onProcessingTime
是InternalTimerServiceImpl
自身的一个方法。这意味着ProcessingTimeService
会在指定时间time
调用InternalTimerServiceImpl
的onProcessingTime
方法。
- 创建一个
registerEventTimeTimer(namespace, time)
:- 类似地,创建一个
TimerHeapInternalTimer
并添加到eventTimeTimersQueue
。 - 事件时间定时器不直接注册到
ProcessingTimeService
,它们依赖于 Watermark 的推进。
- 类似地,创建一个
-
触发定时器:
- 处理时间:
- 当
ProcessingTimeService
检测到某个注册的时间点到达时,它会调用InternalTimerServiceImpl
的onProcessingTime(long time)
方法。 onProcessingTime
方法会检查processingTimeTimersQueue
中所有时间戳小于等于当前time
的 Timer。- 对于每个到期的 Timer,它会设置
keyContext
,然后调用triggerTarget.onProcessingTime(timer)
,这通常会执行用户在ProcessFunction
或类似函数中定义的onTimer
逻辑。 - 处理完当前到期的 Timer 后,如果队列中还有未触发的 Timer,它会重新向
ProcessingTimeService
注册下一个最早的 Timer。
- 当
- 事件时间:
- 当 Flink 的 Task 接收到新的 Watermark 时,会调用
InternalTimerServiceImpl
的advanceWatermark(long time)
方法(或tryAdvanceWatermark
)。 - 这个方法会将
currentWatermark
更新为新的 Watermark 时间。 - 然后,它会检查
eventTimeTimersQueue
中所有时间戳小于等于currentWatermark
的 Timer。 - 对于每个到期的 Timer,设置
keyContext
,然后调用triggerTarget.onEventTime(timer)
。
- 当 Flink 的 Task 接收到新的 Watermark 时,会调用
- 处理时间:
-
ProcessingTimeService
:- 这是一个接口,定义了获取当前处理时间和注册处理时间回调的能力。
- 通常,在 TaskManager 层面会有一个或多个
ProcessingTimeService
的实现,它们可能基于 Java 的ScheduledExecutorService
来调度回调。
ProcessingTimeService.java
// ... existing code ... @PublicEvolving public interface ProcessingTimeService { /** Returns the current processing time. */ long getCurrentProcessingTime(); /** * Registers a task to be executed when (processing) time is {@code timestamp}. * * @param timestamp Time when the task is to be executed (in processing time) * @param target The task to be executed * @return The future that represents the scheduled task. This always returns some future, even * if the timer was shut down */ ScheduledFuture<?> registerTimer(long timestamp, ProcessingTimeCallback target); /** * A callback that can be registered via {@link #registerTimer(long, ProcessingTimeCallback)}. */ @PublicEvolving interface ProcessingTimeCallback { /** // ... existing code ...