beyond1920 commented on a change in pull request #17734:
URL: https://github.com/apache/flink/pull/17734#discussion_r750828675



##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/SemiAntiTimeIntervalJoin.java
##########
@@ -0,0 +1,450 @@
+/*
+ * 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.interval;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.StateTtlConfig;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.MapTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.flink.table.runtime.util.StateConfigUtil.createTtlConfig;
+
+/**
+ * A CoProcessFunction to execute time interval (time-bounded) stream 
semi/anti-join. Two kinds of
+ * time criteria: "L.time between R.time + X and R.time + Y" or "R.time 
between L.time - Y and
+ * L.time - X" X and Y might be negative or positive and X <= Y.
+ */
+public abstract class SemiAntiTimeIntervalJoin
+        extends KeyedCoProcessFunction<RowData, RowData, RowData, RowData> {
+
+    protected static final String LEFT_RECORDS_STATE_NAME = "left-records";
+    protected static final String RIGHT_RECORDS_STATE_NAME = "right-records";
+
+    private final boolean isAntiJoin;
+
+    protected final long leftRelativeSize;
+    protected final long rightRelativeSize;
+
+    protected final long allowedLateness;
+    private final InternalTypeInfo<RowData> leftType;
+    private final InternalTypeInfo<RowData> rightType;
+    private final IntervalJoinFunction joinFunction;
+    private final long stateRetentionTime;
+
+    // state to store rows from the left stream
+    private transient SemiAntiLeftRecordStateView leftRecordStateView;
+    // state to store rows from the right stream
+    private transient SemiAntiRightRecordStateView rightRecordStateView;
+
+    // Current time on the respective input stream.
+    protected long leftOperatorTime = 0L;
+    protected long rightOperatorTime = 0L;
+
+    public SemiAntiTimeIntervalJoin(
+            boolean isAntiJoin,
+            long leftLowerBound,
+            long leftUpperBound,
+            long allowedLateness,
+            InternalTypeInfo<RowData> leftType,
+            InternalTypeInfo<RowData> rightType,
+            IntervalJoinFunction joinFunction,
+            long stateRetentionTime) {
+        this.isAntiJoin = isAntiJoin;
+        this.leftRelativeSize = -leftLowerBound;
+        this.rightRelativeSize = leftUpperBound;
+        if (allowedLateness < 0) {
+            throw new IllegalArgumentException("The allowed lateness must be 
non-negative.");
+        }
+        this.allowedLateness = allowedLateness;
+        this.leftType = leftType;
+        this.rightType = rightType;
+        this.joinFunction = joinFunction;
+        this.stateRetentionTime = stateRetentionTime;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        super.open(parameters);
+        joinFunction.setRuntimeContext(getRuntimeContext());
+        joinFunction.open(parameters);
+
+        // TODO: support joins with unique keys
+        // now states only support no unique key
+        this.leftRecordStateView =
+                new SemiAntiLeftRecordStateView(
+                        getRuntimeContext(),
+                        LEFT_RECORDS_STATE_NAME,
+                        leftType,
+                        createTtlConfig(stateRetentionTime));
+
+        this.rightRecordStateView =
+                new SemiAntiRightRecordStateView(
+                        getRuntimeContext(),
+                        RIGHT_RECORDS_STATE_NAME,
+                        rightType,
+                        createTtlConfig(stateRetentionTime));
+    }
+
+    @Override
+    public void processElement1(RowData leftRow, Context ctx, 
Collector<RowData> out)
+            throws Exception {
+        updateOperatorTime(ctx);
+
+        long timeForLeftRow = getTimeForLeftStream(ctx, leftRow);
+        long rightQualifiedLowerBound = timeForLeftRow - rightRelativeSize;
+        long rightQualifiedUpperBound = timeForLeftRow + leftRelativeSize;
+
+        List<RowDataInfo> associatedRecords =
+                getRowDataAssociatedWithOtherInput(
+                        leftRow,
+                        true,
+                        joinFunction.getJoinCondition(),
+                        rightQualifiedLowerBound,
+                        rightQualifiedUpperBound);
+
+        if (associatedRecords.isEmpty()) {

Review comment:
       Currently, interval join requires both left input and right input are 
append stream. So I think it's ok to emit result by per input record for 
semiJoin. When one left input record comes, it could emit result it find any 
matched records in right side. When right input record comes, it would check if 
it matched records in left side and left side record has never emit before, 
then left side could emit to downstream.
   
   Maybe it would not confused user, because existed left/right/full outer join 
apply this behavior. You could tell whether a record is matched with the other 
side earlier, because you could only tell whether a record could never matched 
the other side until the last time in allowed time ranges, right?

##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/SemiAntiTimeIntervalJoin.java
##########
@@ -0,0 +1,450 @@
+/*
+ * 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.interval;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.StateTtlConfig;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.MapTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.flink.table.runtime.util.StateConfigUtil.createTtlConfig;
+
+/**
+ * A CoProcessFunction to execute time interval (time-bounded) stream 
semi/anti-join. Two kinds of
+ * time criteria: "L.time between R.time + X and R.time + Y" or "R.time 
between L.time - Y and
+ * L.time - X" X and Y might be negative or positive and X <= Y.
+ */
+public abstract class SemiAntiTimeIntervalJoin
+        extends KeyedCoProcessFunction<RowData, RowData, RowData, RowData> {
+
+    protected static final String LEFT_RECORDS_STATE_NAME = "left-records";
+    protected static final String RIGHT_RECORDS_STATE_NAME = "right-records";
+
+    private final boolean isAntiJoin;
+
+    protected final long leftRelativeSize;
+    protected final long rightRelativeSize;
+
+    protected final long allowedLateness;
+    private final InternalTypeInfo<RowData> leftType;
+    private final InternalTypeInfo<RowData> rightType;
+    private final IntervalJoinFunction joinFunction;
+    private final long stateRetentionTime;
+
+    // state to store rows from the left stream
+    private transient SemiAntiLeftRecordStateView leftRecordStateView;
+    // state to store rows from the right stream
+    private transient SemiAntiRightRecordStateView rightRecordStateView;
+
+    // Current time on the respective input stream.
+    protected long leftOperatorTime = 0L;
+    protected long rightOperatorTime = 0L;
+
+    public SemiAntiTimeIntervalJoin(
+            boolean isAntiJoin,
+            long leftLowerBound,
+            long leftUpperBound,
+            long allowedLateness,
+            InternalTypeInfo<RowData> leftType,
+            InternalTypeInfo<RowData> rightType,
+            IntervalJoinFunction joinFunction,
+            long stateRetentionTime) {
+        this.isAntiJoin = isAntiJoin;
+        this.leftRelativeSize = -leftLowerBound;
+        this.rightRelativeSize = leftUpperBound;
+        if (allowedLateness < 0) {
+            throw new IllegalArgumentException("The allowed lateness must be 
non-negative.");
+        }
+        this.allowedLateness = allowedLateness;
+        this.leftType = leftType;
+        this.rightType = rightType;
+        this.joinFunction = joinFunction;
+        this.stateRetentionTime = stateRetentionTime;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        super.open(parameters);
+        joinFunction.setRuntimeContext(getRuntimeContext());
+        joinFunction.open(parameters);
+
+        // TODO: support joins with unique keys
+        // now states only support no unique key
+        this.leftRecordStateView =
+                new SemiAntiLeftRecordStateView(
+                        getRuntimeContext(),
+                        LEFT_RECORDS_STATE_NAME,
+                        leftType,
+                        createTtlConfig(stateRetentionTime));
+
+        this.rightRecordStateView =
+                new SemiAntiRightRecordStateView(
+                        getRuntimeContext(),
+                        RIGHT_RECORDS_STATE_NAME,
+                        rightType,
+                        createTtlConfig(stateRetentionTime));
+    }
+
+    @Override
+    public void processElement1(RowData leftRow, Context ctx, 
Collector<RowData> out)
+            throws Exception {
+        updateOperatorTime(ctx);
+
+        long timeForLeftRow = getTimeForLeftStream(ctx, leftRow);
+        long rightQualifiedLowerBound = timeForLeftRow - rightRelativeSize;
+        long rightQualifiedUpperBound = timeForLeftRow + leftRelativeSize;
+
+        List<RowDataInfo> associatedRecords =
+                getRowDataAssociatedWithOtherInput(
+                        leftRow,
+                        true,
+                        joinFunction.getJoinCondition(),
+                        rightQualifiedLowerBound,
+                        rightQualifiedUpperBound);
+
+        if (associatedRecords.isEmpty()) {

Review comment:
       Currently, interval join requires both left input and right input are 
append stream. So I think it's ok to emit result by per input record for 
semiJoin. When one left input record comes, it could emit result it find any 
matched records in right side. When right input record comes, it would check if 
it matched records in left side and left side record has never emit before, 
then left side could emit to downstream.
   
   Maybe it would not confused user, because existed left/right/full outer join 
apply this behavior. You could tell whether a record is matched with the other 
side earlier, but you could only tell whether a record could never matched the 
other side until the last time in allowed time ranges, right?




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


Reply via email to