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



##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
##########
@@ -0,0 +1,812 @@
+/*
+ * 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.calcite;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable;
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery;
+import org.apache.flink.table.planner.plan.nodes.calcite.LogicalLegacySink;
+import org.apache.flink.table.planner.plan.nodes.calcite.LogicalRank;
+import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSink;
+import org.apache.flink.table.planner.plan.nodes.calcite.LogicalTableAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner;
+import 
org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowTableAggregate;
+import org.apache.flink.table.planner.plan.nodes.hive.LogicalDistribution;
+import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType;
+import org.apache.flink.table.planner.plan.trait.RelWindowProperties;
+import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.TimestampKind;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalExchange;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMatch;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalSnapshot;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.planner.calcite.FlinkTypeFactory.isProctimeIndicatorType;
+import static 
org.apache.flink.table.planner.calcite.FlinkTypeFactory.isRowtimeIndicatorType;
+import static 
org.apache.flink.table.planner.calcite.FlinkTypeFactory.isTimeIndicatorType;
+import static 
org.apache.flink.table.planner.calcite.FlinkTypeFactory.isTimestampLtzIndicatorType;
+import static 
org.apache.flink.table.planner.plan.utils.MatchUtil.isFinalOnMatchTimeIndicator;
+import static 
org.apache.flink.table.planner.plan.utils.MatchUtil.isMatchRowTimeIndicator;
+import static 
org.apache.flink.table.planner.plan.utils.WindowUtil.groupingContainsWindowStartEnd;
+
+/**
+ * Traverses a {@link RelNode} tree and converts fields with {@link 
TimeIndicatorRelDataType} type.
+ * If a time attribute is accessed for a calculation, it will be materialized. 
Forwarding is allowed
+ * in some cases, but not all.
+ */
+public final class RelTimeIndicatorConverter implements RelShuttle {
+
+    private final RexBuilder rexBuilder;
+
+    private RelTimeIndicatorConverter(RexBuilder rexBuilder) {
+        this.rexBuilder = rexBuilder;
+    }
+
+    public static RelNode convert(
+            RelNode rootRel, RexBuilder rexBuilder, boolean 
needFinalTimeIndicatorConversion) {
+        RelTimeIndicatorConverter converter = new 
RelTimeIndicatorConverter(rexBuilder);
+        RelNode convertedRoot = rootRel.accept(converter);
+
+        // LogicalLegacySink and LogicalSink are already converted
+        if (rootRel instanceof LogicalLegacySink
+                || rootRel instanceof LogicalSink
+                || !needFinalTimeIndicatorConversion) {
+            return convertedRoot;
+        }
+        // materialize remaining procTime indicators
+        return converter.materializeProcTime(convertedRoot);
+    }
+
+    @Override
+    public RelNode visit(LogicalIntersect intersect) {
+        return visitSetOp(intersect);
+    }
+
+    @Override
+    public RelNode visit(LogicalUnion union) {
+        return visitSetOp(union);
+    }
+
+    @Override
+    public RelNode visit(LogicalMinus minus) {
+        return visitSetOp(minus);
+    }
+
+    @Override
+    public RelNode visit(LogicalAggregate aggregate) {
+        return visitAggregate(aggregate);
+    }
+
+    @Override
+    public RelNode visit(LogicalSort sort) {
+        return visitSimpleRel(sort);
+    }
+
+    @Override
+    public RelNode visit(LogicalMatch match) {
+        RelNode newInput = match.getInput().accept(this);
+        RexTimeIndicatorMaterializer materializer = new 
RexTimeIndicatorMaterializer(newInput);

Review comment:
       Yes, the previous behavior is wrong. but it is a little hard when I try 
to find a case to cover this, so I skip the test case. I would try harder to 
find a case.... 
   




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