[Flink 1.17 源码] Flink多入度算子Watermark的处理

403 阅读6分钟

Flink多入度算子Watermark的处理

实验场景和现象

想观察一下watermark的生成情况,使用Punctuated策略逐个event生成watermark(3s延迟),以及一个WatermarkPrintSink 打印watermark信息

写了一个简单的Job,流程图如下(disableChanning):

graph LR
  A("Source:SocketStream(1)") -->|REBALANCE| B("Map(16)") --> |FORWARD| C("Timestamps/Watermarks(16)") --> |FORWARD| D("Sink:WatermarkPrintSink(16)")
输入:
hello,1,10000
输出:
WatermarkGenerator> 当前线程ID:105,事件时间:[10000|1970-01-01 08:00:10.000],生成水位线:[7999|1970-01-01 08:00:07.999],事件:KeywordEvent(word=hello, cnt=1, ts=10000)
WatermarkPrintSink subtask 00> 当前线程ID:107,事件时间:[10000|1970-01-01 08:00:10.000],水位线:[-9223372036854775808|No Watermark],事件:KeywordEvent(word=hello, cnt=1, ts=10000)

可以看到WatermarkGenerator生成了水位线但是WatermarkPrintSink显示当前subtask并没有收到新的水位线(7999),这个是因为水位线终究是在事件之后emit

我再输入了一次,WatermarkPrintSink还是显示没有,有点费解,仔细一看发现并不是同一个subtask,每个subtask会有自己的watermarkWebUISink算子的Watermark页面更为直观 在这里插入图片描述

输入:
hello,1,12000
输出:
WatermarkGenerator> 当前线程ID:106,事件时间:[12000|1970-01-01 08:00:12.000],生成水位线:[9999|1970-01-01 08:00:09.999],事件:KeywordEvent(word=hello, cnt=1, ts=12000)
WatermarkPrintSink subtask 01> 当前线程ID:107,事件时间:[12000|1970-01-01 08:00:12.000],水位线:[-9223372036854775808|No Watermark],事件:KeywordEvent(word=hello, cnt=1, ts=12000)

Socket是单并行度Source,采用了rebalance策略轮询(round-robin)发往下游,也就是我发完第17次就能观察到watermark更新,也确实如此。

Sink改为单并行度,将全局并行度设置为4,方便调试,发现仍然需要等待Source将下游算子所有并行度轮询完后才能观察到watermark,这个在阅读官网的过程中也了解过,多入度算子会将所有信道的watermark进行对比选择最小的那个作为算子水位线 Timely Stream Processing | Apache Flink 在这里插入图片描述

两点疑问

  1. 在某一个入度水位线严重滞后甚至直接停滞的情况下,watermark按最小输出不会影响到计算吗,比如窗口?
  2. 多个入度的watermark不会同时到达,什么时候开始/触发计算最小watermark然后emit

问题1:

Flink Event Time 倾斜

常见的数据倾斜往往是因为业务流量分布不均产生,比如大V的流量和普通用户肯定不在一个等级,这种倾斜往往可以预料并处理。

event-time也会出现数据倾斜,这会导致数据的缓存,状态的膨胀,特别是ProcessFunction在窗口计算时会缓存所有数据的情况,checkpoint压力也会增加。

倾斜往往发生在数据摄入阶段,代码难以处理,需要框架层面的支持。

Flink社区有两种方案:

方案1:下游算子反压,抑制watermark过快的channel的数据摄入。

方案2:Source处进行watermark对齐,类似checkpoint

最终选择了方案2,并对Source API进行了重构:Data Sources | Apache Flink ,方案1耦合度太高,barrier也会被延缓,barrier校准同样会产生数据缓存,同时也不能确保解决问题:反压不到Source端,watermark仍可能有很大差距

极端情况下:Source某个并行度并没有分配分区,这种情况下flink会怎么处理?见源码分析 先说结论:有停滞流的机制,用WatermarkStatus标记信道状态IDLE,在比较所有信道的时候将其认定为非aligned ,不参与最小值比较,因此不会造成watermark无法推进的情况

问题2:见源码分析 先说结论:每次有watermark输入的时候会尝试进行emit,会判断所有信道中最小watermark和上次emitwatermark的大小,如果大于,则会emit新的watermark。举个例子:四个入度的信道,一开始都是Long.MIN_VALUE,前三个信道更新了watermark,但是第四个信道一直没更新(还是Long.MIN_VALUE),这样是不会emit新的watermark的,无论前三个信道watermark更新到多少。

源码分析

  • 核心类与概念
    • WatermarkStatus: 告知task是否需要继续接收(expect)信道的watermark
      • WatermarkStatussource生成,会emit两种不同状态:
        • IDLE: source端暂停watermarkemit(比如 kafka source 没有分配 partition
        • ACTIVE: source从idle状态切换回正常emit watermark的状态
      • WatermarkStatus是一种临时的状态,如果是永久性停止生成watermark, 应考虑向下游发送Watermark.MAX_WATERMARK
      • 下游算子如果是idle状态 → 所有入度均为idle
    • StatusWatermarkValveWatermark以及WatermarkStatus如何向下游传递的逻辑
      • 类核心成员:
        • InputChannelStatus[] channelStatuses : 一组输入信道的状态
          • InputChannelStatus包含:
            • watermark:当前信道最新的水位线
            • watermarkStatus:当前信道最新的状态
            • isWatermarkAligned:当前信道的Watermark是否对齐
            • InputChannelStatus 实现了 HeapPriorityQueueElement,会被HeapPriorityQueue管理,在最小堆中根据Watermark进行排序
        • lastOutputWatermarklastOutputWatermarkStatus: 上次emitWatermark以及WatermarkStatus的值
        • HeapPriorityQueue<InputChannelStatus> alignedChannelStatuses 优先队列(堆实现):找出对齐channel中最小的Watermark
      • 初始化:
        • 根据入度进行初始化
        • 每个信道的状态信息:InputChannelStatus
          • watermarkLong.*MIN_VALUE*
          • watermarkStatusACTIVE
          • 该信道watermark是对齐状态
    • HeapPriorityQueue :优先队列的堆实现,用于处理管理信道的watermark状态InputChannelStatus的数据结构
      • org.apache.flink.runtime.state.heap.HeapPriorityQueue,而是org.apache.flink.streaming.runtime.watermarkstatus.HeapPriorityQueue,没有复用heap包下的,是因为避免继承导致JVM不能inline methods,从而降低memory state backend的性能
      • 核心方法:
        • siftUp(int index)siftDown(int index): 堆顺序维护操作:上浮、下沉
        • PriorityComparator :独立的优先级比较器,通过构造器传入,priority相同不代表两个Object equals
        • peek():获取堆顶元素(堆最小值)(不移除)
        • adjustModifiedElement(T element): 更改InputChannelStatuswatermark值后调用该方法恢复堆顺序,本质调用的adjustElementAtIndex(T element, int index):siftDown → 判断是否下沉了(queue(index) != element) → 如果没有下沉 → siftUp
  • 核心流程
    • AbstractStreamTaskNetworkInput#processElement

      org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput#processElement

      private void processElement(StreamElement recordOrMark, DataOutput<T> output) throws Exception {
          if (recordOrMark.isRecord()) {
      				//Record
              output.emitRecord(recordOrMark.asRecord());
          } else if (recordOrMark.isWatermark()) {
      				//Watermark
              statusWatermarkValve.inputWatermark(
                      recordOrMark.asWatermark(), flattenedChannelIndices.get(lastChannel), output);
          } else if (recordOrMark.isLatencyMarker()) {
              //LatencyMarker
              output.emitLatencyMarker(recordOrMark.asLatencyMarker());
          } else if (recordOrMark.isWatermarkStatus()) {
      				//WatermarkStatus
              statusWatermarkValve.inputWatermarkStatus(
                      recordOrMark.asWatermarkStatus(),
                      flattenedChannelIndices.get(lastChannel),
                      output);
          } else {
              throw new UnsupportedOperationException("Unknown type of StreamElement");
          }
      }
      
    • StatusWatermarkValve#inputWatermark

      org.apache.flink.streaming.runtime.watermarkstatus.StatusWatermarkValve#inputWatermark

      public void inputWatermark(Watermark watermark, int channelIndex, DataOutput<?> output)
              throws Exception {
          //WatermarkStatus为Active状态(上一次output的以及当前channel的)
          if (lastOutputWatermarkStatus.isActive()
                  && channelStatuses[channelIndex].watermarkStatus.isActive()) {
      
              long watermarkMillis = watermark.getTimestamp();
              //如果输入watermark <= 当前channel上一个收到watermark的时间戳 会被忽略
              if (watermarkMillis > channelStatuses[channelIndex].watermark) {
                  //更新当前channel的watermark
      						channelStatuses[channelIndex].watermark = watermarkMillis;
      						
      						//当前channel的watermark如果是对齐的
                  if (channelStatuses[channelIndex].isWatermarkAligned) {
      								//调整当前channel在堆(最小堆)中的位置
                      adjustAlignedChannelStatuses(channelStatuses[channelIndex]);
                  } else if (watermarkMillis >= lastOutputWatermark) {
                      //如果未对齐状态且watermark >= 上次output的watermark -> 标记当前channel为对齐状态
                      markWatermarkAligned(channelStatuses[channelIndex]);
                  }
      
                  //尝试output watermark (堆中最小watermark > 上次output的watermark)
                  findAndOutputNewMinWatermarkAcrossAlignedChannels(output);
              }
          }
      }
      
    • findAndOutputNewMinWatermarkAcrossAlignedChannels

      org.apache.flink.streaming.runtime.watermarkstatus.StatusWatermarkValve#findAndOutputNewMinWatermarkAcrossAlignedChannels

      private void findAndOutputNewMinWatermarkAcrossAlignedChannels(DataOutput<?> output)
              throws Exception {
          boolean hasAlignedChannels = !alignedChannelStatuses.isEmpty();
      
          // 存在对齐的channel 且 对齐channel的最小watermark > 上次output的watermark => emit watermark
          if (hasAlignedChannels && alignedChannelStatuses.peek().watermark > lastOutputWatermark) {
              lastOutputWatermark = alignedChannelStatuses.peek().watermark;
              output.emitWatermark(new Watermark(lastOutputWatermark));
          }
      }