iwanttobepowerful commented on code in PR #4392:
URL: https://github.com/apache/calcite/pull/4392#discussion_r2115750737


##########
core/src/main/java/org/apache/calcite/rel/rules/JoinToHyperGraphRule.java:
##########
@@ -70,104 +75,160 @@ protected JoinToHyperGraphRule(Config config) {
       RelOptUtil.decomposeConjunction(origJoin.getCondition(), joinConds);
     }
 
-    // when right is HyperGraph, need shift the leftNodeBit, rightNodeBit, 
condition of HyperEdge
+    // when right is HyperGraph, need shift fields related to bitmap of 
HyperEdge
     int leftNodeCount;
-    int leftFieldCount = left.getRowType().getFieldCount();
     if (left instanceof HyperGraph && right instanceof HyperGraph) {
       leftNodeCount = left.getInputs().size();
       inputs.addAll(left.getInputs());
       inputs.addAll(right.getInputs());
 
-      edges.addAll(((HyperGraph) left).getEdges());
-      edges.addAll(
+      notProjectInputs |= ((HyperGraph) left).getNotProjectInputs();
+      notProjectInputs |= ((HyperGraph) right).getNotProjectInputs() << 
leftNodeCount;
+
+      leftSubEdges.addAll(((HyperGraph) left).getEdges());
+      rightSubEdges.addAll(
           ((HyperGraph) right).getEdges().stream()
-              .map(hyperEdge -> adjustNodeBit(hyperEdge, leftNodeCount, 
leftFieldCount))
+              .map(hyperEdge -> hyperEdge.adjustNodeBit(leftNodeCount))
               .collect(Collectors.toList()));
     } else if (left instanceof HyperGraph) {
       leftNodeCount = left.getInputs().size();
       inputs.addAll(left.getInputs());
       inputs.add(right);
 
-      edges.addAll(((HyperGraph) left).getEdges());
+      notProjectInputs |= ((HyperGraph) left).getNotProjectInputs();
+
+      leftSubEdges.addAll(((HyperGraph) left).getEdges());
     } else if (right instanceof HyperGraph) {
       leftNodeCount = 1;
       inputs.add(left);
       inputs.addAll(right.getInputs());
 
-      edges.addAll(
+      notProjectInputs |= ((HyperGraph) right).getNotProjectInputs() << 
leftNodeCount;
+
+      rightSubEdges.addAll(
           ((HyperGraph) right).getEdges().stream()
-              .map(hyperEdge -> adjustNodeBit(hyperEdge, leftNodeCount, 
leftFieldCount))
+              .map(hyperEdge -> hyperEdge.adjustNodeBit(leftNodeCount))
               .collect(Collectors.toList()));
     } else {
       leftNodeCount = 1;
       inputs.add(left);
       inputs.add(right);
     }
 
-    HashMap<Integer, Integer> fieldIndexToNodeIndexMap = new HashMap<>();
+    // calculate conflict rules
+    Map<Long, Long> conflictRules =
+        ConflictDetectionHelper.makeConflictRules(
+            leftSubEdges,
+            rightSubEdges,
+            origJoin.getJoinType());
+    leftSubEdges.addAll(rightSubEdges);
+
+    Map<Integer, Integer> fieldIndexToNodeIndexMap = new HashMap<>();
+    // the map from input index to the count of fields before it, used to 
convert RexInputRef to
+    // RexNodeAndFieldIndex
+    Map<Integer, Integer> relativePositionInNode = new HashMap<>();
     int fieldCount = 0;
     for (int i = 0; i < inputs.size(); i++) {
+      if (LongBitmap.isOverlap(notProjectInputs, LongBitmap.newBitmap(i))) {
+        continue;
+      }
+      relativePositionInNode.put(i, fieldCount);
       for (int j = 0; j < inputs.get(i).getRowType().getFieldCount(); j++) {
         fieldIndexToNodeIndexMap.put(fieldCount++, i);
       }
     }
     // convert current join condition to hyper edge condition
     for (RexNode joinCond : joinConds) {
-      long leftNodeBits;
-      long rightNodeBits;
+      long leftEndpoint;
+      long rightEndpoint;
+      long leftNodeUsedInPredicate;
+      long rightNodeUsedInPredicate;
+      long initialLeftNodeBits = LongBitmap.newBitmapBetween(0, leftNodeCount);
+      long initialRightNodeBits = LongBitmap.newBitmapBetween(leftNodeCount, 
inputs.size());
       List<Integer> leftRefs = new ArrayList<>();
       List<Integer> rightRefs = new ArrayList<>();
 
-      RexVisitorImpl visitor = new RexVisitorImpl<Void>(true) {
-        @Override public Void visitInputRef(RexInputRef inputRef) {
+      RexShuttle shuttle = new RexShuttle() {
+        @Override public RexNode visitInputRef(RexInputRef inputRef) {
           Integer nodeIndex = 
fieldIndexToNodeIndexMap.get(inputRef.getIndex());
           if (nodeIndex == null) {
-            throw new IllegalArgumentException("RexInputRef refers a dummy 
field: "
-                + inputRef + ", rowType is: " + origJoin.getRowType());
+            throw new AssertionError("When build hyper graph, RexInputRef 
refers "
+                + "a dummy field: " + inputRef + ", rowType is: " + 
origJoin.getRowType());
           }
           if (nodeIndex < leftNodeCount) {
             leftRefs.add(nodeIndex);
           } else {
             rightRefs.add(nodeIndex);
           }
-          return null;
+          Integer fieldOffset = relativePositionInNode.get(nodeIndex);
+          if (fieldOffset == null) {
+            throw new AssertionError("When build hyper graph, failed to map "
+                + "input index to field count before it");
+          }
+          int fieldIndex = inputRef.getIndex() - fieldOffset;
+          if (fieldIndex < 0) {
+            throw new AssertionError("When build hyper graph, failed to 
convert "
+                + "the input ref to the relative position of the field in the 
input");
+          }
+          return new HyperGraph.RexNodeAndFieldIndex(
+              nodeIndex,
+              fieldIndex,
+              inputRef.getName(),
+              inputRef.getType());
         }
       };
-      joinCond.accept(visitor);
+      RexNode hyperEdgeCondition = joinCond.accept(shuttle);
 
-      // when cartesian product, make it to complex hyper edge
+      Map<Long, Long> conflictRulesAfterAbsorb = new HashMap<>();
+      leftNodeUsedInPredicate = LongBitmap.newBitmapFromList(leftRefs);
+      rightNodeUsedInPredicate = LongBitmap.newBitmapFromList(rightRefs);
       if (leftRefs.isEmpty() || rightRefs.isEmpty()) {
-        leftNodeBits = LongBitmap.newBitmapBetween(0, leftNodeCount);
-        rightNodeBits = LongBitmap.newBitmapBetween(leftNodeCount, 
inputs.size());
+        // when cartesian product or degenerate predicate, a complex hyperedge 
is generated to fix
+        // current join operator without exploring more possibilities. See 
section 6.2 in CD-C paper
+        leftEndpoint = initialLeftNodeBits;
+        rightEndpoint = initialRightNodeBits;
       } else {
-        leftNodeBits = LongBitmap.newBitmapFromList(leftRefs);
-        rightNodeBits = LongBitmap.newBitmapFromList(rightRefs);
+        // simplify conflict rules. See section 5.5 in CD-C paper
+        long tes =
+            ConflictDetectionHelper.absorbConflictRulesIntoTES(
+                leftNodeUsedInPredicate | rightNodeUsedInPredicate,
+                conflictRulesAfterAbsorb,
+                conflictRules);
+        rightEndpoint = tes & initialRightNodeBits;
+        leftEndpoint = tes & ~rightEndpoint;
       }
-      edges.add(
+      leftSubEdges.add(
           new HyperEdge(
-              leftNodeBits,
-              rightNodeBits,
+              leftEndpoint,
+              rightEndpoint,
+              leftNodeUsedInPredicate,
+              rightNodeUsedInPredicate,
+              conflictRulesAfterAbsorb,
+              initialLeftNodeBits,
+              initialRightNodeBits,
               origJoin.getJoinType(),
-              joinCond));
+              hyperEdgeCondition));
+    }
+
+    if (!origJoin.getJoinType().projectsRight()) {
+      notProjectInputs |= LongBitmap.newBitmapBetween(leftNodeCount, 
inputs.size());
     }
     result =
         new HyperGraph(
             origJoin.getCluster(),
             origJoin.getTraitSet(),
             inputs,

Review Comment:
   
https://github.com/cockroachdb/cockroach/blob/475172846823a877e07eabb5e8da1a996e510d17/pkg/sql/opt/xform/join_order_builder.go#L438
   
   For any given non-inner join, exactly one edge is constructed.
   
   do we have same logic?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to