JingsongLi commented on a change in pull request #8901: 
[FLINK-13003][table-planner-blink] Support Temporal TableFunction Join in blink 
planner
URL: https://github.com/apache/flink/pull/8901#discussion_r298442640
 
 

 ##########
 File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalRowTimeJoinOperator.java
 ##########
 @@ -0,0 +1,376 @@
+/*
+ * 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.join.temporal;
+
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+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.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.JoinedRow;
+import org.apache.flink.table.dataformat.util.BaseRowUtil;
+import org.apache.flink.table.generated.GeneratedJoinCondition;
+import org.apache.flink.table.generated.JoinCondition;
+import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * This operator works by keeping on the state collection of probe and build 
records to process
+ * on next watermark. The idea is that between watermarks we are collecting 
those elements
+ * and once we are sure that there will be no updates we emit the correct 
result and clean up the
+ * state.
+ *
+ * <p>Cleaning up the state drops all of the "old" values from the probe side, 
where "old" is defined
+ * as older then the current watermark. Build side is also cleaned up in the 
similar fashion,
+ * however we always keep at least one record - the latest one - even if it's 
past the last
+ * watermark.
+ *
+ * <p>One more trick is how the emitting results and cleaning up is triggered. 
It is achieved
+ * by registering timers for the keys. We could register a timer for every 
probe and build
+ * side element's event time (when watermark exceeds this timer, that's when 
we are emitting and/or
+ * cleaning up the state). However this would cause huge number of registered 
timers. For example
+ * with following evenTimes of probe records accumulated: {1, 2, 5, 8, 9}, if 
we
+ * had received Watermark(10), it would trigger 5 separate timers for the same 
key. To avoid that
+ * we always keep only one single registered timer for any given key, 
registered for the minimal
+ * value. Upon triggering it, we process all records with event times older 
then or equal to
+ * currentWatermark.
+ */
+public class TemporalRowTimeJoinOperator
+       extends AbstractStreamOperator<BaseRow>
+       implements TwoInputStreamOperator<BaseRow, BaseRow, BaseRow>, 
Triggerable<Object, VoidNamespace> {
+
+       private static final long serialVersionUID = 6642514795175288193L;
+
+       private static final String NEXT_LEFT_INDEX_STATE_NAME = "next-index";
+       private static final String LEFT_STATE_NAME = "left";
+       private static final String RIGHT_STATE_NAME = "right";
+       private static final String REGISTERED_TIMER_STATE_NAME = "timer";
+       private static final String TIMERS_STATE_NAME = "timers";
+
+       private final BaseRowTypeInfo leftType;
+       private final BaseRowTypeInfo rightType;
+       private final GeneratedJoinCondition generatedJoinCondition;
+       private final int leftTimeAttribute;
+       private final int rightTimeAttribute;
+
+       private final RowtimeComparator rightRowtimeComparator;
+
+       /**
+        * Incremental index generator for {@link #leftState}'s keys.
+        */
+       private transient ValueState<Long> nextLeftIndex;
+
+       /**
+        * Mapping from artificial row index (generated by `nextLeftIndex`) 
into the left side `Row`.
+        * We can not use List to accumulate Rows, because we need efficient 
deletes of the oldest rows.
+        *
+        * <p>TODO: this could be OrderedMultiMap[Jlong, Row] indexed by row's 
timestamp, to avoid
+        * full map traversals (if we have lots of rows on the state that 
exceed `currentWatermark`).
+        */
+       private transient MapState<Long, BaseRow> leftState;
+
+       /**
+        * Mapping from timestamp to right side `Row`.
+        *
+        * <p>TODO: having `rightState` as an OrderedMapState would allow us to 
avoid sorting cost
+        * once per watermark
+        */
+       private transient MapState<Long, BaseRow> rightState;
+
+       // Long for correct handling of default null
+       private transient ValueState<Long> registeredTimer;
+       private transient TimestampedCollector<BaseRow> collector;
+       private transient InternalTimerService<VoidNamespace> timerService;
+
+       private transient JoinCondition joinCondition;
+       private transient JoinedRow outRow;
+
+       public TemporalRowTimeJoinOperator(
+                       BaseRowTypeInfo leftType,
+                       BaseRowTypeInfo rightType,
+                       GeneratedJoinCondition generatedJoinCondition,
+                       int leftTimeAttribute,
+                       int rightTimeAttribute) {
+               this.leftType = leftType;
+               this.rightType = rightType;
+               this.generatedJoinCondition = generatedJoinCondition;
+               this.leftTimeAttribute = leftTimeAttribute;
+               this.rightTimeAttribute = rightTimeAttribute;
+               this.rightRowtimeComparator = new 
RowtimeComparator(rightTimeAttribute);
+       }
+
+       @Override
+       public void open() throws Exception {
+               joinCondition = 
generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader());
+               FunctionUtils.setFunctionRuntimeContext(joinCondition, 
getRuntimeContext());
 
 Review comment:
   `joinCondition` is a `RichFunction`, so you don't need invoke 
`FunctionUtils`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

Reply via email to