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

[GitHub] [calcite] JingDas opened a new pull request, #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

JingDas opened a new pull request, #3264:
URL: https://github.com/apache/calcite/pull/3264

   This feature is mainly about expanding ProjectJoinRemoveRule to support inner join remove
   by the foreign-unique constraint in the catalog.
   
   The main steps are as follows:
   1. Analyse the column used by project, and then split them to left and right side.
   2. Acccording to the project info above and outer join type, bail out in some scene.
   3. Get join info such as join keys.
   4. For inner join check foreign and unique keys, these may use
   RelMetadataQuery#getForeignKeys(newly add, similar to RelMetadataQuery#getUniqueKeys).
   5. Check remove side join keys are areColumnsUnique both for outer join and inner join.
   6. If all done, calculate the fianl project and transform. 
   
   Test cases are added to RelOptRulesTest and RelMetadataTest.


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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239153027


##########
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:
   OK, got your point, I will add it.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230228425


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.

Review Comment:
   You are right, the getForeignKeys method is similar to getUniqueKeys.   
   The result of getForeignKeys method represent the foreign key pos above the current relNode, and the index is "0-based".   
   My comment was misunderstood, I'll edit it.



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592301517

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229970092


##########
testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java:
##########
@@ -99,6 +108,12 @@ private void registerTableEmpNullables(MockTable empNullablesTable, Fixture fixt
     empNullablesTable.addColumn("COMM", fixture.intTypeNull);
     empNullablesTable.addColumn("DEPTNO", fixture.intTypeNull);
     empNullablesTable.addColumn("SLACKER", fixture.booleanTypeNull);
+    empNullablesTable.setReferentialConstraints(
+        Lists.newArrayList(
+            RelReferentialConstraintImpl.of(
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "EMPNULLABLES"),
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "DEPT"),
+        Lists.newArrayList(IntPair.of(7, 0)))));

Review Comment:
   Same here, a comment mentioning the column name from both tables would help clarify



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230002190


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {
+    sql("select name as dname, emp.deptno, dept.name from emp left outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysJoinRightOuter() {
+    sql("select name as dname, emp.deptno, dept.name, emp.deptno from dept right outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1, 3)));
+  }
+
+  @Test void testForeignKeysJoinOuterEmpty() {
+    sql("select name as dname, emp.deptno, dept.name from dept left outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysJoinFullOuterEmpty() {
+    sql("select name as dname, emp.deptno from emp full outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysJoinAggregateFilter() {
+    sql("select dept.name, emp_agg.deptno, emp_agg.ename "
+        + "from dept "
+        + "right join "
+        + "(select count(sal), deptno, ename from emp group by deptno, ename) emp_agg "
+        + "on dept.deptno = emp_agg.deptno "
+        + "where emp_agg.ename = 'job'")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysValuesEmpty() {
+    sql("values(1,2,3)")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysUnion() {

Review Comment:
   Can we have tests for `intersect` and `minus` 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.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229931137


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

Review Comment:
   Apart from the typo, I don't see any class with this name, can you clarify?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229997965


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {
+    sql("select name as dname, emp.deptno, dept.name from emp left outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysJoinRightOuter() {

Review Comment:
   ```suggestion
     @Test void testForeignKeysRightOuterJoin() {
   ```



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233406738


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {
+    sql("select name as dname, emp.deptno, dept.name from emp left outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysJoinRightOuter() {
+    sql("select name as dname, emp.deptno, dept.name, emp.deptno from dept right outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1, 3)));
+  }
+
+  @Test void testForeignKeysJoinOuterEmpty() {
+    sql("select name as dname, emp.deptno, dept.name from dept left outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysJoinFullOuterEmpty() {
+    sql("select name as dname, emp.deptno from emp full outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysJoinAggregateFilter() {
+    sql("select dept.name, emp_agg.deptno, emp_agg.ename "
+        + "from dept "
+        + "right join "
+        + "(select count(sal), deptno, ename from emp group by deptno, ename) emp_agg "
+        + "on dept.deptno = emp_agg.deptno "
+        + "where emp_agg.ename = 'job'")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysValuesEmpty() {
+    sql("values(1,2,3)")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysUnion() {

Review Comment:
   Add it



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233407095


##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java:
##########
@@ -505,6 +509,29 @@ public static RelMetadataQuery instance() {
     }
   }
 
+  /**
+   * Returns the
+   * {@link BuiltInMetadata.ForeignKeys#getForeignKeys(boolean)}
+   * statistic.
+   *
+   * @param rel         the relational expression
+   * @param ignoreNulls if true, ignore 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)

Review Comment:
   Adjust them to be the same.



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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1638050122

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![83.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [83.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229979863


##########
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:
   I don't get the need for a `BooleanSupplier` here, can't we achieve the same with a plain `boolean`?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229973761


##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -20,27 +20,35 @@
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.mapping.IntPair;
 
 import org.immutables.value.Value;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.Set;
+import java.util.function.BooleanSupplier;
 import java.util.stream.Collectors;
 
 /**
  * Planner rule that matches an {@link Project}
- * on a {@link Join} and removes the join provided that the join is a left join
- * or right join and the join keys are unique.
+ * on a {@link Join} and removes the join provided that the join is a left or
+ * right join and the join keys are unique, and removes the join provided that
+ * the join is inner join and the join keys are foreign and unique key,
+ * foreign key is not nullable.

Review Comment:
   Nit: I am no native speaker so I don't have a strong opinion on this, but the following reads "better" to me, feel free to adopt or not:
   ```suggestion
    * and the foreign key is not nullable.
   ```



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229967418


##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java:
##########
@@ -505,6 +509,29 @@ public static RelMetadataQuery instance() {
     }
   }
 
+  /**
+   * Returns the
+   * {@link BuiltInMetadata.ForeignKeys#getForeignKeys(boolean)}
+   * statistic.
+   *
+   * @param rel         the relational expression
+   * @param ignoreNulls if true, ignore 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)
+   */
+  public ImmutableBitSet getForeignKeys(RelNode rel, boolean ignoreNulls) {
+    for (;;) {
+      try {
+        return foreignKeysHandler.getForeignKeys(rel, this, ignoreNulls);
+      } catch (MetadataHandlerProvider.NoHandler e) {
+        foreignKeysHandler =
+            revise(BuiltInMetadata.ForeignKeys.Handler.class);

Review Comment:
   No need to split the line I think:
   
   ```suggestion
           foreignKeysHandler = revise(BuiltInMetadata.ForeignKeys.Handler.class);
   ```



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229938527


##########
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;
+    }

Review Comment:
   ```suggestion
       if (referentialConstraints == null || referentialConstraints.isEmpty()) {
         return EMPTY_BIT_SET;
       }
   ```



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239142348


##########
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:
   OK, I get it. I will add some tests containing `SetOp`.



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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1606060814

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [4 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![91.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/90-16px.png '91.7%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [91.7% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![2.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '2.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [2.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1630336089

   > The `RelOptForeignKey` component has been added to represent the composite or single foreign key and the unique constraint that foreign key references. Sorry to bother you, @asolimando @julianhyde would you help me to review the code?
   
   Thanks @JingDas, I will review again as soon as I have some time. 
   
   In the meantime, I have approved the pending workflows, but I have also noticed that there are 18 code smells detected by SonarCloud, could you do a pass on this list and fix whatever can be reasonably fixed?
   
   As for any automatic tool not everything makes sense and there might even be false positives.


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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1628802537

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![84.2%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '84.2%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [84.2% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233328069


##########
testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java:
##########
@@ -99,6 +108,12 @@ private void registerTableEmpNullables(MockTable empNullablesTable, Fixture fixt
     empNullablesTable.addColumn("COMM", fixture.intTypeNull);
     empNullablesTable.addColumn("DEPTNO", fixture.intTypeNull);
     empNullablesTable.addColumn("SLACKER", fixture.booleanTypeNull);
+    empNullablesTable.setReferentialConstraints(
+        Lists.newArrayList(
+            RelReferentialConstraintImpl.of(
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "EMPNULLABLES"),
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "DEPT"),
+        Lists.newArrayList(IntPair.of(7, 0)))));

Review Comment:
   Yeah, add it.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230775481


##########
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:
   As the [discussion](https://github.com/apache/calcite/pull/3264#discussion_r1230251674) above, maybe we don't need to add a test where `ignoreNulls` equals `true`. What do you think?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229966534


##########
core/src/main/java/org/apache/calcite/rel/metadata/RelMetadataQuery.java:
##########
@@ -505,6 +509,29 @@ public static RelMetadataQuery instance() {
     }
   }
 
+  /**
+   * Returns the
+   * {@link BuiltInMetadata.ForeignKeys#getForeignKeys(boolean)}
+   * statistic.
+   *
+   * @param rel         the relational expression
+   * @param ignoreNulls if true, ignore 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)

Review Comment:
   In `BuiltInMetadata` you have a different javadoc for return, please check and align them



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592316371

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592316023

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592312433

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![88.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '88.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [88.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230215780


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.

Review Comment:
   @asolimando Thx for your detailed and comprehensive code review.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1607733084

   Adding "request review" from another committer given the comments in the associated Jira ticket.


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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1605161042

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![88.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '88.3%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [88.3% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233327411


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {

Review Comment:
   Add testForeignKeysFilter and testForeignKeysFilterEmpty 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: commits-unsubscribe@calcite.apache.org

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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229975086


##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -20,27 +20,35 @@
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.mapping.IntPair;
 
 import org.immutables.value.Value;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.Set;
+import java.util.function.BooleanSupplier;
 import java.util.stream.Collectors;
 
 /**
  * Planner rule that matches an {@link Project}
- * on a {@link Join} and removes the join provided that the join is a left join
- * or right join and the join keys are unique.
+ * on a {@link Join} and removes the join provided that the join is a left or
+ * right join and the join keys are unique, and removes the join provided that
+ * the join is inner join and the join keys are foreign and unique key,
+ * foreign key is not nullable.
  *
  * <p>For instance,

Review Comment:
   We could add another example showing your new feature here, I think it would help understand the textual description (same as the original example did for the original description)



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229999293


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {
+    sql("select name as dname, emp.deptno, dept.name from emp left outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysJoinRightOuter() {
+    sql("select name as dname, emp.deptno, dept.name, emp.deptno from dept right outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1, 3)));
+  }
+
+  @Test void testForeignKeysJoinOuterEmpty() {

Review Comment:
   ```suggestion
     @Test void testForeignKeysOuterJoinEmpty() {
   ```



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229998963


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {
+    sql("select name as dname, emp.deptno, dept.name from emp left outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysJoinRightOuter() {
+    sql("select name as dname, emp.deptno, dept.name, emp.deptno from dept right outer join emp"

Review Comment:
   Can we also add left/right outer join but with the opposite tables' order?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229997303


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {

Review Comment:
   ```suggestion
     @Test void testForeignKeysLeftOuterJoin() {
   ```



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230285149


##########
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) {
+      final List<RelDataTypeField> fieldList = rel.getRowType().getFieldList();
+      foreignKeys = foreignKeys.stream()
+          .filter(index -> !fieldList.get(index).getType().isNullable())
+          .collect(Collectors.toList());
+    }
+    return foreignKeys.isEmpty() ? EMPTY_BIT_SET : ImmutableBitSet.of(foreignKeys);
+  }
+
+  public ImmutableBitSet getForeignKeys(SetOp rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+
+    ImmutableBitSet foreignKeys = ImmutableBitSet.of();
+    for (RelNode input : rel.getInputs()) {
+      ImmutableBitSet inputForeignKeys = mq.getForeignKeys(input, ignoreNulls);
+      if (inputForeignKeys.isEmpty()) {
+        return EMPTY_BIT_SET;
+      }
+      foreignKeys = foreignKeys.isEmpty()
+          ? inputForeignKeys : foreignKeys.intersect(inputForeignKeys);
+    }
+    if (foreignKeys.isEmpty()) {

Review Comment:
   Yeah, it is also redundant somewhere else. I will fix them



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592324720

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239151614


##########
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:
   OK, I agree with you. Thanks for the suggestion.



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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1628857286

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![84.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '84.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [84.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1631768472

   > > The `RelOptForeignKey` component has been added to represent the composite or single foreign key and the unique constraint that foreign key references. Sorry to bother you, @asolimando @julianhyde would you help me to review the code?
   > 
   > Thanks @JingDas, I will review again as soon as I have some time.
   > 
   > In the meantime, I have approved the pending workflows, but I have also noticed that there are 18 code smells detected by SonarCloud, could you do a pass on this list and fix whatever can be reasonably fixed?
   > 
   > As for any automatic tool not everything makes sense and there might even be false positives.
   
   


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


[GitHub] [calcite] JingDas closed pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove by the foreign-unique constraint in the catalog

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas closed pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove by the foreign-unique constraint in the catalog
URL: https://github.com/apache/calcite/pull/3264


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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239146276


##########
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:
   I think you are right, I agree with you. 
   This newline operation is edited by check style.
   After run `./gradlew autostyleApply`, The `project` will go to the next line. 
   After I give it a try that moving the `project` in the same row, then run `./gradlew build`,
   it says that the following files have format violations.
   
   ```
   core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java
         @@ -148,14 +148,14 @@
          
              RelNode node;
              if (isLeftSideReserved) {
         -······node·=·project.copy(project.getTraitSet(),␊
         -··········join.getLeft(),·project.getProjects(),·project.getRowType());␊
         +······node·=␊
         +··········project.copy(project.getTraitSet(),·join.getLeft(),·project.getProjects(),·project.getRowType());␊
              } else {
                final List<RexNode> newExprs = project.getProjects().stream()
                    .map(expr -> RexUtil.shift(expr, -leftFieldsNum))
                    .collect(Collectors.toList());
         -······node·=·project.copy(project.getTraitSet(),␊
         -··········join.getRight(),·newExprs,·project.getRowType());␊
         +······node·=␊
         +··········project.copy(project.getTraitSet(),·join.getRight(),·newExprs,·project.getRowType());␊
              }
              call.transformTo(node);
            }
     Run './gradlew autostyleApply' to fix the violations.
   ```
   



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239146276


##########
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:
   I think you are right, I agree with you. 
   This newline operation is edited by check style.
   After run `./gradlew autostyleApply`, The `project` will go to the next line. 
   When I give it a try that moving the `project` in the same row. When run `./gradlew build`
   Then it says that the following files have format violations.
   `core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java
         @@ -148,14 +148,14 @@
          
              RelNode node;
              if (isLeftSideReserved) {
         -······node·=·project.copy(project.getTraitSet(),␊
         -··········join.getLeft(),·project.getProjects(),·project.getRowType());␊
         +······node·=␊
         +··········project.copy(project.getTraitSet(),·join.getLeft(),·project.getProjects(),·project.getRowType());␊
              } else {
                final List<RexNode> newExprs = project.getProjects().stream()
                    .map(expr -> RexUtil.shift(expr, -leftFieldsNum))
                    .collect(Collectors.toList());
         -······node·=·project.copy(project.getTraitSet(),␊
         -··········join.getRight(),·newExprs,·project.getRowType());␊
         +······node·=␊
         +··········project.copy(project.getTraitSet(),·join.getRight(),·newExprs,·project.getRowType());␊
              }
              call.transformTo(node);
            }
     Run './gradlew autostyleApply' to fix the violations.
   `



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229984348


##########
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);
+    final List<Integer> joinKeys = isLeftSideReserved.getAsBoolean() ? rightKeys : leftKeys;
     if (!Boolean.TRUE.equals(
-        mq.areColumnsUnique(isLeftJoin ? join.getRight() : join.getLeft(),
-        columns.build()))) {
+        mq.areColumnsUnique(isLeftSideReserved.getAsBoolean() ? join.getRight() : join.getLeft(),
+            ImmutableBitSet.of(joinKeys)))) {
       return;
     }
 
     RelNode node;
-    if (isLeftJoin) {
+    if (isLeftSideReserved.getAsBoolean()) {
       node = project
           .copy(project.getTraitSet(), join.getLeft(), project.getProjects(),
               project.getRowType());

Review Comment:
   Style check doesn't complain but it feels weird to indent like that fluent expressions with a single call, what about the following?
   
   ```suggestion
         node = project.copy(project.getTraitSet(), join.getLeft(), project.getProjects(), 
             project.getRowType());
   ```



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229976973


##########
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());

Review Comment:
   Please extract the common expressions like `join.getLeft().getRowType().getFieldCount()`, the lambda is already long enough



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229969431


##########
testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java:
##########
@@ -86,6 +89,12 @@ private void registerTableEmp(MockTable empTable, Fixture fixture) {
     empTable.addColumn("COMM", fixture.intType);
     empTable.addColumn("DEPTNO", fixture.intType);
     empTable.addColumn("SLACKER", fixture.booleanType);
+    empTable.setReferentialConstraints(
+        Lists.newArrayList(
+            RelReferentialConstraintImpl.of(
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "EMP"),
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "DEPT"),
+        Lists.newArrayList(IntPair.of(7, 0)))));

Review Comment:
   Might help to state what columns are linked for clarity (in a comment)



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229986113


##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -127,6 +151,49 @@ public ProjectJoinRemoveRule(
     call.transformTo(node);
   }
 
+  /**
+   * Check as following:
+   * 1. Make sure that every foreign column always be foreign key.

Review Comment:
   Again, no native speaker so take this with grain of salt, but this reads "better" to me:
   
   ```suggestion
      * 1. Make sure that every foreign column is always a foreign key.
   ```



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230533811


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {

Review Comment:
   It‘s exactly same, i will fix it.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230725524


##########
testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java:
##########
@@ -468,6 +483,9 @@ private void registerStructTypeTables(Fixture fixture) {
             "customBigInt"),
         typeFactory -> typeFactory.createSqlType(SqlTypeName.BIGINT));
 
+    // Register "DEPT" table.

Review Comment:
   For this case,
   my consideration is that we should need to know the referenced table and the unique key
   before creating the foreign key.
   It dosn't cause a exception when we register dept table after emp table in the test.
   Out of habit, I prefer to register the dept table first and then the emp table.
   It also feels OK to keep the previous registration order.



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1596177694

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [3 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![88.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '88.3%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [88.3% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233407546


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.

Review Comment:
   I am not a native speaker, Thanks for your comment grammar check.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229944269


##########
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:
   Looks the opposite to me, if `ignoreNulls` is true, then you want to filter them out (executing the body of the if), no?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230007500


##########
core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java:
##########
@@ -6019,6 +6019,87 @@ private HepProgram getTransitiveProgram() {
         .check();
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5756">[CALCITE-5756]
+   * Expand ProjectJoinRemoveRule to support inner join remove</a>.
+   * Similar to {@link #testProjectJoinRemove4()};
+   * Should remove the right join since the join key is referential constraints
+   * and the right is unique. */
+  @Test void testProjectJoinRemove11() {
+    final String sql = "SELECT e.deptno\n"
+        + "FROM sales.emp e\n"
+        + "INNER JOIN sales.dept d ON e.deptno = d.deptno";
+    sql(sql).withRule(CoreRules.PROJECT_JOIN_REMOVE)
+        .check();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5756">[CALCITE-5756]
+   * Expand ProjectJoinRemoveRule to support inner join remove</a>.
+   * Should not remove the right join since the join key is
+   * not referential constraints. */
+  @Test void testProjectJoinRemove12() {
+    final String sql = "SELECT e1.deptno\n"
+        + "FROM sales.emp e1\n"
+        + "INNER JOIN sales.emp e2 ON e1.deptno = e2.deptno";
+    sql(sql).withRule(CoreRules.PROJECT_JOIN_REMOVE)
+        .checkUnchanged();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5756">[CALCITE-5756]
+   * Expand ProjectJoinRemoveRule to support inner join remove</a>.
+   * Should not remove the right join since the project use columns in the right
+   * input of the join. */
+  @Test void testProjectJoinRemove13() {
+    final String sql = "SELECT e.deptno, d.name\n"
+        + "FROM sales.emp e\n"
+        + "INNER JOIN sales.dept d ON e.deptno = d.deptno";
+    sql(sql).withRule(CoreRules.PROJECT_JOIN_REMOVE)
+        .checkUnchanged();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5756">[CALCITE-5756]
+   * Expand ProjectJoinRemoveRule to support inner join remove</a>.
+   * The project references the last column of the left input.
+   * The rule should be fired and remove right.*/
+  @Test void testProjectJoinRemove14() {
+    final String sql = "SELECT e.deptno, e.slacker\n"
+        + "FROM sales.emp e\n"
+        + "INNER JOIN sales.dept d ON e.deptno = d.deptno";
+    sql(sql).withRule(CoreRules.PROJECT_JOIN_REMOVE)
+        .check();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5756">[CALCITE-5756]
+   * Expand ProjectJoinRemoveRule to support inner join remove</a>.
+   * Similar to {@link #testProjectJoinRemove11()};
+   * Should remove the left join since the join key of the left input is
+   * unique and the join key of the right input is foreign key, only use the right. */
+  @Test void testProjectJoinRemove15() {
+    final String sql = "SELECT e.slacker\n"
+        + "FROM sales.dept d\n"
+        + "INNER JOIN sales.emp e ON d.deptno = e.deptno";
+    sql(sql).withRule(CoreRules.PROJECT_JOIN_REMOVE)
+        .check();
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5756">[CALCITE-5756]
+   * Expand ProjectJoinRemoveRule to support inner join remove</a>.
+   * Similar to {@link #testProjectJoinRemove15()};
+   * Should not remove the left join, even the join key of the left input is

Review Comment:
   ```suggestion
      * Should not remove the left join, even if the join key of the left input is
   ```



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230003227


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {
+    sql("select deptno, ename from emp where ename = 'lucy'")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysInnerJoinLeft() {
+    sql("select emp.deptno, dept.name, emp.deptno from emp, dept")
+        .assertForeignKeys(equalTo(bitSetOf(0, 2)));
+  }
+
+  @Test void testForeignKeysInnerJoinRight() {
+    sql("select dept.name, emp.deptno, emp.deptno from dept, emp")
+        .assertForeignKeys(equalTo(bitSetOf(1, 2)));
+  }
+
+  @Test void testForeignKeysJoinLeftOuter() {
+    sql("select name as dname, emp.deptno, dept.name from emp left outer join dept"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysJoinRightOuter() {
+    sql("select name as dname, emp.deptno, dept.name, emp.deptno from dept right outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1, 3)));
+  }
+
+  @Test void testForeignKeysJoinOuterEmpty() {
+    sql("select name as dname, emp.deptno, dept.name from dept left outer join emp"
+        + " on emp.deptno = dept.deptno")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysJoinFullOuterEmpty() {

Review Comment:
   ```suggestion
     @Test void testForeignKeysFullOuterJoinEmpty() {
   ```



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229994979


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {

Review Comment:
   What's the difference with the test right below this one?



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


[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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1606057134

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![89.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '89.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [89.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1240502530


##########
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:
   OK, I would fall back to using `ignoreNulls`.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229981772


##########
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);
+    final List<Integer> joinKeys = isLeftSideReserved.getAsBoolean() ? rightKeys : leftKeys;
     if (!Boolean.TRUE.equals(

Review Comment:
   ```suggestion
       if (Boolean.FALSE.equals(
   ```



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592324452

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592312918

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230243310


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore null values when determining

Review Comment:
   I read some docs of Mysql and Oracle as following:  
   [Oracle foreign keys](https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/constraint.html#GUID-1055EA97-BA6F-4764-A15F-1024FD5B6DFE).  
   [Mysql foreign keys](https://dev.mysql.com/doc/mysql-reslimits-excerpt/5.7/en/ansi-diff-foreign-keys.html).  
   and found that foreign key may be null when the parent table(containg unique key) delete the unique key.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230290424


##########
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()

Review Comment:
   OK, got your point. 
   I think we can represent multi foreign keys in a  bit set if a table has multi referential constraints. 
   And retain the right index position. 
   For example:
   `SELECT ename, deptno, deptno
   FROM emp `
   where deptno is foreign key. 
   We should returen {1, 2} when call getForeignKeys method.
   



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


[GitHub] [calcite] asolimando commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1591788720

   Almost forgot: the commit message must match the title of the Jira ticket, at the moment they are different


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230228425


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.

Review Comment:
   You are right, the getForeignKeys method is similar to getUniqueKeys.   The result of getForeignKeys method represent the foreign keys pos above the current relNode, and the index is "0-based".   My comment was misunderstood, I'll edit it.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229958810


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

Review Comment:
   What if in the aggregation we have something other than grouping sets?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229992734


##########
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:
   It seems that we are not covering cases with `ignoreNulls` to `true`, can we add such tests?



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230290424


##########
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()

Review Comment:
   OK, got your point. 
   I think we can represent multi foreign keys in a  bit set if a table has multi referential constraints. 
   And retain the right index position. 
   For example:
   `SELECT ename, deptno, deptno
   FROM emp `
   where deptno is foreign key. 
   We should returen {1, 2} when call getForeignKeys method. 
   For the case above, getForeignKeys for Table Scan, `Set<IntPair>` is better. I will fix it.
   



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230256011


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore 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

Review Comment:
   I make a mistake. It should return empty ImmutableBitSet but never null If the current relNode(mainly underlying tableScan relNode correspondingly) does't contains foreign keys. I will fix it.



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592326305

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592326555

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233328045


##########
testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java:
##########
@@ -86,6 +89,12 @@ private void registerTableEmp(MockTable empTable, Fixture fixture) {
     empTable.addColumn("COMM", fixture.intType);
     empTable.addColumn("DEPTNO", fixture.intType);
     empTable.addColumn("SLACKER", fixture.booleanType);
+    empTable.setReferentialConstraints(
+        Lists.newArrayList(
+            RelReferentialConstraintImpl.of(
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "EMP"),
+        Lists.newArrayList(DEFAULT_CATALOG, DEFAULT_SCHEMA, "DEPT"),
+        Lists.newArrayList(IntPair.of(7, 0)))));

Review Comment:
   Add the comment.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233327287


##########
core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java:
##########
@@ -6019,6 +6019,87 @@ private HepProgram getTransitiveProgram() {
         .check();
   }
 

Review Comment:
   Add testProjectJoinRemove17 and testProjectJoinRemove18 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: commits-unsubscribe@calcite.apache.org

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


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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1637618169

   I some confusion about the failed results of the CI tests.
   CI failed test log as fllowing:
   ```
   org.apache.calcite.test.RelOptRulesTest > testProjectJoinRemove25() failure marker
   FAILURE   0.0sec, org.apache.calcite.test.RelOptRulesTest > testProjectJoinRemove25()
       java.lang.AssertionError: Expected planAfter must not be present when using unchanged=true or calling checkUnchanged.
           at org.apache.calcite.test.RelOptFixture.checkPlanning(RelOptFixture.java:397)
           at org.apache.calcite.test.RelOptFixture.check(RelOptFixture.java:330)
           at org.apache.calcite.test.RelOptFixture.checkUnchanged(RelOptFixture.java:322)
           at org.apache.calcite.test.RelOptRulesTest.testProjectJoinRemove25(RelOptRulesTest.java:6297)
   ```
   In RelOptFixture.java:397 it throw the exception. 
   But in RelOptRulesTes#testProjectJoinRemove25, I use checkUnchanged() method which means `unchanged` is true. It should run  into [RelOptFixture.java:395](https://github.com/JingDas/calcite/blob/4ef5adde44825f2069aad0f1bc3fbb3de6872693/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java#L392C1-L403C40).
   
   
   
   
   


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


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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1637967350

   > Aha! It seems calcite main code change, And the CI seems to use the new [main logic](https://github.com/apache/calcite/blob/3ab8003106507b1664ad2c42860789776a816cd3/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java#L394C1-L400C45). I will fix it according to main code usage.
   
   Have fixed it.


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


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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1631771189

   Sorry, it was closed by mistake,I reOpen it and fixed the most necessary SonarCloud detected code.


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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239146276


##########
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:
   I think you are right, I also agree with you. 
   This newline operation is edited by check style.
   I'll give it a try as we agreed.



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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1631708837

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![83.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [83.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1628840747

   The `RelOptForeignKey` component has been added to represent the composite or single foreign key and the unique constraint that foreign key references.
   Sorry to bother you, @asolimando @julianhyde  would you help me to review the code?
   


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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1656985684

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![83.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [83.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1596563758

   @JingDas thanks for addressing the comments, I am on a business trip at the moment, I should be able to review again by the end of this week.
   
   FYI: when you are done with comments and you want the reviewers to take another look you can also use the "request review" button close to the reviewers' name in the top-right part of the GitHub page


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229961703


##########
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()) {

Review Comment:
   `projectRight()` is true for (left?) anti joins too, would this be ok for them as well?
   
   If so, maybe we can make the comment more generic.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229996123


##########
core/src/test/java/org/apache/calcite/test/RelMetadataTest.java:
##########
@@ -273,6 +273,104 @@ final RelMetadataFixture sql(String sql) {
                 / (DEPT_SIZE + EMP_SIZE)));
   }
 
+  // ----------------------------------------------------------------------
+  // Tests for getForeignKeys
+  // ----------------------------------------------------------------------
+
+  @Test void testForeignKeysAggregateCallEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKeyEmpty() {
+    sql("select hiredate, sum(sal) from emp group by hiredate")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysAggregateKey() {
+    sql("select count(sal), deptno from emp group by deptno")
+        .assertForeignKeys(equalTo(bitSetOf(1)));
+  }
+
+  @Test void testForeignKeysTableOnly() {
+    sql("select deptno, ename from emp")
+        .assertForeignKeys(equalTo(bitSetOf(0)));
+  }
+
+  @Test void testForeignKeysTableOnlyEmpty() {
+    sql("select ename, job from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysExpressionEmpty() {
+    sql("select deptno + 1, ename from emp")
+        .assertForeignKeysAreEmpty();
+  }
+
+  @Test void testForeignKeysFilter() {

Review Comment:
   Maybe add a test using the FK column in the filter too?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229993251


##########
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) {

Review Comment:
   ```suggestion
     public RelMetadataFixture assertForeignKeys(Matcher<ImmutableBitSet> constraintMatcher) {
   ```



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230256886


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

Review Comment:
   Yeah, you are right. "RelMdReferentialConstraits" should be "RelMdForeignKeys". I will fix it.



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592313787

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![88.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '88.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [88.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592319139

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![88.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '88.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [88.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229952677


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

Review Comment:
   Isn't it equivalent to:
   ```suggestion
       return allOutColumns;
   ```
   ?



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229906537


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore null values when determining

Review Comment:
   Can you explain why we need this?
   
   Maybe I am bit rusty on FK relationships, but I guess that `unknownAsFalse` semantics will apply (like for `WHERE` clauses), so there is no way a row can "point" to a `NULL` value in the parent table.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229900002


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.

Review Comment:
   I am not sure I understand this sentence, is it something like "where each bit position represents the column ordinal for the foreign key" what you mean or similar?
   
   I mean, I get that the bit position is encoding the ordinal position of each foreign key, but I am not sure about the wording.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230016779


##########
core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java:
##########
@@ -6019,6 +6019,87 @@ private HepProgram getTransitiveProgram() {
         .check();
   }
 

Review Comment:
   Can we add some queries where the simplification happens having also some filters?



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


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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1637662500

   Aha! It seems calcite main code change, And the CI seems to use the new [main logic](https://github.com/apache/calcite/blob/3ab8003106507b1664ad2c42860789776a816cd3/testkit/src/main/java/org/apache/calcite/test/RelOptFixture.java#L394C1-L400C45). Should I try to rebase the calcite upstream main code and commit it again?


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229989016


##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -127,6 +151,49 @@ public ProjectJoinRemoveRule(
     call.transformTo(node);
   }
 
+  /**
+   * Check as following:
+   * 1. Make sure that every foreign column always be foreign key.
+   * 2. The target of foreign key is the correct unique key correspondingly.
+   */
+  private static boolean areForeignKeysValid(List<Integer> foreignColumns,
+      List<Integer> uniqueColumns,
+      ImmutableBitSet foreignKeys,
+      RelMetadataQuery mq,
+      RelNode foreignSideRel,
+      RelNode uniqueSideRel) {

Review Comment:
   As long as stylecheck passes I am OK, but I have rarely seen in Calcite formatting each parameter on a newline like that, I think the following could also work:
   
   ```suggestion
     private static boolean areForeignKeysValid(List<Integer> foreignColumns,
         List<Integer> uniqueColumns, ImmutableBitSet foreignKeys, RelMetadataQuery mq,
         RelNode foreignSideRel, RelNode uniqueSideRel) {
   ```
   
   Not necessary to change if nobody else complains and you like this style.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229986845


##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -127,6 +151,49 @@ public ProjectJoinRemoveRule(
     call.transformTo(node);
   }
 
+  /**
+   * Check as following:
+   * 1. Make sure that every foreign column always be foreign key.
+   * 2. The target of foreign key is the correct unique key correspondingly.

Review Comment:
   Same as per the other "linguistic" comments:
   
   ```suggestion
      * 2. The target of foreign key is the correct corresponding unique key.
   ```



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230733958


##########
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:
   This assertion is used in three places and `BooleanSupplier` is added primarily for reuse,
   maybe it's better. What do you think?



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230775481


##########
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:
   As the [discussion](https://github.com/apache/calcite/pull/3264#discussion_r1230251674) above, maybe we don't need to add a test where `ignoreNulls` equals `true`.



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592327226

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229936763


##########
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) {
+      final List<RelDataTypeField> fieldList = rel.getRowType().getFieldList();
+      foreignKeys = foreignKeys.stream()
+          .filter(index -> !fieldList.get(index).getType().isNullable())
+          .collect(Collectors.toList());
+    }
+    return foreignKeys.isEmpty() ? EMPTY_BIT_SET : ImmutableBitSet.of(foreignKeys);
+  }
+
+  public ImmutableBitSet getForeignKeys(SetOp rel, RelMetadataQuery mq,
+      boolean ignoreNulls) {
+
+    ImmutableBitSet foreignKeys = ImmutableBitSet.of();
+    for (RelNode input : rel.getInputs()) {
+      ImmutableBitSet inputForeignKeys = mq.getForeignKeys(input, ignoreNulls);
+      if (inputForeignKeys.isEmpty()) {
+        return EMPTY_BIT_SET;
+      }
+      foreignKeys = foreignKeys.isEmpty()
+          ? inputForeignKeys : foreignKeys.intersect(inputForeignKeys);
+    }
+    if (foreignKeys.isEmpty()) {

Review Comment:
   I think this is redundant, if `isEmpty()` returns true, no need to return an empty bitset



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230301200


##########
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 also consider it some times. Actually I get the idea from `UniqueKeys#(boolean ignoreNulls)`. 
   My understanding is that if `ignoreNulls ` is true, we allow result contains null. 
   The logical is consistent to `UniqueKeys#(boolean ignoreNulls)`. 
   Maybe it's better to use "containNulls". 
   If "containNulls" is true, we allow result contains null.
   If "containNulls" is false, we do not allow result contains null.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229907949


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore 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

Review Comment:
   Can you sketch an example for the "empty vs null" case 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.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230243310


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore null values when determining

Review Comment:
   I read some docs of Mysql and Oracle as following:  
   [Orcal foreign keys](https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/constraint.html#GUID-1055EA97-BA6F-4764-A15F-1024FD5B6DFE).  
   [Mysql foreign keys](https://dev.mysql.com/doc/mysql-reslimits-excerpt/5.7/en/ansi-diff-foreign-keys.html).  
   and found that foreign key may be null when the parent table(containg unique key) delete the unique key.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230251674


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore null values when determining

Review Comment:
   In our inner join removing scene. It will make a mistake when forein key is nullable. because after removing inner join side which contains unique keys, If foreign key is nullable, the result will contain some null foreign key result wrongly. 
   In other scene, it maybe wants foreign keys containg null. 
   And BuiltInMetadata.ForeignKeys is a common component, so i add "ignoreNulls"option for getForeignKeys method. 
   BuiltInMetadata.ForeignKeys can be used for different scene, and in our scene it should not contains null.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230251674


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.
+     *
+     * @param ignoreNulls if true, ignore null values when determining

Review Comment:
   In our inner join removing scene. It will make a mistake when forein key is nullable. because after removing inner join side which contains unique keys, If foreign key is nullable, the result will contain some null foreign key result wrongly. 
   In other scene, it maybe want foreign keys containg null. 
   And BuiltInMetadata.ForeignKeys is a common component, so i add "ignoreNulls"option for getForeignKeys method. 
   BuiltInMetadata.ForeignKeys can be used for different scene, and in our scene it should not contains null.



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1240522123


##########
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:
   OK, I will remove it.



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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1631783134

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![83.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '83.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [83.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229940799


##########
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()

Review Comment:
   If you really want unique, you have to remove duplicates (using a `Set<IntPair>` possibly).
   
   If you don't need unique, please update the field name not to confuse readers.



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239142348


##########
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:
   OK, I get it.



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


[GitHub] [calcite] JingDas 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

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1239144990


##########
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:
   You are right, I get it, I will fix it.



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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229931137


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

Review Comment:
   Apart from the typo, I don't see any class with this name, did you mean `RelMdForeignKeys` 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.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229900002


##########
core/src/main/java/org/apache/calcite/rel/metadata/BuiltInMetadata.java:
##########
@@ -105,6 +105,44 @@ interface Handler extends MetadataHandler<UniqueKeys> {
     }
   }
 
+  /**
+   * Metadata about which columns have foreign keys.
+   */
+  public interface ForeignKeys extends Metadata {
+    MetadataDef<ForeignKeys> DEF =
+        MetadataDef.of(ForeignKeys.class, ForeignKeys.Handler.class,
+            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.

Review Comment:
   I am not sure I understand this sentence, is it something like "where each bit position represents the column ordinal for the foreign key" what you mean or similar?
   
   I mean, I get that the bit position is encoding the ordinal position of each foreign key, but I am not sure about the wording.
   
   EDIT: by looking at the javadoc for uniquekey, we might also want to mention that indexing is "0-based" here too



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592314490

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![88.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '88.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [88.0% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592326427

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.9%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.9%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.9% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230522207


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

Review Comment:
   ForieignKeys info are available in a table which is often the bottom part fo a whole relNode.
   The foreign key may be invalid when propagate derivation from the bottom to up.
   I think that a foreign key propagate from the bottom to up in aggregate,
   if it is in the groupset, it can maintain the validity,
   if it is in the other position except groupset, it can't maintain the validity, such as count(), sum() aggregate function.



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230893813


##########
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()) {

Review Comment:
   ```
    public boolean projectsRight() {
       return this != SEMI && this != ANTI;
    }
   ```
   This code above is in `JoinRelType`.
   If we want to get right side projects from join, it can't be anti or semi join.
   So if `projectsRight` is true, we can get right project,
   if `projectsRight` is false, we can't get right projects, can only get left projects.
   Only semi join is mentioned in this comment, I will also add anti join to the comment.
   



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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1592311023

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [1 Code Smell](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


[GitHub] [calcite] sonarcloud[bot] commented on pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1590922653

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![87.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '87.8%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [87.8% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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


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

Posted by "asolimando (via GitHub)" <gi...@apache.org>.
asolimando commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1229970794


##########
testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderSimple.java:
##########
@@ -468,6 +483,9 @@ private void registerStructTypeTables(Fixture fixture) {
             "customBigInt"),
         typeFactory -> typeFactory.createSqlType(SqlTypeName.BIGINT));
 
+    // Register "DEPT" table.

Review Comment:
   Just out of curiosity, do you know why the `DEPT` table wasn't registered before and we need it now?



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1230301200


##########
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 also consider it some times. Actually I get the idea from `UniqueKeys#(boolean ignoreNulls)`. 
   My understanding is that if `ignoreNulls ` is true, we allow result contains null. 
   The logical is consistent to `UniqueKeys#(boolean ignoreNulls)`. 
   Maybe it's better to use "containNulls". 
   If "containNulls" is true, we allow result contains null.
   If "containNulls" is false, we do not allow result contains null.
   What do you think?



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


[GitHub] [calcite] JingDas commented on a diff in pull request #3264: [CALCITE-5756] Expand ProjectJoinRemoveRule to support inner join remove

Posted by "JingDas (via GitHub)" <gi...@apache.org>.
JingDas commented on code in PR #3264:
URL: https://github.com/apache/calcite/pull/3264#discussion_r1233327567


##########
core/src/main/java/org/apache/calcite/rel/rules/ProjectJoinRemoveRule.java:
##########
@@ -20,27 +20,35 @@
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelRule;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.mapping.IntPair;
 
 import org.immutables.value.Value;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.Set;
+import java.util.function.BooleanSupplier;
 import java.util.stream.Collectors;
 
 /**
  * Planner rule that matches an {@link Project}
- * on a {@link Join} and removes the join provided that the join is a left join
- * or right join and the join keys are unique.
+ * on a {@link Join} and removes the join provided that the join is a left or
+ * right join and the join keys are unique, and removes the join provided that
+ * the join is inner join and the join keys are foreign and unique key,
+ * foreign key is not nullable.
  *
  * <p>For instance,

Review Comment:
   Add "Another instance" comment.



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


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

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #3264:
URL: https://github.com/apache/calcite/pull/3264#issuecomment-1606063855

   Kudos, SonarCloud Quality Gate passed!&nbsp; &nbsp; [![Quality Gate passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/passed-16px.png 'Quality Gate passed')](https://sonarcloud.io/dashboard?id=apache_calcite&pullRequest=3264)
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_calcite&pullRequest=3264&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL) [2 Code Smells](https://sonarcloud.io/project/issues?id=apache_calcite&pullRequest=3264&resolved=false&types=CODE_SMELL)
   
   [![89.5%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '89.5%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list) [89.5% Coverage](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_calcite&pullRequest=3264&metric=new_duplicated_lines_density&view=list)
   
   


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