gustavodemorais commented on code in PR #26966: URL: https://github.com/apache/flink/pull/26966#discussion_r2344414825
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java: ########## @@ -723,6 +737,71 @@ private boolean containsSnapshot(RelNode relNode) { } } + /** + * Processes state TTL hints for a given side (left or right) of a join operation. Review Comment: I've renamed it to handleStateTtlHintsForInput ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/StateTtlHint.java: ########## @@ -92,6 +96,32 @@ public static Map<Integer, Long> getStateTtlFromHintOnBiRel(List<RelHint> hints) return stateTtlFromHint; } + /** + * Get the state ttl from hints from the listOptions inside the STATE_TTL {@link RelHint} if + * present. Else if returns an empty map. Used for nodes with multiple inputs such as {@link + * StreamPhysicalMultiJoin}. + * + * @return The key of the map is the input side. The value of the map is the state ttl in Review Comment: I've refactored the javadoc for the function ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/rules/MultiJoin.java: ########## @@ -0,0 +1,306 @@ +/* + * 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.calcite.rel.rules; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.calcite.linq4j.Ord; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.AbstractRelNode; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelWriter; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.calcite.util.ImmutableNullableList; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +/** + * A MultiJoin represents a join of N inputs, whereas regular Joins represent strictly binary joins. Review Comment: I've created a calcite issue, update the javadoc to link to the issue CALCITE-7164 and added a bit more context. I've also added FLINK MODIFICATION BEGIN/END to the parts of the class we've changed -- 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