You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/01/02 03:45:03 UTC

[GitHub] [flink] beyond1920 commented on a change in pull request #10694: [FLINK-15381] [table-planner-blink] correct collation derive logic on RelSubset in RelMdCollation

beyond1920 commented on a change in pull request #10694: [FLINK-15381] [table-planner-blink] correct collation derive logic on RelSubset in RelMdCollation
URL: https://github.com/apache/flink/pull/10694#discussion_r362365384
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMdCollation.java
 ##########
 @@ -0,0 +1,561 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.adapter.enumerable.EnumerableCorrelate;
+import org.apache.calcite.adapter.enumerable.EnumerableHashJoin;
+import org.apache.calcite.adapter.enumerable.EnumerableMergeJoin;
+import org.apache.calcite.adapter.enumerable.EnumerableNestedLoopJoin;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Match;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.SortExchange;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * FlinkRelMdCollation supplies a default implementation of
+ * {@link org.apache.calcite.rel.metadata.RelMetadataQuery#collations}
+ * for the standard logical algebra.
+ */
+public class FlinkRelMdCollation implements MetadataHandler<BuiltInMetadata.Collation> {
+	public static final RelMetadataProvider SOURCE =
+			ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.COLLATIONS.method, new FlinkRelMdCollation());
+
+	//~ Constructors -----------------------------------------------------------
+
+	private FlinkRelMdCollation() {
+	}
+
+	//~ Methods ----------------------------------------------------------------
+
+	public MetadataDef<BuiltInMetadata.Collation> getDef() {
+		return BuiltInMetadata.Collation.DEF;
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(TableScan scan, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(table(scan.getTable()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Values values, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(values(mq, values.getRowType(), values.getTuples()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Project project,
+			RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(project(mq, project.getInput(), project.getProjects()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Filter rel, RelMetadataQuery mq) {
+		return mq.collations(rel.getInput());
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Calc calc, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(calc(mq, calc.getInput(), calc.getProgram()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(SortExchange sort, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(sort(sort.getCollation()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Sort sort, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(sort(sort.getCollation()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Window rel, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(window(mq, rel.getInput(), rel.groups));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(
+			EnumerableCorrelate join,
+			RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(
+				enumerableCorrelate(mq, join.getLeft(), join.getRight(), join.getJoinType()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(
+			EnumerableMergeJoin join,
+			RelMetadataQuery mq) {
+		// In general a join is not sorted. But a merge join preserves the sort
+		// order of the left and right sides.
+		return com.google.common.collect.ImmutableList.copyOf(mergeJoin(
+				mq,
+				join.getLeft(),
+				join.getRight(),
+				join.analyzeCondition().leftKeys,
+				join.analyzeCondition().rightKeys));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(
+			EnumerableHashJoin join,
+			RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(
+				enumerableHashJoin(mq, join.getLeft(), join.getRight(), join.getJoinType()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(
+			EnumerableNestedLoopJoin join,
+			RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(
+				enumerableNestedLoopJoin(mq, join.getLeft(), join.getRight(), join.getJoinType()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(Match rel, RelMetadataQuery mq) {
+		return com.google.common.collect.ImmutableList.copyOf(match(
+				mq,
+				rel.getInput(),
+				rel.getRowType(),
+				rel.getPattern(),
+				rel.isStrictStart(),
+				rel.isStrictEnd(),
+				rel.getPatternDefinitions(),
+				rel.getMeasures(),
+				rel.getAfter(),
+				rel.getSubsets(),
+				rel.isAllRows(),
+				rel.getPartitionKeys(),
+				rel.getOrderKeys(),
+				rel.getInterval()));
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(TableModify rel, RelMetadataQuery mq) {
+		return mq.collations(rel.getInput());
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(HepRelVertex rel, RelMetadataQuery mq) {
+		return mq.collations(rel.getCurrentRel());
+	}
+
+	public com.google.common.collect.ImmutableList<RelCollation> collations(RelSubset subset, RelMetadataQuery mq) {
+		if (!Bug.CALCITE_1048_FIXED) {
+			//if the best node is null, so we can get the collation based original node, due to
+			//the original node is logically equivalent as the rel.
+			RelNode rel = Util.first(subset.getBest(), subset.getOriginal());
+			return mq.collations(rel);
+		} else {
+			throw new RuntimeException("CALCITE_1048 is fixed, so check this method again!");
 
 Review comment:
   It's unsafe to throw exception here. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services