You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "asolimando (via GitHub)" <gi...@apache.org> on 2023/06/22 17:28:52 UTC

[GitHub] [calcite] asolimando commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove by the foreign-unique constraint in the catalog

asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1238730578


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -114,18 +114,18 @@ public interface ForeignKeys extends Metadata {
             BuiltInMethod.FOREIGN_KEYS.method);
 
     /**
-     * Determines the list of foreign keys for this expression. Foreign keys are
-     * represented as an {@link org.apache.calcite.util.ImmutableBitSet}, where
-     * each bit position represents the column ordinal is foreign key.
+     * Extract foreign keys from relNode. Foreign keys are represented as an
+     * {@link org.apache.calcite.util.ImmutableBitSet}, where each bit position
+     * represents a 0-based output column ordinal.

Review Comment:
   Nitpick: maybe we want to use a link to `RelNode` class in the javadoc?
   
   ```suggestion
        * Extract foreign keys from {@link org.apache.calcite.rel.RelNode}. 
        * Foreign keys are represented as an {@link org.apache.calcite.util.ImmutableBitSet}, 
        * where each bit position represents a 0-based output column ordinal.
   ```



##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -114,18 +114,18 @@ public interface ForeignKeys extends Metadata {
             BuiltInMethod.FOREIGN_KEYS.method);
 
     /**
-     * Determines the list of foreign keys for this expression. Foreign keys are
-     * represented as an {@link org.apache.calcite.util.ImmutableBitSet}, where
-     * each bit position represents the column ordinal is foreign key.
+     * Extract foreign keys from relNode. Foreign keys are represented as an
+     * {@link org.apache.calcite.util.ImmutableBitSet}, where each bit position
+     * represents a 0-based output column ordinal.
      *
-     * @param ignoreNulls if true, ignore null values when determining
-     *                    whether the keys are foreign keys
+     * @param containNulls if true, allow containing null values when determining
+     *                     whether the keys are foreign keys
      *
-     * @return set of foreign keys, or empty set if this information
-     * cannot be determined (whereas null indicates definitely
-     * no constraint at all)
+     * @return bit set of foreign keys, or empty if not enough information is
+     * available to make that determination (whereas empty indicates definitely
+     * no foreign keys at all)

Review Comment:
   The part in the parenthesis is now redundant, let's remove it



##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMdForeignKeys.java:
##########
@@ -174,72 +176,88 @@ private static ImmutableBitSet getProjectForeignKeys(SingleRel rel, RelMetadataQ
         allOutColumns = allOutColumns.union(outColumns);
       }
     }
-    return allOutColumns.isEmpty() ? EMPTY_BIT_SET : allOutColumns;
+    return allOutColumns;
   }
 
   public ImmutableBitSet getForeignKeys(TableScan rel, RelMetadataQuery mq,
-      boolean ignoreNulls) {
+      boolean containNulls) {
     final RelOptTable table = rel.getTable();
     final BuiltInMetadata.ForeignKeys.Handler handler =
         table.unwrap(BuiltInMetadata.ForeignKeys.Handler.class);
     if (handler != null) {
-      return handler.getForeignKeys(rel, mq, ignoreNulls);
+      return handler.getForeignKeys(rel, mq, containNulls);
     }
 
     final List<RelReferentialConstraint> referentialConstraints =
         table.getReferentialConstraints();
-    if (referentialConstraints == null || referentialConstraints.size() == 0) {
+    if (referentialConstraints == null || referentialConstraints.isEmpty()) {
       return EMPTY_BIT_SET;
     }
-    final List<IntPair> foreignUniquePair = referentialConstraints.stream()
+    Set<Integer> foreignKeys = referentialConstraints.stream()
         .map(RelReferentialConstraint::getColumnPairs)
         .flatMap(Collection::stream)
-        .collect(Collectors.toList());
+        .map(pair -> {
+          return pair.source;
+        })

Review Comment:
   Can be simplified to:
   ```suggestion
           .map(pair -> pair.source)
   ```



##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -78,47 +99,62 @@ public ProjectJoinRemoveRule(
     final Project project = call.rel(0);
     final Join join = call.rel(1);
     final boolean isLeftJoin = join.getJoinType() == JoinRelType.LEFT;
-    int lower = isLeftJoin
-        ? join.getLeft().getRowType().getFieldCount() : 0;
-    int upper = isLeftJoin
-        ? join.getRowType().getFieldCount()
-        : join.getLeft().getRowType().getFieldCount();
-
-    // Check whether the project uses columns whose index is between
-    // lower(included) and upper(excluded).
-    for (RexNode expr: project.getProjects()) {
-      if (RelOptUtil.InputFinder.bits(expr).asList().stream().anyMatch(
-          i -> i >= lower && i < upper)) {
-        return;
-      }
-    }
+    final boolean isRightJoin = join.getJoinType() == JoinRelType.RIGHT;
+    final boolean isInnerJoin = join.getJoinType() == JoinRelType.INNER;
 
-    final List<Integer> leftKeys = new ArrayList<>();
-    final List<Integer> rightKeys = new ArrayList<>();
-    RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(),
-        join.getCondition(), leftKeys, rightKeys,
-        new ArrayList<>());
+    // Check project range
+    ImmutableBitSet projectBits = RelOptUtil.InputFinder.bits(project.getProjects(), null);
+    final int leftFieldsNum = join.getLeft().getRowType().getFieldCount();
+    final boolean onlyUseLeft = projectBits.asList().stream()
+        .allMatch(i -> i >= 0 && i < leftFieldsNum);
+    final boolean onlyUseRight = projectBits.asList().stream().allMatch(i -> i >= leftFieldsNum
+            && i < join.getRowType().getFieldCount());
 
-    final List<Integer> joinKeys = isLeftJoin ? rightKeys : leftKeys;
-    final ImmutableBitSet.Builder columns = ImmutableBitSet.builder();
-    joinKeys.forEach(columns::set);
+    if (!onlyUseLeft && !onlyUseRight) {
+      return;
+    }
+    if (isLeftJoin && !onlyUseLeft) {
+      return;
+    }
+    if (isRightJoin && !onlyUseRight) {
+      return;
+    }
 
+    JoinInfo joinInfo = join.analyzeCondition();
+    final List<Integer> leftKeys = joinInfo.leftKeys;
+    final List<Integer> rightKeys = joinInfo.rightKeys;
     final RelMetadataQuery mq = call.getMetadataQuery();
-    if (!Boolean.TRUE.equals(
-        mq.areColumnsUnique(isLeftJoin ? join.getRight() : join.getLeft(),
-        columns.build()))) {
+
+    // For inner join, should also check foreign keys additionally
+    if (JoinRelType.INNER == join.getJoinType()) {
+      final ImmutableBitSet leftForeignKeys = mq.getForeignKeys(join.getLeft(), false);
+      final ImmutableBitSet rightForeignKeys = mq.getForeignKeys(join.getRight(), false);
+      if (onlyUseLeft && !areForeignKeysValid(
+          leftKeys, rightKeys, leftForeignKeys, mq, join.getLeft(), join.getRight())) {
+        return;
+      }
+      if (onlyUseRight && !areForeignKeysValid(
+          rightKeys, leftKeys, rightForeignKeys, mq, join.getRight(), join.getLeft())) {
+        return;
+      }

Review Comment:
   Bail out fast and avoids calling `getForeignKeys` for the right side if you should already bail out on left:
   ```suggestion
               final ImmutableBitSet leftForeignKeys = mq.getForeignKeys(join.getLeft(), false);
         if (onlyUseLeft && !areForeignKeysValid(
             leftKeys, rightKeys, leftForeignKeys, mq, join.getLeft(), join.getRight())) {
           return;
         }
         final ImmutableBitSet rightForeignKeys = mq.getForeignKeys(join.getRight(), false);
         if (onlyUseRight && !areForeignKeysValid(
             rightKeys, leftKeys, rightForeignKeys, mq, join.getRight(), join.getLeft())) {
           return;
         }
   ```



##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMdForeignKeys.java:
##########
@@ -174,72 +176,88 @@ private static ImmutableBitSet getProjectForeignKeys(SingleRel rel, RelMetadataQ
         allOutColumns = allOutColumns.union(outColumns);
       }
     }
-    return allOutColumns.isEmpty() ? EMPTY_BIT_SET : allOutColumns;
+    return allOutColumns;
   }
 
   public ImmutableBitSet getForeignKeys(TableScan rel, RelMetadataQuery mq,
-      boolean ignoreNulls) {
+      boolean containNulls) {
     final RelOptTable table = rel.getTable();
     final BuiltInMetadata.ForeignKeys.Handler handler =
         table.unwrap(BuiltInMetadata.ForeignKeys.Handler.class);
     if (handler != null) {
-      return handler.getForeignKeys(rel, mq, ignoreNulls);
+      return handler.getForeignKeys(rel, mq, containNulls);
     }
 
     final List<RelReferentialConstraint> referentialConstraints =
         table.getReferentialConstraints();
-    if (referentialConstraints == null || referentialConstraints.size() == 0) {
+    if (referentialConstraints == null || referentialConstraints.isEmpty()) {
       return EMPTY_BIT_SET;
     }
-    final List<IntPair> foreignUniquePair = referentialConstraints.stream()
+    Set<Integer> foreignKeys = referentialConstraints.stream()
         .map(RelReferentialConstraint::getColumnPairs)
         .flatMap(Collection::stream)
-        .collect(Collectors.toList());
+        .map(pair -> {
+          return pair.source;
+        })
+        .collect(Collectors.toSet());
 
-    List<Integer> foreignKeys = IntPair.left(foreignUniquePair);
-    if (!ignoreNulls) {
+    if (!containNulls) {
       final List<RelDataTypeField> fieldList = rel.getRowType().getFieldList();
       foreignKeys = foreignKeys.stream()
           .filter(index -> !fieldList.get(index).getType().isNullable())
-          .collect(Collectors.toList());
+          .collect(Collectors.toSet());
     }
-    return foreignKeys.isEmpty() ? EMPTY_BIT_SET : ImmutableBitSet.of(foreignKeys);
+    return ImmutableBitSet.of(foreignKeys);
   }
 
-  public ImmutableBitSet getForeignKeys(SetOp rel, RelMetadataQuery mq,
-      boolean ignoreNulls) {
+  /**
+   * The foreign keys of Union are precisely the intersection of its every
+   * input foreign keys.
+   */
+  public ImmutableBitSet getForeignKeys(Union rel, RelMetadataQuery mq,
+      boolean containNulls) {
 
     ImmutableBitSet foreignKeys = ImmutableBitSet.of();
     for (RelNode input : rel.getInputs()) {
-      ImmutableBitSet inputForeignKeys = mq.getForeignKeys(input, ignoreNulls);
+      ImmutableBitSet inputForeignKeys = mq.getForeignKeys(input, containNulls);
       if (inputForeignKeys.isEmpty()) {
         return EMPTY_BIT_SET;
       }
       foreignKeys = foreignKeys.isEmpty()
           ? inputForeignKeys : foreignKeys.intersect(inputForeignKeys);
     }
-    if (foreignKeys.isEmpty()) {
-      return EMPTY_BIT_SET;
+    return foreignKeys;
+  }
+
+  /**
+   * The foreign keys of Intersect are precisely the union set of its every

Review Comment:
   I think it's the same as per `Union`, that is, the intersection of the FKs from the operands. Because you can apply this set operations on whatever pair of relations having the "same" schema type, but you can very well mix up columns with different FK relationship for the same position.
   
   The same applies to `Minus`, so you probably want to revert back to a single implementation for `SetOp` and revise the tests.
   
   In any case, `areForeignKeysValid` will bail out on such cases, as it will try to track down the original column via `RelMetadataQuery.getColumnOrigin()`, and if a mix is found, more origins will be detected, and it will return `null`.
   
   Since it's a bit tricky I suggest to add at least some basic tests to `RelOptRulesTest` per each set operator to confirm our understanding (simple case and tricky one like a mix of column origin under the same position).



##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMdForeignKeys.java:
##########
@@ -174,72 +176,88 @@ private static ImmutableBitSet getProjectForeignKeys(SingleRel rel, RelMetadataQ
         allOutColumns = allOutColumns.union(outColumns);
       }
     }
-    return allOutColumns.isEmpty() ? EMPTY_BIT_SET : allOutColumns;
+    return allOutColumns;
   }
 
   public ImmutableBitSet getForeignKeys(TableScan rel, RelMetadataQuery mq,
-      boolean ignoreNulls) {
+      boolean containNulls) {
     final RelOptTable table = rel.getTable();
     final BuiltInMetadata.ForeignKeys.Handler handler =
         table.unwrap(BuiltInMetadata.ForeignKeys.Handler.class);
     if (handler != null) {
-      return handler.getForeignKeys(rel, mq, ignoreNulls);
+      return handler.getForeignKeys(rel, mq, containNulls);
     }
 
     final List<RelReferentialConstraint> referentialConstraints =
         table.getReferentialConstraints();
-    if (referentialConstraints == null || referentialConstraints.size() == 0) {
+    if (referentialConstraints == null || referentialConstraints.isEmpty()) {
       return EMPTY_BIT_SET;
     }
-    final List<IntPair> foreignUniquePair = referentialConstraints.stream()
+    Set<Integer> foreignKeys = referentialConstraints.stream()

Review Comment:
   `final` modifier was good, we can keep it:
   ```suggestion
       final Set<Integer> foreignKeys = referentialConstraints.stream()
   ```



##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -78,40 +86,56 @@ public ProjectJoinRemoveRule(
     final Project project = call.rel(0);
     final Join join = call.rel(1);
     final boolean isLeftJoin = join.getJoinType() == JoinRelType.LEFT;
-    int lower = isLeftJoin
-        ? join.getLeft().getRowType().getFieldCount() : 0;
-    int upper = isLeftJoin
-        ? join.getRowType().getFieldCount()
-        : join.getLeft().getRowType().getFieldCount();
-
-    // Check whether the project uses columns whose index is between
-    // lower(included) and upper(excluded).
-    for (RexNode expr: project.getProjects()) {
-      if (RelOptUtil.InputFinder.bits(expr).asList().stream().anyMatch(
-          i -> i >= lower && i < upper)) {
-        return;
-      }
-    }
+    final boolean isRightJoin = join.getJoinType() == JoinRelType.RIGHT;
+    final boolean isInnerJoin = join.getJoinType() == JoinRelType.INNER;
 
-    final List<Integer> leftKeys = new ArrayList<>();
-    final List<Integer> rightKeys = new ArrayList<>();
-    RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(),
-        join.getCondition(), leftKeys, rightKeys,
-        new ArrayList<>());
+    // Check project range
+    ImmutableBitSet projectBits = RelOptUtil.InputFinder.bits(project.getProjects(), null);
+    boolean onlyUseLeft = projectBits.asList().stream()
+        .allMatch(i -> i >= 0 && i < join.getLeft().getRowType().getFieldCount());
+    boolean onlyUseRight = projectBits.asList().stream()
+        .allMatch(i -> i >= join.getLeft().getRowType().getFieldCount()
+            && i < join.getRowType().getFieldCount());
 
-    final List<Integer> joinKeys = isLeftJoin ? rightKeys : leftKeys;
-    final ImmutableBitSet.Builder columns = ImmutableBitSet.builder();
-    joinKeys.forEach(columns::set);
+    if (!onlyUseLeft && !onlyUseRight) {
+      return;
+    }
+    if (isLeftJoin && !onlyUseLeft) {
+      return;
+    }
+    if (isRightJoin && !onlyUseRight) {
+      return;
+    }
 
+    JoinInfo joinInfo = join.analyzeCondition();
+    final List<Integer> leftKeys = joinInfo.leftKeys;
+    final List<Integer> rightKeys = joinInfo.rightKeys;
     final RelMetadataQuery mq = call.getMetadataQuery();
+
+    // For inner join, should also check foreign keys additionally
+    if (JoinRelType.INNER == join.getJoinType()) {
+      final ImmutableBitSet leftForeignKeys = mq.getForeignKeys(join.getLeft(), false);
+      final ImmutableBitSet rightForeignKeys = mq.getForeignKeys(join.getRight(), false);
+      if (onlyUseLeft && !areForeignKeysValid(
+          leftKeys, rightKeys, leftForeignKeys, mq, join.getLeft(), join.getRight())) {
+        return;
+      }
+      if (onlyUseRight && !areForeignKeysValid(
+          rightKeys, leftKeys, rightForeignKeys, mq, join.getRight(), join.getLeft())) {
+        return;
+      }
+    }
+
+    BooleanSupplier isLeftSideReserved = () -> isLeftJoin || (isInnerJoin && onlyUseLeft);

Review Comment:
   Sorry, I can't use the "suggest change" feature as some lines have been deleted, but what I meant is the following:
   ```
       boolean isLeftSideReserved = isLeftJoin || (isInnerJoin && onlyUseLeft);
       final List<Integer> joinKeys = isLeftSideReserved ? rightKeys : leftKeys;
       if (Boolean.FALSE.equals(
           mq.areColumnsUnique(isLeftSideReserved ? join.getRight() : join.getLeft(),
               ImmutableBitSet.of(joinKeys)))) {
         return;
       }
   
       RelNode node;
       if (isLeftSideReserved) {
   ```
   In this way you reuse the variable but you use a simpler primitive `boolean` type. WDYT?



##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -127,6 +163,46 @@ public ProjectJoinRemoveRule(
     call.transformTo(node);
   }
 
+  /**
+   * Check as following:
+   * 1. Make sure that every foreign column is always a foreign key.
+   * 2. The target of foreign key is the correct corresponding unique key.
+   */
+  private static boolean areForeignKeysValid(List<Integer> foreignColumns,
+      List<Integer> uniqueColumns, ImmutableBitSet foreignKeys, RelMetadataQuery mq,
+      RelNode foreignSideRel, RelNode uniqueSideRel) {
+    if (foreignKeys.isEmpty()) {
+      return false;
+    }
+    if (!foreignKeys.contains(ImmutableBitSet.of(foreignColumns))) {
+      return false;
+    }
+    List<RelReferentialConstraint> referentialConstraints;
+    for (IntPair foreignUniqueKey : IntPair.zip(foreignColumns, uniqueColumns)) {
+      RelColumnOrigin foreignOrigin = mq.getColumnOrigin(foreignSideRel, foreignUniqueKey.source);
+      RelColumnOrigin uniqueOrigin = mq.getColumnOrigin(uniqueSideRel, foreignUniqueKey.target);
+      if (foreignOrigin == null || uniqueOrigin == null) {
+        return false;
+      }

Review Comment:
   You want to bail out as soon as possible in rules, and doing so also spares a call to `RelMetadataQuery` which are generally expensive:
   ```suggestion
         RelColumnOrigin foreignOrigin = mq.getColumnOrigin(foreignSideRel, foreignUniqueKey.source);
         if (foreignOrigin == null) {
           return false;
         }
         RelColumnOrigin uniqueOrigin = mq.getColumnOrigin(uniqueSideRel, foreignUniqueKey.target);
         if (uniqueOrigin == null) {
           return false;
         }
   ```



##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java:
##########
@@ -510,24 +510,24 @@ public static RelMetadataQuery instance() {
   }
 
   /**
-   * Returns the
-   * {@link BuiltInMetadata.ForeignKeys#getForeignKeys(boolean)}
-   * statistic.
+   * Extract foreign keys from relNode. Foreign keys are represented as an
+   * {@link org.apache.calcite.util.ImmutableBitSet}, where each bit position
+   * represents a 0-based output column ordinal.

Review Comment:
   Similarly to `BuiltInMetadata`:
   ```suggestion
      * Extract foreign keys from {@link org.apache.calcite.rel.RelNode}.
      * Foreign keys are represented as an {@link org.apache.calcite.util.ImmutableBitSet},
      * where each bit position represents a 0-based output column ordinal.
   ```



##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -128,21 +141,20 @@ public ProjectJoinRemoveRule(
 
     BooleanSupplier isLeftSideReserved = () -> isLeftJoin || (isInnerJoin && onlyUseLeft);
     final List<Integer> joinKeys = isLeftSideReserved.getAsBoolean() ? rightKeys : leftKeys;
-    if (!Boolean.TRUE.equals(
+    if (Boolean.FALSE.equals(
         mq.areColumnsUnique(isLeftSideReserved.getAsBoolean() ? join.getRight() : join.getLeft(),
             ImmutableBitSet.of(joinKeys)))) {
       return;
     }
 
     RelNode node;
     if (isLeftSideReserved.getAsBoolean()) {
-      node = project
-          .copy(project.getTraitSet(), join.getLeft(), project.getProjects(),
-              project.getRowType());
+      node =
+          project.copy(project.getTraitSet(), join.getLeft(),
+              project.getProjects(), project.getRowType());

Review Comment:
   Sorry to be pedantic on this, but more lines mean more scrolling and its easier to reason on what can fit into a "page", so in cases like this, when the end result is not a very dense line, I prefer to not go to the next line.
   
   ```suggestion
         node = project.copy(project.getTraitSet(), join.getLeft(),
             project.getProjects(), project.getRowType());
   ```
   WDYT?



##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -78,47 +99,62 @@ public ProjectJoinRemoveRule(
     final Project project = call.rel(0);
     final Join join = call.rel(1);
     final boolean isLeftJoin = join.getJoinType() == JoinRelType.LEFT;
-    int lower = isLeftJoin
-        ? join.getLeft().getRowType().getFieldCount() : 0;
-    int upper = isLeftJoin
-        ? join.getRowType().getFieldCount()
-        : join.getLeft().getRowType().getFieldCount();
-
-    // Check whether the project uses columns whose index is between
-    // lower(included) and upper(excluded).
-    for (RexNode expr: project.getProjects()) {
-      if (RelOptUtil.InputFinder.bits(expr).asList().stream().anyMatch(
-          i -> i >= lower && i < upper)) {
-        return;
-      }
-    }
+    final boolean isRightJoin = join.getJoinType() == JoinRelType.RIGHT;
+    final boolean isInnerJoin = join.getJoinType() == JoinRelType.INNER;
 
-    final List<Integer> leftKeys = new ArrayList<>();
-    final List<Integer> rightKeys = new ArrayList<>();
-    RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(),
-        join.getCondition(), leftKeys, rightKeys,
-        new ArrayList<>());
+    // Check project range
+    ImmutableBitSet projectBits = RelOptUtil.InputFinder.bits(project.getProjects(), null);
+    final int leftFieldsNum = join.getLeft().getRowType().getFieldCount();
+    final boolean onlyUseLeft = projectBits.asList().stream()
+        .allMatch(i -> i >= 0 && i < leftFieldsNum);
+    final boolean onlyUseRight = projectBits.asList().stream().allMatch(i -> i >= leftFieldsNum
+            && i < join.getRowType().getFieldCount());

Review Comment:
   To match the style you used above:
   ```suggestion
       final boolean onlyUseRight = projectBits.asList().stream()
           .allMatch(i -> i >= leftFieldsNum && i < join.getRowType().getFieldCount());
   ```



##########
testkit/src/main/java/org/apache/calcite/test/RelMetadataFixture.java:
##########
@@ -264,6 +265,24 @@ public RelMetadataFixture assertPercentageOriginalRows(Matcher<Double> matcher)
     return this;
   }
 
+  public RelMetadataFixture assertForeignKeys(
+      Matcher<ImmutableBitSet> constraintMatcher) {
+    RelNode rel = toRel();
+    RelMetadataQuery mq = rel.getCluster().getMetadataQuery();
+    ImmutableBitSet foreignKeys = mq.getForeignKeys(rel, false);

Review Comment:
   I see your point, but we are still supporting a "mode" we aren't testing, and which comes with bits of code, so it needs proper testing like any other part of our codebase, we should have tests for `ignoreNulls = true` too.



##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMdForeignKeys.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.IntPair;
+
+import com.google.common.collect.ImmutableListMultimap;
+import com.google.common.collect.Maps;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * RelMdReferentialConstraits supplies a default implementation of
+ * {@link RelMetadataQuery#getForeignKeys} for the standard logical algebra.
+ * The relNodes supported are the same to {@link RelMetadataQuery#getUniqueKeys(RelNode)}
+ */
+public class RelMdForeignKeys
+    implements MetadataHandler<BuiltInMetadata.ForeignKeys> {
+  public static final ImmutableBitSet EMPTY_BIT_SET = ImmutableBitSet.of();
+  public static final RelMetadataProvider SOURCE =
+      ReflectiveRelMetadataProvider.reflectiveSource(
+          new RelMdForeignKeys(), BuiltInMetadata.ForeignKeys.Handler.class);
+
+//~ Constructors -----------------------------------------------------------
+
+  private RelMdForeignKeys() {}
+
+//~ Methods ----------------------------------------------------------------
+
+  @Override public MetadataDef<BuiltInMetadata.ForeignKeys> getDef() {
+    return BuiltInMetadata.ForeignKeys.DEF;
+  }
+
+  public ImmutableBitSet getForeignKeys(Filter rel, RelMetadataQuery mq, boolean ignoreNulls) {
+    return mq.getForeignKeys(rel.getInput(), ignoreNulls);
+  }
+
+  public ImmutableBitSet getForeignKeys(Sort rel, RelMetadataQuery mq, boolean ignoreNulls) {
+    return mq.getForeignKeys(rel.getInput(), ignoreNulls);
+  }
+
+  public ImmutableBitSet getForeignKeys(Correlate rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    return mq.getForeignKeys(rel.getLeft(), ignoreNulls);
+  }
+
+  public ImmutableBitSet getForeignKeys(TableModify rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    return mq.getForeignKeys(rel.getInput(), ignoreNulls);
+  }
+
+  public ImmutableBitSet getForeignKeys(Join rel, RelMetadataQuery mq, boolean ignoreNulls) {
+    final RelNode left = rel.getLeft();
+    final RelNode right = rel.getRight();
+    if (!rel.getJoinType().projectsRight()) {
+      // only return the foreign keys from the LHS since a semijoin only
+      // returns the LHS
+      return mq.getForeignKeys(left, ignoreNulls);
+    }
+
+    int nLeftColumns = rel.getLeft().getRowType().getFieldList().size();
+    ImmutableBitSet outForeignKeys = ImmutableBitSet.of();
+
+    if (!rel.getJoinType().generatesNullsOnLeft() || ignoreNulls) {
+      ImmutableBitSet leftInputForeignKeys = mq.getForeignKeys(left, ignoreNulls);
+      outForeignKeys = outForeignKeys.union(leftInputForeignKeys);
+    }
+    if (!rel.getJoinType().generatesNullsOnRight() || ignoreNulls) {
+      ImmutableBitSet rightInputForeignKeys = mq.getForeignKeys(right, ignoreNulls);
+      ImmutableBitSet.Builder rightOutForeignKeys = ImmutableBitSet.builder();
+      for (int index : rightInputForeignKeys.asList()) {
+        rightOutForeignKeys.set(index + nLeftColumns);
+      }
+      outForeignKeys = outForeignKeys.union(rightOutForeignKeys.build());
+    }
+    return outForeignKeys;
+  }
+
+  public ImmutableBitSet getForeignKeys(Aggregate rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    final ImmutableBitSet groupSet = rel.getGroupSet();
+    if (groupSet.isEmpty()) {
+      return EMPTY_BIT_SET;
+    }
+    final ImmutableBitSet inputForeignKeys = mq.getForeignKeys(rel.getInput(), ignoreNulls);
+    if (inputForeignKeys.isEmpty()) {
+      return EMPTY_BIT_SET;
+    }
+    return groupSet.intersect(inputForeignKeys);
+  }
+
+  public ImmutableBitSet getForeignKeys(Project rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    return getProjectForeignKeys(rel, mq, ignoreNulls, rel.getProjects());
+  }
+
+  public ImmutableBitSet getForeignKeys(Calc rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    RexProgram program = rel.getProgram();
+    return getProjectForeignKeys(rel, mq, ignoreNulls,
+        Util.transform(program.getProjectList(), program::expandLocalRef));
+  }
+
+  private static ImmutableBitSet getProjectForeignKeys(SingleRel rel, RelMetadataQuery mq,
+      boolean ignoreNulls,
+      List<RexNode> projExprs) {
+
+    // Single input can be mapped to multiple outputs
+    final ImmutableListMultimap.Builder<Integer, Integer> inToOutIndexBuilder =
+        ImmutableListMultimap.builder();
+    final ImmutableBitSet.Builder inColumnsBuilder = ImmutableBitSet.builder();
+    for (int i = 0; i < projExprs.size(); i++) {
+      RexNode projExpr = projExprs.get(i);
+      if (projExpr instanceof RexInputRef) {
+        int inputIndex = ((RexInputRef) projExpr).getIndex();
+        inToOutIndexBuilder.put(inputIndex, i);
+        inColumnsBuilder.set(inputIndex);
+      }
+    }
+    final ImmutableBitSet inColumnsUsed = inColumnsBuilder.build();
+    if (inColumnsUsed.isEmpty()) {
+      return EMPTY_BIT_SET;
+    }
+
+    final Map<Integer, ImmutableBitSet> mapInToOutPos =
+        Maps.transformValues(inToOutIndexBuilder.build().asMap(), ImmutableBitSet::of);
+    final ImmutableBitSet inputForeignKeys =
+        mq.getForeignKeys(rel.getInput(), ignoreNulls);
+    if (inputForeignKeys.isEmpty()) {
+      return EMPTY_BIT_SET;
+    }
+
+    ImmutableBitSet allOutColumns = ImmutableBitSet.of();
+    for (int index : inputForeignKeys.asList()) {
+      ImmutableBitSet outColumns = mapInToOutPos.get(index);
+      if (outColumns != null && !outColumns.isEmpty()) {
+        allOutColumns = allOutColumns.union(outColumns);
+      }
+    }
+    return allOutColumns.isEmpty() ? EMPTY_BIT_SET : allOutColumns;
+  }
+
+  public ImmutableBitSet getForeignKeys(TableScan rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+    final RelOptTable table = rel.getTable();
+    final BuiltInMetadata.ForeignKeys.Handler handler =
+        table.unwrap(BuiltInMetadata.ForeignKeys.Handler.class);
+    if (handler != null) {
+      return handler.getForeignKeys(rel, mq, ignoreNulls);
+    }
+
+    final List<RelReferentialConstraint> referentialConstraints =
+        table.getReferentialConstraints();
+    if (referentialConstraints == null || referentialConstraints.size() == 0) {
+      return EMPTY_BIT_SET;
+    }
+    final List<IntPair> foreignUniquePair = referentialConstraints.stream()
+        .map(RelReferentialConstraint::getColumnPairs)
+        .flatMap(Collection::stream)
+        .collect(Collectors.toList());
+
+    List<Integer> foreignKeys = IntPair.left(foreignUniquePair);
+    if (!ignoreNulls) {

Review Comment:
   I see what you mean and I agree, naming is particularly hard on this parameter, I think what is used in `UniqueKeys` is a bit confusing, but it's even more confusing to have a mix of both, so for uniformity I'd rather switch back to `ignoreNulls` and rely on the javadoc to explain properly what it means



##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java:
##########
@@ -510,24 +510,24 @@ public static RelMetadataQuery instance() {
   }
 
   /**
-   * Returns the
-   * {@link BuiltInMetadata.ForeignKeys#getForeignKeys(boolean)}
-   * statistic.
+   * Extract foreign keys from relNode. Foreign keys are represented as an
+   * {@link org.apache.calcite.util.ImmutableBitSet}, where each bit position
+   * represents a 0-based output column ordinal.
    *
    * @param rel         the relational expression
-   * @param ignoreNulls if true, ignore null values when determining
-   *                    whether the keys are foreign keys
+   * @param containNulls if true, allow containing null values when determining
+   *                     whether the keys are foreign keys
    *
-   * @return            set of keys, or empty set if this information cannot be determined
-   *                    (whereas empty set indicates definitely no keys at all)
+   * @return bit set of foreign keys, or empty if not enough information is
+   * available to make that determination (whereas empty indicates definitely

Review Comment:
   Here too, now that we don't say we return `null` anymore, the part within parentheses seems redundant



-- 
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: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org