引子
表值函數(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_start
、window_end
和window_time
。SqlWindowTableFunction
及其各個實現類的主要工作是校驗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的邏輯節點(即調用SqlWindowTableFunction
的LogicalTableFunctionScan
節點)轉化爲物理節點(StreamPhysicalWindowTableFunction
)。 -
ConverterRule:
StreamPhysicalWindowAggregateRule
該規則將含有window_start
、window_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
爲例,對應的邏輯如下。
@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
民那晚安晚安。