xuyangzhong commented on code in PR #24150: URL: https://github.com/apache/flink/pull/24150#discussion_r1462661705
########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceWindowAggProcessor.java: ########## @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.operators.aggregate.window.processors; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.runtime.state.internal.InternalValueState; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.RowDataUtil; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunctionBase; +import org.apache.flink.table.runtime.operators.window.TimeWindow; +import org.apache.flink.table.runtime.operators.window.groupwindow.assigners.GroupWindowAssigner; +import org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction; +import org.apache.flink.table.runtime.operators.window.groupwindow.triggers.EventTimeTriggers; +import org.apache.flink.table.runtime.operators.window.groupwindow.triggers.ProcessingTimeTriggers; +import org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigner; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingTriggerContextBase; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowContextBase; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowTimerServiceImpl; +import org.apache.flink.table.runtime.util.TimeWindowUtil; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; + +/** + * An window aggregate processor implementation which works for {@link UnsliceAssigner}, e.g. + * session windows. + */ +public class UnsliceWindowAggProcessor extends AbstractWindowAggProcessor<TimeWindow> + implements UnslicingWindowProcessor<TimeWindow> { + + private final UnsliceAssigner<TimeWindow> unsliceAssigner; + + private final Trigger<TimeWindow> trigger; + + // ---------------------------------------------------------------------------------------- + + private transient MetricGroup metrics; + + protected transient MergingWindowProcessFunction<RowData, TimeWindow> windowFunction; + + private transient TriggerContextImpl triggerContext; + + public UnsliceWindowAggProcessor( + GeneratedNamespaceAggsHandleFunction<TimeWindow> genAggsHandler, + UnsliceAssigner<TimeWindow> unsliceAssigner, + TypeSerializer<RowData> accSerializer, + int indexOfCountStar, + ZoneId shiftTimeZone) { + super( + genAggsHandler, + unsliceAssigner, + accSerializer, + unsliceAssigner.isEventTime(), + indexOfCountStar, + shiftTimeZone); + this.unsliceAssigner = unsliceAssigner; + if (isEventTime) { + trigger = EventTimeTriggers.afterEndOfWindow(); + } else { + trigger = ProcessingTimeTriggers.afterEndOfWindow(); + } + } + + @Override + public void open(Context<TimeWindow> context) throws Exception { + super.open(context); + this.metrics = context.getRuntimeContext().getMetricGroup(); + this.windowFunction = + new MergingWindowProcessFunction<>( + unsliceAssigner.getInnerMergingWindowAssigner(), + aggregator, + unsliceAssigner + .getInnerMergingWindowAssigner() + .getWindowSerializer(new ExecutionConfig()), + 0L); + + triggerContext = new TriggerContextImpl(); + triggerContext.open(); + + WindowContextImpl windowContext = + new WindowContextImpl( + shiftTimeZone, + ctx.getTimerService(), + internalWindowState, + null, + aggregator, + unsliceAssigner.getInnerMergingWindowAssigner(), + triggerContext); + + this.windowFunction.open(windowContext); + } + + @Override + public boolean processElement(RowData key, RowData element) throws Exception { + // the windows which the input row should be placed into + Optional<TimeWindow> affectedWindowOp = + unsliceAssigner.assignStateNamespace(element, clockService, windowFunction); + boolean isElementDropped = true; + if (affectedWindowOp.isPresent()) { + TimeWindow affectedWindow = affectedWindowOp.get(); + isElementDropped = false; + + RowData acc = windowState.value(affectedWindow); + if (acc == null) { + acc = aggregator.createAccumulators(); + } + aggregator.setAccumulators(affectedWindow, acc); + + if (RowDataUtil.isAccumulateMsg(element)) { + aggregator.accumulate(element); + } else { + aggregator.retract(element); + } + acc = aggregator.getAccumulators(); + windowState.update(affectedWindow, acc); + } + + // the actual window which the input row is belongs to + Optional<TimeWindow> actualWindowOp = + unsliceAssigner.assignActualWindow(element, clockService, windowFunction); + if (actualWindowOp.isPresent()) { + TimeWindow actualWindow = actualWindowOp.get(); + isElementDropped = false; + triggerContext.setWindow(actualWindow); + // register a timer for the window to fire and clean up + // TODO support allowedLateness + long triggerTime = toEpochMillsForTimer(actualWindow.maxTimestamp(), shiftTimeZone); + if (isEventTime) { + triggerContext.registerEventTimeTimer(triggerTime); + } else { + triggerContext.registerProcessingTimeTimer(triggerTime); + } + } + return isElementDropped; + } + + @Override + public void fireWindow(long timerTimestamp, TimeWindow window) throws Exception { + windowFunction.prepareAggregateAccumulatorForEmit(window); + RowData aggResult = aggregator.getValue(window); + triggerContext.setWindow(window); + final boolean checkNeedFire; + if (isEventTime) { + checkNeedFire = triggerContext.onEventTime(timerTimestamp); + } else { + checkNeedFire = triggerContext.onProcessingTime(timerTimestamp); + } + // we shouldn't emit an empty window + if (checkNeedFire && !emptySupplier.get()) { + collect(aggResult); + } + } + + @Override + public void clearWindow(long timerTimestamp, TimeWindow window) throws Exception { + windowFunction.cleanWindowIfNeeded(window, timerTimestamp); + } + + @Override + public void advanceProgress(long progress) throws Exception {} Review Comment: Because this function is mainly used to flush buffered data into states, and currently, UnsliceWindowAggProcessor does not support Window Buffer yet. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
