Flink SQL窗口表值函数(Window TVF)聚合实现原理浅析

引子

表值函数(table-valued function, TVF),顾名思义就是指返回值是一张表的函数,在Oracle、SQL Server等数据库中屡见不鲜。而在Flink的上一个稳定版本1.13中,社区通过FLIP-145提出了窗口表值函数(window TVF)的实现,用于替代旧版的窗口分组(grouped window)语法。

举个栗子,在1.13之前,我们需要写如下的Flink SQL语句来做10秒的滚动窗口聚合:

SELECT TUMBLE_START(procTime, INTERVAL '10' SECONDS) AS window_start,merchandiseId,COUNT(1) AS sellCount
FROM rtdw_dwd.kafka_order_done_log
GROUP BY TUMBLE(procTime, INTERVAL '10' SECONDS),merchandiseId;

在1.13版本中,则可以改写成如下的形式:

SELECT window_start,window_end,merchandiseId,COUNT(1) AS sellCount
FROM TABLE( TUMBLE(TABLE rtdw_dwd.kafka_order_done_log, DESCRIPTOR(procTime), INTERVAL '10' SECONDS) )
GROUP BY window_start,window_end,merchandiseId;

根据设计文档的描述,窗口表值函数的思想来自2019年的SIGMOD论文<<One SQL to Rule Them All>>,而表值函数属于SQL 2016标准的一部分。Calcite从1.25版本起也开始提供对滚动窗口和滑动窗口TVF的支持。除了标准化、易于实现之外,窗口TVF还支持旧版语法所不具备的一些特性,如Local-Global聚合优化、Distinct解热点优化、Top-N支持、GROUPING SETS语法等。

接下来本文简单探究一下基于窗口TVF的聚合逻辑,以及对累积窗口TVF做一点简单的改进。

SQL定义

窗口TVF函数的类图如下所示。

Flink SQL在Calcite原生的SqlWindowTableFunction的基础上加了指示窗口时间的三列,即window_startwindow_endwindow_timeSqlWindowTableFunction及其各个实现类的主要工作是校验TVF的操作数是否合法(通过内部抽象类AbstractOperandMetadata和对应的子类OperandMetadataImpl)。这一部分不再赘述,在下文改进累积窗口TVF的代码中会涉及到。

物理计划

如果看官对Calcite基础概念和Flink SQL的执行流程不了解,请务必先参考上一篇文章<<From Calcite to Tampering with Flink SQL>>

目前窗口TVF不能单独使用,需要配合窗口聚合或Top-N一起使用。以上文中的聚合为例,观察其执行计划如下。

EXPLAIN 
SELECT window_start,window_end,merchandiseId,COUNT(1) AS sellCount
FROM TABLE( TUMBLE(TABLE rtdw_dwd.kafka_order_done_log, DESCRIPTOR(procTime), INTERVAL '10' SECONDS) )
GROUP BY window_start,window_end,merchandiseId;

== Abstract Syntax Tree ==
LogicalAggregate(group=[{0, 1, 2}], sellCount=[COUNT()])
+- LogicalProject(window_start=[$48], window_end=[$49], merchandiseId=[$10])
   +- LogicalTableFunctionScan(invocation=[TUMBLE($47, DESCRIPTOR($47), 10000:INTERVAL SECOND)], rowType=[RecordType(BIGINT ts, /* ...... */, TIMESTAMP_LTZ(3) *PROCTIME* procTime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)])
      +- LogicalProject(ts=[$0], /* ...... */, procTime=[PROCTIME()])
         +- LogicalTableScan(table=[[hive, rtdw_dwd, kafka_order_done_log]])

== Optimized Physical Plan ==
Calc(select=[window_start, window_end, merchandiseId, sellCount])
+- WindowAggregate(groupBy=[merchandiseId], window=[TUMBLE(time_col=[procTime], size=[10 s])], select=[merchandiseId, COUNT(*) AS sellCount, start('w$) AS window_start, end('w$) AS window_end])
   +- Exchange(distribution=[hash[merchandiseId]])
      +- Calc(select=[merchandiseId, PROCTIME() AS procTime])
         +- TableSourceScan(table=[[hive, rtdw_dwd, kafka_order_done_log]], fields=[ts, /* ...... */])

== Optimized Execution Plan ==
Calc(select=[window_start, window_end, merchandiseId, sellCount])
+- WindowAggregate(groupBy=[merchandiseId], window=[TUMBLE(time_col=[procTime], size=[10 s])], select=[merchandiseId, COUNT(*) AS sellCount, start('w$) AS window_start, end('w$) AS window_end])
   +- Exchange(distribution=[hash[merchandiseId]])
      +- Calc(select=[merchandiseId, PROCTIME() AS procTime])
         +- TableSourceScan(table=[[hive, rtdw_dwd, kafka_order_done_log]], fields=[ts, /* ...... */])

在Flink SQL规则集中,与如上查询相关的规则按顺序依次是:

  • ConverterRule:StreamPhysicalWindowTableFunctionRule
    该规则将调用窗口TVF的逻辑节点(即调用SqlWindowTableFunctionLogicalTableFunctionScan节点)转化为物理节点(StreamPhysicalWindowTableFunction)。
  • ConverterRule:StreamPhysicalWindowAggregateRule
    该规则将含有window_startwindow_end字段的逻辑聚合节点FlinkLogicalAggregate转化为物理的窗口聚合节点StreamPhysicalWindowAggregate以及其上的投影StreamPhysicalCalc。在有其他分组字段的情况下,还会根据FlinkRelDistribution#hash生成StreamPhysicalExchange节点。
  • RelOptRule:PullUpWindowTableFunctionIntoWindowAggregateRule
    顾名思义,该规则将上面两个规则产生的RelNode进行整理,消除代表窗口TVF的物理节点,并将它的语义上拉至聚合节点中,形成最终的物理计划。

然后,StreamPhysicalWindowAggregate节点翻译成StreamExecWindowAggregate节点,进入执行阶段。

切片化窗口与执行

笔者在很久之前曾写过一篇《Flink滑动窗口原理与细粒度滑动窗口的性能问题》,其中提到粒度太碎的滑动窗口会使得状态和Timer膨胀,比较危险,应该用滚动窗口+在线存储+读时聚合的方法代替。社区在设计窗口TVF聚合时显然考虑到了这点,提出了切片化窗口(sliced window)的概念,并以此为基础设计了一套与DataStream API Windowing不同的窗口机制。

如下图的累积窗口所示,每两条纵向虚线之间的部分就是一个切片(slice)。

切片的本质就是将滑动/累积窗口化为滚动窗口,并尽可能地复用中间计算结果,降低状态压力。自然地,前文所述的Local-Global聚合优化、Distinct解热点优化就都可以无缝应用了。

那么,切片是如何分配的呢?答案是通过SliceAssigner体系,其类图如下。

注意`CumulativeSliceAssigner`多了一个`isIncremental()`方法,这是下文所做优化的一步

可见,对于滚动窗口而言,一个窗口就是一个切片;而对滑动/累积窗口而言,一个窗口可能包含多个切片,一个切片也可能位于多个窗口中。所以共享切片的窗口要特别注意切片的过期与合并。以负责累积窗口的CumulativeSliceAssigner为例,对应的逻辑如下。

@Override
public Iterable<Long> expiredSlices(long windowEnd) {
    long windowStart = getWindowStart(windowEnd);
    long firstSliceEnd = windowStart + step;
    long lastSliceEnd = windowStart + maxSize;
    if (windowEnd == firstSliceEnd) {
        // we share state in the first slice, skip cleanup for the first slice
        reuseExpiredList.clear();
    } else if (windowEnd == lastSliceEnd) {
        // when this is the last slice,
        // we need to cleanup the shared state (i.e. first slice) and the current slice
        reuseExpiredList.reset(windowEnd, firstSliceEnd);
    } else {
        // clean up current slice
        reuseExpiredList.reset(windowEnd);
    }
    return reuseExpiredList;
}

@Override
public void mergeSlices(long sliceEnd, MergeCallback callback) throws Exception {
    long windowStart = getWindowStart(sliceEnd);
    long firstSliceEnd = windowStart + step;
    if (sliceEnd == firstSliceEnd) {
        // if this is the first slice, there is nothing to merge
        reuseToBeMergedList.clear();
    } else {
        // otherwise, merge the current slice state into the first slice state
        reuseToBeMergedList.reset(sliceEnd);
    }
    callback.merge(firstSliceEnd, reuseToBeMergedList);
}

可见,累积窗口的中间结果会被合并到第一个切片中。窗口未结束时,除了第一个切片之外的其他切片触发后都会过期。

实际处理切片化窗口的算子名为SlicingWindowOperator,它实际上是SlicingWindowProcessor的简单封装。SlicingWindowProcessor的体系如下。

SlicingWindowProcessor的三个重要组成部分分别是:

  • WindowBuffer:在托管内存区域分配的窗口数据缓存,避免在窗口未实际触发时高频访问状态;
  • WindowValueState:窗口的状态,其schema为[key, window_end, accumulator]。窗口结束时间作为窗口状态的命名空间(namespace);
  • NamespaceAggsHandleFunction:通过代码生成器AggsHandlerCodeGenerator生成的聚合函数体。注意它并不是一个AggregateFunction,但是大致遵循其规范。

每当一条数据到来时,调用AbstractWindowAggProcessor#processElement()方法,比较容易理解了。

@Override
public boolean processElement(RowData key, RowData element) throws Exception {
    long sliceEnd = sliceAssigner.assignSliceEnd(element, clockService);
    if (!isEventTime) {
        // always register processing time for every element when processing time mode
        windowTimerService.registerProcessingTimeWindowTimer(sliceEnd);
    }
    if (isEventTime && isWindowFired(sliceEnd, currentProgress, shiftTimeZone)) {
        // the assigned slice has been triggered, which means current element is late,
        // but maybe not need to drop
        long lastWindowEnd = sliceAssigner.getLastWindowEnd(sliceEnd);
        if (isWindowFired(lastWindowEnd, currentProgress, shiftTimeZone)) {
            // the last window has been triggered, so the element can be dropped now
            return true;
        } else {
            windowBuffer.addElement(key, sliceStateMergeTarget(sliceEnd), element);
            // we need to register a timer for the next unfired window,
            // because this may the first time we see elements under the key
            long unfiredFirstWindow = sliceEnd;
            while (isWindowFired(unfiredFirstWindow, currentProgress, shiftTimeZone)) {
                unfiredFirstWindow += windowInterval;
            }
            windowTimerService.registerEventTimeWindowTimer(unfiredFirstWindow);
            return false;
        }
    } else {
        // the assigned slice hasn't been triggered, accumulate into the assigned slice
        windowBuffer.addElement(key, sliceEnd, element);
        return false;
    }
}

而当切片需要被合并时,先从WindowValueState中取出已有的状态,再遍历切片,并调用NamespaceAggsHandleFunction#merge()方法进行合并,最后更新状态。

@Override
public void merge(@Nullable Long mergeResult, Iterable<Long> toBeMerged) throws Exception {
    // get base accumulator
    final RowData acc;
    if (mergeResult == null) {
        // null means the merged is not on state, create a new acc
        acc = aggregator.createAccumulators();
    } else {
        RowData stateAcc = windowState.value(mergeResult);
        if (stateAcc == null) {
            acc = aggregator.createAccumulators();
        } else {
            acc = stateAcc;
        }
    }
    // set base accumulator
    aggregator.setAccumulators(mergeResult, acc);
    // merge slice accumulators
    for (Long slice : toBeMerged) {
        RowData sliceAcc = windowState.value(slice);
        if (sliceAcc != null) {
            aggregator.merge(slice, sliceAcc);
        }
    }
    // set merged acc into state if the merged acc is on state
    if (mergeResult != null) {
        windowState.update(mergeResult, aggregator.getAccumulators());
    }
}

看官若要观察codegen出来的聚合函数的代码,可在log4j.properties文件中加上:

logger.codegen.name = org.apache.flink.table.runtime.generated
logger.codegen.level = DEBUG

一点改进

我司有很多天级聚合+秒级触发的Flink作业,在DataStream API时代多由ContinuousProcessingTimeTrigger实现,1.13版本之前的SQL则需要添加table.exec.emit.early-fire系列参数。正式采用1.13版本后,累积窗口(cumulate window)完美契合此类需求。但是,有些作业的key规模比较大,在一天的晚些时候会频繁向下游Redis刷入大量数据,造成不必要的压力。因此,笔者对累积窗口TVF做了略有侵入的小改动,通过一个布尔参数INCREMENTAL可控制只输出切片之间发生变化的聚合结果。操作很简单:

  • 修改SqlCumulateTableFunction函数的签名,以及配套的窗口参数类CumulativeWindowSpec等;
  • 修改SliceSharedWindowAggProcess#fireWindow()方法,如下。
@Override
public void fireWindow(Long windowEnd) throws Exception {
    sliceSharedAssigner.mergeSlices(windowEnd, this);
    // we have set accumulator in the merge() method
    RowData aggResult = aggregator.getValue(windowEnd);
    if (!isWindowEmpty()) {
        if (sliceSharedAssigner instanceof CumulativeSliceAssigner
                && ((CumulativeSliceAssigner) sliceSharedAssigner).isIncremental()) {
            RowData stateValue = windowState.value(windowEnd);
            if (stateValue == null || !stateValue.equals(aggResult)) {
                collect(aggResult);
            }
        } else {
            collect(aggResult);
        }
    }
    // we should register next window timer here,
    // because slices are shared, maybe no elements arrived for the next slices
    // ......
}

具体可参见这个commit。当然,此方案会带来访问状态的overhead,后续会做极限压测以观察性能,并做适当修改。

The End

民那晚安晚安。

最后编辑于
©著作权归作者所有,转载或内容合作请联系作者
  • 序言:七十年代末,一起剥皮案震惊了整个滨河市,随后出现的几起案子,更是在滨河造成了极大的恐慌,老刑警刘岩,带你破解...
    沈念sama阅读 157,012评论 4 359
  • 序言:滨河连续发生了三起死亡事件,死亡现场离奇诡异,居然都是意外死亡,警方通过查阅死者的电脑和手机,发现死者居然都...
    沈念sama阅读 66,589评论 1 290
  • 文/潘晓璐 我一进店门,熙熙楼的掌柜王于贵愁眉苦脸地迎上来,“玉大人,你说我怎么就摊上这事。” “怎么了?”我有些...
    开封第一讲书人阅读 106,819评论 0 237
  • 文/不坏的土叔 我叫张陵,是天一观的道长。 经常有香客问我,道长,这世上最难降的妖魔是什么? 我笑而不...
    开封第一讲书人阅读 43,652评论 0 202
  • 正文 为了忘掉前任,我火速办了婚礼,结果婚礼上,老公的妹妹穿的比我还像新娘。我一直安慰自己,他们只是感情好,可当我...
    茶点故事阅读 51,954评论 3 285
  • 文/花漫 我一把揭开白布。 她就那样静静地躺着,像睡着了一般。 火红的嫁衣衬着肌肤如雪。 梳的纹丝不乱的头发上,一...
    开封第一讲书人阅读 40,381评论 1 210
  • 那天,我揣着相机与录音,去河边找鬼。 笑死,一个胖子当着我的面吹牛,可吹牛的内容都是我干的。 我是一名探鬼主播,决...
    沈念sama阅读 31,687评论 2 310
  • 文/苍兰香墨 我猛地睁开眼,长吁一口气:“原来是场噩梦啊……” “哼!你这毒妇竟也来了?” 一声冷哼从身侧响起,我...
    开封第一讲书人阅读 30,404评论 0 194
  • 序言:老挝万荣一对情侣失踪,失踪者是张志新(化名)和其女友刘颖,没想到半个月后,有当地人在树林里发现了一具尸体,经...
    沈念sama阅读 34,082评论 1 238
  • 正文 独居荒郊野岭守林人离奇死亡,尸身上长有42处带血的脓包…… 初始之章·张勋 以下内容为张勋视角 年9月15日...
    茶点故事阅读 30,355评论 2 241
  • 正文 我和宋清朗相恋三年,在试婚纱的时候发现自己被绿了。 大学时的朋友给我发了我未婚夫和他白月光在一起吃饭的照片。...
    茶点故事阅读 31,880评论 1 255
  • 序言:一个原本活蹦乱跳的男人离奇死亡,死状恐怖,灵堂内的尸体忽然破棺而出,到底是诈尸还是另有隐情,我是刑警宁泽,带...
    沈念sama阅读 28,249评论 2 250
  • 正文 年R本政府宣布,位于F岛的核电站,受9级特大地震影响,放射性物质发生泄漏。R本人自食恶果不足惜,却给世界环境...
    茶点故事阅读 32,864评论 3 232
  • 文/蒙蒙 一、第九天 我趴在偏房一处隐蔽的房顶上张望。 院中可真热闹,春花似锦、人声如沸。这庄子的主人今日做“春日...
    开封第一讲书人阅读 26,007评论 0 8
  • 文/苍兰香墨 我抬头看了看天上的太阳。三九已至,却和暖如春,着一层夹袄步出监牢的瞬间,已是汗流浃背。 一阵脚步声响...
    开封第一讲书人阅读 26,760评论 0 192
  • 我被黑心中介骗来泰国打工, 没想到刚下飞机就差点儿被人妖公主榨干…… 1. 我叫王不留,地道东北人。 一个月前我还...
    沈念sama阅读 35,394评论 2 269
  • 正文 我出身青楼,却偏偏与公主长得像,于是被迫代替她去往敌国和亲。 传闻我的和亲对象是个残疾皇子,可洞房花烛夜当晚...
    茶点故事阅读 35,281评论 2 259

推荐阅读更多精彩内容