Flink sql Group Windows 窗口源码解析

​窗口(Window)是 Flink 众多优势之一。

窗口可以将数据流装入大小有限的“桶”中,再对每个“桶”加以处理

随着实时数仓和 flink sql 越来越成熟,flink sql 又变成 flink 中最常用的 api

在这种场景下,我们来看看 flink sql 窗口的源码。

(练习一下表达能力,扯一些有的没的)

窗口样例

先来看 flink sql Group Windows 的写法:

select date_format(now(), 'yyyy-MM-dd HH:mm:ss')
     ,date_format(TUMBLE_START(proc_time, INTERVAL '1' minute), 'yyyy-MM-dd HH:mm:ss') AS wStart
     ,date_format(TUMBLE_END(proc_time, INTERVAL '1' minute), 'yyyy-MM-dd HH:mm:ss') AS wEnd
     ,count(user_id) pv
     ,count(distinct user_id) uv
     ,max(user_id)
from user_log
group by TUMBLE(proc_time, INTERVAL '1' minute)
;

Group Windows 源码

先看下执行流图:

先来看个简单的 Group Windows 的源码,Group Windows 就是和 Stream api 类似的 window,通过解析 sql ,调用 StreamExecGroupWindowAggregate.translateToPlanInternal 创建 window operator

StreamExecGroupWindowAggregate.translateToPlanInternal


  @SuppressWarnings("unchecked")
    @Override
    protected Transformation<RowData> translateToPlanInternal(
            PlannerBase planner, ExecNodeConfig config) {
        final boolean isCountWindow;
        
        ......

        // agg 函数 见下图
        final AggregateInfoList aggInfoList =
                transformToStreamAggregateInfoList(
                        inputRowType,
                        JavaScalaConversionUtil.toScala(Arrays.asList(aggCalls)),
                        aggCallNeedRetractions,
                        needRetraction,
                        true, // isStateBackendDataViews
                        true); // needDistinctInfo
        // 生成的 agg 函数代码
        final GeneratedClass<?> aggCodeGenerator =
                createAggsHandler(
                        aggInfoList,
                        config,
                        planner.getRelBuilder(),
                        inputRowType.getChildren(),
                        shiftTimeZone);

        .......

        // 创建 窗口算子
        final WindowOperator<?, ?> operator =
                createWindowOperator(
                        config,
                        aggCodeGenerator,
                        equaliser,
                        accTypes,
                        windowPropertyTypes,
                        aggValueTypes,
                        inputRowType.getChildren().toArray(new LogicalType[0]),
                        inputTimeFieldIndex,
                        shiftTimeZone,
                        inputCountIndex);
        .........
    }

  • 注: 代码比较长,截取部分

aggInfoList : 

aggCodeGenerator: 

StreamExecGroupWindowAggregate.createWindowOperator

    private WindowOperator<?, ?> createWindowOperator(
            ReadableConfig config,
            GeneratedClass<?> aggsHandler,
            GeneratedRecordEqualiser recordEqualiser,
            LogicalType[] accTypes,
            LogicalType[] windowPropertyTypes,
            LogicalType[] aggValueTypes,
            LogicalType[] inputFields,
            int timeFieldIndex,
            ZoneId shiftTimeZone,
            int inputCountIndex) {
        WindowOperatorBuilder builder =
                WindowOperatorBuilder.builder()
                        .withInputFields(inputFields)
                        .withShiftTimezone(shiftTimeZone)
                        .withInputCountIndex(inputCountIndex);
        // 根据窗口类型窗口对应的 窗口 builder 
        if (window instanceof TumblingGroupWindow) {
        	// 滚动窗口
            TumblingGroupWindow tumblingWindow = (TumblingGroupWindow) window;
            FieldReferenceExpression timeField = tumblingWindow.timeField();
            ValueLiteralExpression size = tumblingWindow.size();
            if (isProctimeAttribute(timeField) && hasTimeIntervalType(size)) {
            	// 处理时间窗口
                builder = builder.tumble(toDuration(size)).withProcessingTime();
            } else if (isRowtimeAttribute(timeField) && hasTimeIntervalType(size)) {
            	// 事件时间窗口
                builder = builder.tumble(toDuration(size)).withEventTime(timeFieldIndex);
            } else if (isProctimeAttribute(timeField) && hasRowIntervalType(size)) {
            	// count 窗口
                builder = builder.countWindow(toLong(size));
            } else {
                // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
                // before applying the  windowing logic. Otherwise, this would be the same as a
                // ProcessingTimeTumblingGroupWindow
                throw new UnsupportedOperationException(
                        "Event-time grouping windows on row intervals are currently not supported.");
            }
        } else if (window instanceof SlidingGroupWindow) {
        	// 滑动窗口
            
            .....
            }
        } else if (window instanceof SessionGroupWindow) {
        	// session 窗口
            .....
        } else {
            throw new TableException("Unsupported window: " + window.toString());
        }

        // 窗口触发策略,比如示例 sql ,事件时间,窗口结束才触发
        WindowEmitStrategy emitStrategy = WindowEmitStrategy.apply(config, window);
        if (emitStrategy.produceUpdates()) {
            // mark this operator will send retraction and set new trigger
            builder.produceUpdates()
                    .triggering(emitStrategy.getTrigger())
                    .withAllowedLateness(Duration.ofMillis(emitStrategy.getAllowLateness()));
        }

        if (aggsHandler instanceof GeneratedNamespaceAggsHandleFunction) {
        	// agg 
            return builder.aggregate(
                            (GeneratedNamespaceAggsHandleFunction<?>) aggsHandler,
                            recordEqualiser,
                            accTypes,
                            aggValueTypes,
                            windowPropertyTypes)
            		// 创建 AggregateWindowOperator
                    .build();
        } else if (aggsHandler instanceof GeneratedNamespaceTableAggsHandleFunction) {
        	// table agg
            return builder.aggregate(
                            (GeneratedNamespaceTableAggsHandleFunction<?>) aggsHandler,
                            accTypes,
                            aggValueTypes,
                            windowPropertyTypes)
                    .build();
        } else {
            throw new TableException(
                    "Unsupported agg handler class: " + aggsHandler.getClass().getSimpleName());
        }
    }

窗口触发策略: 

-----------补充----------
如果配置了提前触发参数:

set table.exec.emit.early-fire.enabled = true;
set table.exec.emit.early-fire.delay = 5000;

WindowOperatorBuilder.build

        public AggregateWindowOperator build() {
            checkNotNull(windowOperatorBuilder.trigger, "trigger is not set");
            if (generatedAggregateFunction != null && generatedEqualiser != null) {
            	// agg function 不为空
                //noinspection unchecked
                return new AggregateWindowOperator(
                        generatedAggregateFunction,
                        generatedEqualiser,
                        windowOperatorBuilder.windowAssigner,
                        windowOperatorBuilder.trigger,
                        windowOperatorBuilder.windowAssigner.getWindowSerializer(
                                new ExecutionConfig()),
                        windowOperatorBuilder.inputFieldTypes,
                        windowOperatorBuilder.accumulatorTypes,
                        windowOperatorBuilder.aggResultTypes,
                        windowOperatorBuilder.windowPropertyTypes,
                        windowOperatorBuilder.rowtimeIndex,
                        windowOperatorBuilder.produceUpdates,
                        windowOperatorBuilder.allowedLateness,
                        windowOperatorBuilder.shiftTimeZone,
                        windowOperatorBuilder.inputCountIndex);
            } else {
               
               ..... 
            }
        }

AggregateWindowOperator.AggregateWindowOperator 直接调用 父类 WindowOperator 的构造放

WindowOperator(
            WindowAssigner<W> windowAssigner,
            Trigger<W> trigger,
            TypeSerializer<W> windowSerializer,
            LogicalType[] inputFieldTypes,
            LogicalType[] accumulatorTypes,
            LogicalType[] aggResultTypes,
            LogicalType[] windowPropertyTypes,
            int rowtimeIndex,
            boolean produceUpdates,
            long allowedLateness,
            ZoneId shiftTimeZone,
            int inputCountIndex) {
        checkArgument(allowedLateness >= 0);
        this.windowAssigner = checkNotNull(windowAssigner);
        this.trigger = checkNotNull(trigger);
        this.windowSerializer = checkNotNull(windowSerializer);
        this.inputFieldTypes = checkNotNull(inputFieldTypes);
        this.accumulatorTypes = checkNotNull(accumulatorTypes);
        this.aggResultTypes = checkNotNull(aggResultTypes);
        this.windowPropertyTypes = checkNotNull(windowPropertyTypes);
        this.allowedLateness = allowedLateness;
        this.produceUpdates = produceUpdates;

        // rowtime index should >= 0 when in event time mode
        checkArgument(!windowAssigner.isEventTime() || rowtimeIndex >= 0);
        this.rowtimeIndex = rowtimeIndex;
        this.shiftTimeZone = shiftTimeZone;
        this.recordCounter = RecordCounter.of(inputCountIndex);

        setChainingStrategy(ChainingStrategy.ALWAYS);
    }

sql api 的 WindowOperator 和 stream api 的 WindowOperator 基本一样

核心方法:

// 数据进来的时候调用
public void processElement(StreamRecord<RowData> record) 

// 事件时间调用
public void onEventTime(InternalTimer<K, W> timer)

// 处理时间调用
public void onProcessingTime(InternalTimer<K, W> timer)

和抽象方法 emitWindowResult,在 AggregateWindowOperator 中实现

// 输出窗口结果
protected abstract void emitWindowResult(W window) throws Exception;

AggregateWindowOperator.emitWindowResult:

 @Override
protected void emitWindowResult(W window) throws Exception {
    windowFunction.prepareAggregateAccumulatorForEmit(window);
    RowData acc = aggWindowAggregator.getAccumulators();
    RowData aggResult = aggWindowAggregator.getValue(window);
    // 判断是否发出 更新(-U) 消息,配置提前触发参数的窗口会发出更新消息
    if (produceUpdates) {
    	// 获取上一个窗口数据
        previousState.setCurrentNamespace(window);
        RowData previousAggResult = previousState.value();

        // 判断 acc 是否为空,
        if (!recordCounter.recordCountIsZero(acc)) {
            // has emitted result for the window
            // 判断上一窗口是否为 空
            if (previousAggResult != null) {
            	// 上一窗口不为空
                // current agg is not equal to the previous emitted, should emit retract

            	// 当前窗口结果是否和上一窗口结果相同
                if (!equaliser.equals(aggResult, previousAggResult)) {
                	// 不相同,发出,撤回上一窗口结果和新的结果
                    // send UPDATE_BEFORE
                    collect(
                            RowKind.UPDATE_BEFORE,
                            (RowData) getCurrentKey(),
                            previousAggResult);
                    // send UPDATE_AFTER
                    collect(RowKind.UPDATE_AFTER, (RowData) getCurrentKey(), aggResult);
                    // update previousState
                    previousState.update(aggResult);
                }
                // if the previous agg equals to the current agg, no need to send retract and
                // accumulate
            }
            // the first fire for the window, only send INSERT
            else {
            	// 上一窗口为空,发出 本窗口结果
                // send INSERT
                collect(RowKind.INSERT, (RowData) getCurrentKey(), aggResult);
                // update previousState
                previousState.update(aggResult);
            }
        } else {
        	// acc 为空,没有当前数据需要触发

        	// 判断之前窗口数据是否为空,为空撤回上一个窗口数据
            // has emitted result for the window
            // we retracted the last record for this key
            if (previousAggResult != null) {
                // send DELETE
                collect(RowKind.DELETE, (RowData) getCurrentKey(), previousAggResult);
                // clear previousState
                previousState.clear();
            }
            // if the counter is zero, no need to send accumulate
        }
    } else {
    	// 不需要的直接 发出 +I 数据
        if (!recordCounter.recordCountIsZero(acc)) {
            // send INSERT
            collect(RowKind.INSERT, (RowData) getCurrentKey(), aggResult);
        }
        // if the counter is zero, no need to send accumulate
        // there is no possible skip `if` branch when `produceUpdates` is false
    }
}

逻辑有点复杂,画了个图:

  • 0
    点赞
  • 0
    收藏
    觉得还不错? 一键收藏
  • 0
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值