Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
/*
* 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;

import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.binary.NullAwareGetters;
import org.apache.flink.table.runtime.generated.JoinCondition;

/** Utility to take null filters into consideration when apply join condition. */
public class JoinConditionWithNullFilters extends AbstractRichFunction implements JoinCondition {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This class can wrap setRuntimeContext too.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does it need a AbstractRichFunction? I cannot see the useage.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

JoinConditionWithNullFilters extends AbstractRichFunction because we need call open function of joinCondition to handle the case that joinCondition contains some special udf overrides open function.


private final JoinCondition backingJoinCondition;

/** Should filter null keys. */
private final int[] nullFilterKeys;

/** No keys need to filter null. */
private final boolean nullSafe;

/** Filter null to all keys. */
private final boolean filterAllNulls;

private NullAwareGetters joinKey;

public JoinConditionWithNullFilters(
JoinCondition backingJoinCondition, boolean[] filterNullKeys) {
this.backingJoinCondition = backingJoinCondition;
this.nullFilterKeys = NullAwareJoinHelper.getNullFilterKeys(filterNullKeys);
this.nullSafe = nullFilterKeys.length == 0;
this.filterAllNulls = nullFilterKeys.length == filterNullKeys.length;
}

@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
backingJoinCondition.open(parameters);
}

@Override
public void close() throws Exception {
super.close();
backingJoinCondition.close();
}

@Override
public boolean apply(RowData left, RowData right) {
if (!nullSafe) { // is not null safe, return false if any null exists
// key is always BinaryRowData
if (filterAllNulls ? joinKey.anyNull() : joinKey.anyNull(nullFilterKeys)) {
// find null present, return false directly
return false;
}
}
// test condition
return backingJoinCondition.apply(left, right);
}

public void setCurrentJoinKey(NullAwareGetters joinKey) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have some concern that we will forget invoking of this method.
I think we can just pass a KeyContext to this class in the constructor. What do you think?

this.joinKey = joinKey;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,15 @@

package org.apache.flink.table.runtime.operators.join.stream;

import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.binary.NullAwareGetters;
import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
import org.apache.flink.table.runtime.generated.JoinCondition;
import org.apache.flink.table.runtime.operators.join.NullAwareJoinHelper;
import org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters;
import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
import org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateView;
import org.apache.flink.table.runtime.operators.join.stream.state.OuterJoinRecordStateView;
Expand Down Expand Up @@ -60,14 +58,7 @@ public abstract class AbstractStreamingJoinOperator extends AbstractStreamOperat
protected final JoinInputSideSpec leftInputSideSpec;
protected final JoinInputSideSpec rightInputSideSpec;

/** Should filter null keys. */
private final int[] nullFilterKeys;

/** No keys need to filter null. */
private final boolean nullSafe;

/** Filter null to all keys. */
private final boolean filterAllNulls;
private final boolean[] filterNullKeys;

protected final long stateRetentionTime;

Expand All @@ -88,9 +79,7 @@ public AbstractStreamingJoinOperator(
this.leftInputSideSpec = leftInputSideSpec;
this.rightInputSideSpec = rightInputSideSpec;
this.stateRetentionTime = stateRetentionTime;
this.nullFilterKeys = NullAwareJoinHelper.getNullFilterKeys(filterNullKeys);
this.nullSafe = nullFilterKeys.length == 0;
this.filterAllNulls = nullFilterKeys.length == filterNullKeys.length;
this.filterNullKeys = filterNullKeys;
}

@Override
Expand All @@ -100,9 +89,8 @@ public void open() throws Exception {
JoinCondition condition =
generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader());
condition.setRuntimeContext(getRuntimeContext());
condition.open(new Configuration());

this.joinCondition = new JoinConditionWithNullFilters(condition);
this.joinCondition = new JoinConditionWithNullFilters(condition, filterNullKeys);
this.joinCondition.open(new Configuration());

this.collector = new TimestampedCollector<>(output);
}
Expand All @@ -111,35 +99,7 @@ public void open() throws Exception {
public void close() throws Exception {
super.close();
if (joinCondition != null) {
joinCondition.backingJoinCondition.close();
}
}

// ----------------------------------------------------------------------------------------
// Utility Classes
// ----------------------------------------------------------------------------------------

private class JoinConditionWithNullFilters extends AbstractRichFunction
implements JoinCondition {

final JoinCondition backingJoinCondition;

private JoinConditionWithNullFilters(JoinCondition backingJoinCondition) {
this.backingJoinCondition = backingJoinCondition;
}

@Override
public boolean apply(RowData left, RowData right) {
if (!nullSafe) { // is not null safe, return false if any null exists
// key is always BinaryRowData
NullAwareGetters joinKey = (NullAwareGetters) getCurrentKey();
if (filterAllNulls ? joinKey.anyNull() : joinKey.anyNull(nullFilterKeys)) {
// find null present, return false directly
return false;
}
}
// test condition
return backingJoinCondition.apply(left, right);
joinCondition.close();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
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.binary.NullAwareGetters;
import org.apache.flink.table.data.util.RowDataUtil;
import org.apache.flink.table.data.utils.JoinedRowData;
import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
Expand Down Expand Up @@ -207,6 +208,7 @@ private void processElement(
RowKind inputRowKind = input.getRowKind();
input.setRowKind(RowKind.INSERT); // erase RowKind for later state updating

joinCondition.setCurrentJoinKey((NullAwareGetters) getCurrentKey());
AssociatedRecords associatedRecords =
AssociatedRecords.of(input, inputIsLeft, otherSideStateView, joinCondition);
if (isAccumulateMsg) { // record is accumulate
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.binary.NullAwareGetters;
import org.apache.flink.table.data.util.RowDataUtil;
import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
Expand Down Expand Up @@ -102,6 +103,7 @@ public void open() throws Exception {
@Override
public void processElement1(StreamRecord<RowData> element) throws Exception {
RowData input = element.getValue();
joinCondition.setCurrentJoinKey((NullAwareGetters) getCurrentKey());
AssociatedRecords associatedRecords =
AssociatedRecords.of(input, true, rightRecordStateView, joinCondition);
if (associatedRecords.isEmpty()) {
Expand Down Expand Up @@ -166,6 +168,7 @@ public void processElement2(StreamRecord<RowData> element) throws Exception {
RowKind inputRowKind = input.getRowKind();
input.setRowKind(RowKind.INSERT); // erase RowKind for later state updating

joinCondition.setCurrentJoinKey((NullAwareGetters) getCurrentKey());
AssociatedRecords associatedRecords =
AssociatedRecords.of(input, false, leftRecordStateView, joinCondition);
if (isAccumulateMsg) { // record is accumulate
Expand Down