twalthr commented on code in PR #26647: URL: https://github.com/apache/flink/pull/26647#discussion_r2137507998
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultiJoin.java: ########## @@ -0,0 +1,444 @@ +/* + * 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.planner.plan.nodes.exec.stream; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; +import org.apache.flink.table.planner.plan.nodes.exec.StateMetadata; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.planner.plan.nodes.exec.utils.TransformationMetadata; +import org.apache.flink.table.planner.plan.utils.JoinUtil; +import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; +import org.apache.flink.table.runtime.generated.JoinCondition; +import org.apache.flink.table.runtime.generated.MultiJoinCondition; +import org.apache.flink.table.runtime.operators.join.stream.StreamingMultiJoinOperator; +import org.apache.flink.table.runtime.operators.join.stream.StreamingMultiJoinOperator.JoinType; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.JoinKeyExtractor; +import org.apache.flink.table.runtime.operators.join.stream.utils.JoinInputSideSpec; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.calcite.rex.RexNode; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Stream {@link StreamExecNode} for N-way Joins. This node handles multi-way joins in streaming + * mode, supporting different join types and conditions for each input. + */ +@ExecNodeMetadata( + name = "stream-exec-multi-join", + version = 1, + producedTransformations = StreamExecMultiJoin.MULTI_JOIN_TRANSFORMATION, + minPlanVersion = FlinkVersion.v2_0, + minStateVersion = FlinkVersion.v2_0) +public class StreamExecMultiJoin extends ExecNodeBase<RowData> + implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> { + + public static final String MULTI_JOIN_TRANSFORMATION = "multi-join"; + + // Field names for JSON serialization + private static final String FIELD_NAME_ID = "id"; + private static final String FIELD_NAME_TYPE = "type"; + private static final String FIELD_NAME_CONFIGURATION = "configuration"; + private static final String FIELD_NAME_INPUT_PROPERTIES = "inputProperties"; + private static final String FIELD_NAME_OUTPUT_TYPE = "outputType"; + private static final String FIELD_NAME_DESCRIPTION = "description"; + private static final String FIELD_NAME_JOIN_TYPES = "joinTypes"; + private static final String FIELD_NAME_JOIN_CONDITIONS = "joinConditions"; + private static final String FIELD_NAME_GENERATED_MULTI_JOIN_CONDITION = + "generatedMultiJoinCondition"; + private static final String FIELD_NAME_JOIN_ATTRIBUTE_MAP = "joinAttributeMap"; + private static final String FIELD_NAME_INPUT_SIDE_SPECS = "inputSideSpecs"; + private static final String FIELD_NAME_STATE_METADATA_LIST = "stateMetadataList"; Review Comment: It's called `FIELD_NAME_STATE` in `ExecNode`. Take a look at StreamExecJoin. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeMultiJoinJsonSerdeTest.java: ########## @@ -0,0 +1,181 @@ +/* + * 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.planner.plan.nodes.exec.serde; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.calcite.FlinkTypeSystem; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeGraph; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMultiJoin; +import org.apache.flink.table.runtime.operators.join.stream.StreamingMultiJoinOperator.JoinType; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +class ExecNodeMultiJoinJsonSerdeTest { Review Comment: I'm fine with this test for now. But we should replace it with a restore test later. ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultiJoin.java: ########## @@ -0,0 +1,388 @@ +/* + * 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.planner.plan.nodes.exec.stream; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; +import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; +import org.apache.flink.table.planner.plan.nodes.exec.StateMetadata; +import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.planner.plan.nodes.exec.utils.TransformationMetadata; +import org.apache.flink.table.planner.plan.utils.JoinUtil; +import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; +import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; +import org.apache.flink.table.runtime.generated.JoinCondition; +import org.apache.flink.table.runtime.operators.join.stream.StreamingMultiJoinOperator.JoinType; +import org.apache.flink.table.runtime.operators.join.stream.StreamingMultiJoinOperatorFactory; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor.ConditionAttributeRef; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.JoinKeyExtractor; +import org.apache.flink.table.runtime.operators.join.stream.utils.JoinInputSideSpec; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.calcite.rex.RexNode; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Stream {@link StreamExecNode} for N-way Joins. This node handles multi-way joins in streaming + * mode, supporting different join types and conditions for each input. + */ +@ExecNodeMetadata( + name = "stream-exec-multi-join", + version = 1, + producedTransformations = StreamExecMultiJoin.MULTI_JOIN_TRANSFORMATION, + minPlanVersion = FlinkVersion.v2_1, + minStateVersion = FlinkVersion.v2_1) +public class StreamExecMultiJoin extends ExecNodeBase<RowData> + implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> { + + public static final String MULTI_JOIN_TRANSFORMATION = "multi-join"; + private static final String FIELD_NAME_JOIN_TYPES = "joinTypes"; + private static final String FIELD_NAME_JOIN_CONDITIONS = "joinConditions"; + private static final String FIELD_NAME_JOIN_ATTRIBUTE_MAP = "joinAttributeMap"; + private static final String FIELD_NAME_INPUT_UPSERT_KEYS = "inputUpsertKeys"; + private static final String FIELD_NAME_STATE_METADATA_LIST = "stateMetadataList"; + private static final String FIELD_NAME_MULTI_JOIN_CONDITION = "multiJoinCondition"; + + @JsonProperty(FIELD_NAME_JOIN_TYPES) + private final List<JoinType> joinTypes; + + @JsonProperty(FIELD_NAME_JOIN_CONDITIONS) + private final List<? extends @Nullable RexNode> joinConditions; + + @SuppressWarnings({"unused", "FieldCanBeLocal"}) + @JsonProperty(FIELD_NAME_MULTI_JOIN_CONDITION) + private final RexNode multiJoinCondition; + + @JsonProperty(FIELD_NAME_JOIN_ATTRIBUTE_MAP) + private final Map<Integer, List<ConditionAttributeRef>> joinAttributeMap; + + @JsonProperty(FIELD_NAME_INPUT_UPSERT_KEYS) + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + // List of upsert keys for each input, where each inner list corresponds to an input + // The reason it's a List<List<int[]>> is that SQL allows only one primary key but + // multiple upsert (unique) keys per input + private final List<List<int[]>> inputUpsertKeys; + + @JsonProperty(FIELD_NAME_STATE_METADATA_LIST) + @JsonInclude(JsonInclude.Include.NON_NULL) + private final List<StateMetadata> stateMetadataList; + + public StreamExecMultiJoin( + final ReadableConfig tableConfig, + final List<JoinType> joinTypes, + final List<? extends @Nullable RexNode> joinConditions, + @Nullable final RexNode multiJoinCondition, + final Map<Integer, List<AttributeBasedJoinKeyExtractor.ConditionAttributeRef>> Review Comment: ```suggestion final Map<Integer, List<ConditionAttributeRef>> ``` also below because you important the class already, which makes sense for readibility -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org