JingsongLi commented on a change in pull request #15760: URL: https://github.com/apache/flink/pull/15760#discussion_r620782487
########## File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java ########## @@ -0,0 +1,558 @@ +/* + * 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.join.window; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.runtime.state.internal.InternalListState; +import org.apache.flink.streaming.api.operators.InternalTimer; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.KeyContext; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.RowDataUtil; +import org.apache.flink.table.data.utils.JoinedRowData; +import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; +import org.apache.flink.table.runtime.generated.JoinCondition; +import org.apache.flink.table.runtime.operators.TableStreamOperator; +import org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters; +import org.apache.flink.table.runtime.operators.window.state.WindowListState; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.types.RowKind; + +import java.util.IdentityHashMap; +import java.util.List; + +/** + * Streaming window join operator. + * + * <p>Note: currently, {@link WindowJoinOperator} doesn't support early-fire and late-arrival. Thus + * late elements (elements belong to emitted windows) will be simply dropped. + */ +public abstract class WindowJoinOperator extends TableStreamOperator<RowData> + implements TwoInputStreamOperator<RowData, RowData, RowData>, + Triggerable<RowData, Long>, + KeyContext { + + private static final long serialVersionUID = 1L; + + private static final String LEFT_LATE_ELEMENTS_DROPPED_METRIC_NAME = + "leftNumLateRecordsDropped"; + private static final String LEFT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME = + "leftLateRecordsDroppedRate"; + private static final String RIGHT_LATE_ELEMENTS_DROPPED_METRIC_NAME = + "rightNumLateRecordsDropped"; + private static final String RIGHT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME = + "rightLateRecordsDroppedRate"; + private static final String WATERMARK_LATENCY_METRIC_NAME = "watermarkLatency"; + private static final String LEFT_RECORDS_STATE_NAME = "left-records"; + private static final String RIGHT_RECORDS_STATE_NAME = "right-records"; + + protected final InternalTypeInfo<RowData> leftType; + protected final InternalTypeInfo<RowData> rightType; + private final GeneratedJoinCondition generatedJoinCondition; + + private final int leftWindowEndIndex; + private final int rightWindowEndIndex; + + private final boolean[] filterNullKeys; + + /** Flag to prevent duplicate function.close() calls in close() and dispose(). */ + private transient boolean functionsClosed = false; + + private transient InternalTimerService<Long> internalTimerService; + + // ------------------------------------------------------------------------ + protected transient JoinConditionWithNullFilters joinCondition; + + /** This is used for emitting elements with a given timestamp. */ + protected transient TimestampedCollector<RowData> collector; + + private transient WindowListState<Long> leftWindowState; + private transient WindowListState<Long> rightWindowState; + + // ------------------------------------------------------------------------ + // Metrics + // ------------------------------------------------------------------------ + + private transient Counter leftNumLateRecordsDropped; + private transient Meter leftLateRecordsDroppedRate; + private transient Counter rightNumLateRecordsDropped; + private transient Meter rightLateRecordsDroppedRate; + private transient Gauge<Long> watermarkLatency; + + WindowJoinOperator( + InternalTypeInfo<RowData> leftType, + InternalTypeInfo<RowData> rightType, + GeneratedJoinCondition generatedJoinCondition, + int leftWindowEndIndex, + int rightWindowEndIndex, + boolean[] filterNullKeys) { + this.leftType = leftType; + this.rightType = rightType; + this.generatedJoinCondition = generatedJoinCondition; + this.leftWindowEndIndex = leftWindowEndIndex; + this.rightWindowEndIndex = rightWindowEndIndex; + this.filterNullKeys = filterNullKeys; + } + + @Override + public void open() throws Exception { + super.open(); + functionsClosed = false; + + this.collector = new TimestampedCollector<>(output); + collector.eraseTimestamp(); + + final LongSerializer windowSerializer = LongSerializer.INSTANCE; + + internalTimerService = getInternalTimerService("window-timers", windowSerializer, this); + + // init join condition + JoinCondition condition = + generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader()); + this.joinCondition = new JoinConditionWithNullFilters(condition, filterNullKeys, this); + this.joinCondition.setRuntimeContext(getRuntimeContext()); + this.joinCondition.open(new Configuration()); + + // init state + ListStateDescriptor<RowData> leftRecordStateDesc = + new ListStateDescriptor<>(LEFT_RECORDS_STATE_NAME, leftType); + ListState<RowData> leftListState = + getOrCreateKeyedState(windowSerializer, leftRecordStateDesc); + this.leftWindowState = + new WindowListState<>((InternalListState<RowData, Long, RowData>) leftListState); + + ListStateDescriptor<RowData> rightRecordStateDesc = + new ListStateDescriptor<>(RIGHT_RECORDS_STATE_NAME, rightType); + ListState<RowData> rightListState = + getOrCreateKeyedState(windowSerializer, rightRecordStateDesc); + this.rightWindowState = + new WindowListState<>((InternalListState<RowData, Long, RowData>) rightListState); + + // metrics + this.leftNumLateRecordsDropped = metrics.counter(LEFT_LATE_ELEMENTS_DROPPED_METRIC_NAME); + this.leftLateRecordsDroppedRate = + metrics.meter( + LEFT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME, + new MeterView(leftNumLateRecordsDropped)); + this.rightNumLateRecordsDropped = metrics.counter(RIGHT_LATE_ELEMENTS_DROPPED_METRIC_NAME); + this.rightLateRecordsDroppedRate = + metrics.meter( + RIGHT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME, + new MeterView(rightNumLateRecordsDropped)); + this.watermarkLatency = + metrics.gauge( + WATERMARK_LATENCY_METRIC_NAME, + () -> { + long watermark = internalTimerService.currentWatermark(); + if (watermark < 0) { + return 0L; + } else { + return internalTimerService.currentProcessingTime() - watermark; + } + }); + } + + @Override + public void close() throws Exception { + super.close(); + collector = null; + functionsClosed = true; + if (joinCondition != null) { + joinCondition.close(); + } + } + + @Override + public void dispose() throws Exception { + super.dispose(); + collector = null; + if (!functionsClosed) { + functionsClosed = true; + if (joinCondition != null) { + joinCondition.close(); + } + } + } + + @Override + public void processElement1(StreamRecord<RowData> element) throws Exception { + processElement(element, leftWindowEndIndex, leftLateRecordsDroppedRate, leftWindowState); + } + + @Override + public void processElement2(StreamRecord<RowData> element) throws Exception { + processElement(element, rightWindowEndIndex, rightLateRecordsDroppedRate, rightWindowState); + } + + private void processElement( + StreamRecord<RowData> element, + int windowEndIndex, + Meter lateRecordsDroppedRate, + WindowListState<Long> recordState) + throws Exception { + RowData inputRow = element.getValue(); + long windowEnd = inputRow.getLong(windowEndIndex); Review comment: Can you replace this to `SliceAssigner`? It is better for extending this to `PullUpTVF`. -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org