From e4788732021abdaac4fb008ee65a2a75e72d688d Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 18 Dec 2024 15:55:26 +0300 Subject: [PATCH 01/28] planner test works --- .../calcite/exec/LogicalRelImplementor.java | 20 + .../AbstractRightMaterializedJoinNode.java | 174 ++++ .../query/calcite/exec/rel/HashJoinNode.java | 759 ++++++++++++++++++ .../query/calcite/prepare/Cloner.java | 7 + .../calcite/prepare/IgniteRelShuttle.java | 6 + .../query/calcite/prepare/PlannerPhase.java | 2 + .../query/calcite/rel/IgniteHashJoin.java | 123 +++ .../query/calcite/rel/IgniteRelVisitor.java | 5 + .../calcite/rule/HashJoinConverterRule.java | 88 ++ .../CorrelatedNestedLoopJoinPlannerTest.java | 4 +- .../calcite/planner/HashJoinPlannerTest.java | 228 ++++++ .../calcite/planner/MergeJoinPlannerTest.java | 3 +- .../ignite/testsuites/PlannerTestSuite.java | 2 + 13 files changed, 1418 insertions(+), 3 deletions(-) create mode 100644 modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java create mode 100644 modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java create mode 100644 modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java create mode 100644 modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java create mode 100644 modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java index 30737feed7a78..a3c0d17a6b7d6 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.query.calcite.exec; import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -48,6 +49,7 @@ import org.apache.ignite.internal.processors.query.calcite.exec.rel.CorrelatedNestedLoopJoinNode; import org.apache.ignite.internal.processors.query.calcite.exec.rel.FilterNode; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateNode; +import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashJoinNode; import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox; import org.apache.ignite.internal.processors.query.calcite.exec.rel.IndexSpoolNode; import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectNode; @@ -73,6 +75,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashIndexSpool; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexBound; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexCount; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; @@ -265,6 +268,23 @@ public LogicalRelImplementor( return node; } + /** {@inheritDoc} */ + @Override public Node visit(IgniteHashJoin rel) { + RelDataType outType = rel.getRowType(); + RelDataType leftType = rel.getLeft().getRowType(); + RelDataType rightType = rel.getRight().getRowType(); + JoinRelType joinType = rel.getJoinType(); + + Node node = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, rel.analyzeCondition()); + + Node leftInput = visit(rel.getLeft()); + Node rightInput = visit(rel.getRight()); + + node.register(Arrays.asList(leftInput, rightInput)); + + return node; + } + /** {@inheritDoc} */ @Override public Node visit(IgniteCorrelatedNestedLoopJoin rel) { RelDataType outType = rel.getRowType(); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java new file mode 100644 index 0000000000000..1d7f403cfc8b1 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -0,0 +1,174 @@ +/* + * 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.ignite.internal.processors.query.calcite.exec.rel; + +import java.util.ArrayDeque; +import java.util.Deque; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; +import org.apache.ignite.internal.util.typedef.F; +import org.jetbrains.annotations.Nullable; + +/** Right-part materialized join node. Holds data from the right part locally. */ +public abstract class AbstractRightMaterializedJoinNode extends AbstractNode { + /** Special flag which marks that all the rows are received. */ + protected static final int NOT_WAITING = -1; + + /** */ + protected final Deque leftInBuf = new ArrayDeque<>(IN_BUFFER_SIZE); + + /** */ + protected boolean inLoop; + + /** */ + protected int requested; + + /** */ + protected int waitingLeft; + + /** */ + protected int waitingRight; + + /** */ + protected @Nullable Row left; + + /** */ + protected AbstractRightMaterializedJoinNode(ExecutionContext ctx, RelDataType rowType) { + super(ctx, rowType); + } + + /** */ + protected abstract void join() throws Exception; + + /** */ + protected abstract void pushRight(Row row) throws Exception; + + /** {@inheritDoc} */ + @Override public void request(int rowsCnt) throws Exception { + assert !F.isEmpty(sources()) && sources().size() == 2; + assert rowsCnt > 0 && requested == 0; + + checkState(); + + requested = rowsCnt; + + if (!inLoop) + context().execute(this::doJoin, this::onError); + } + + /** {@inheritDoc} */ + @Override protected void rewindInternal() { + requested = 0; + waitingLeft = 0; + waitingRight = 0; + left = null; + + leftInBuf.clear(); + } + + /** {@inheritDoc} */ + @Override protected Downstream requestDownstream(int idx) { + if (idx == 0) { + return new Downstream<>() { + @Override public void push(Row row) throws Exception { + pushLeft(row); + } + + @Override public void end() throws Exception { + endLeft(); + } + + @Override public void onError(Throwable e) { + AbstractRightMaterializedJoinNode.this.onError(e); + } + }; + } + else if (idx == 1) { + return new Downstream<>() { + @Override public void push(Row row) throws Exception { + pushRight(row); + } + + @Override public void end() throws Exception { + endRight(); + } + + @Override public void onError(Throwable e) { + AbstractRightMaterializedJoinNode.this.onError(e); + } + }; + } + + throw new IndexOutOfBoundsException(); + } + + /** */ + private void pushLeft(Row row) throws Exception { + assert downstream() != null; + assert waitingLeft > 0; + + checkState(); + + --waitingLeft; + + leftInBuf.add(row); + + join(); + } + + /** */ + private void endLeft() throws Exception { + assert downstream() != null; + assert waitingLeft > 0; + + checkState(); + + waitingLeft = NOT_WAITING; + + join(); + } + + /** */ + private void endRight() throws Exception { + assert downstream() != null; + assert waitingRight > 0; + + checkState(); + + waitingRight = NOT_WAITING; + + join(); + } + + /** */ + protected Node leftSource() { + return sources().get(0); + } + + /** */ + protected Node rightSource() { + return sources().get(1); + } + + /** */ + private void doJoin() throws Exception { + checkState(); + + join(); + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java new file mode 100644 index 0000000000000..ccb5730265d07 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -0,0 +1,759 @@ +/* + * 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.ignite.internal.processors.query.calcite.exec.rel; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Objects; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; +import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; +import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; + +/** HashJoin implementor. */ +public abstract class HashJoinNode extends AbstractRightMaterializedJoinNode { + /** */ + private static final int INITIAL_CAPACITY = 128; + + /** All keys with null-fields are mapped to this object. */ + private static final Key NULL_KEY = new Key(); + + /** */ + private final int[] leftJoinPositions; + + /** */ + private final int[] rightJoinPositions; + + /** Output row handler. */ + protected final RowHandler outRowHnd; + + /** */ + protected final Map> hashStore = new HashMap<>(INITIAL_CAPACITY); + + /** */ + protected Iterator rightIt = Collections.emptyIterator(); + + /** + * Creates hash join node. + * + * @param ctx Execution context. + * @param rowType Row type. + * @param info Join info. + * @param outRowHnd Output row handler. + */ + protected HashJoinNode(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { + super(ctx, rowType); + + leftJoinPositions = info.leftKeys.toIntArray(); + rightJoinPositions = info.rightKeys.toIntArray(); + + assert leftJoinPositions.length == rightJoinPositions.length; + + this.outRowHnd = outRowHnd; + } + + /** {@inheritDoc} */ + @Override protected void rewindInternal() { + rightIt = Collections.emptyIterator(); + + hashStore.clear(); + + super.rewindInternal(); + } + + /** Creates certain join node. */ + public static HashJoinNode create( + ExecutionContext ctx, + RelDataType outRowType, + RelDataType leftRowType, + RelDataType rightRowType, + JoinRelType type, + JoinInfo info + ) { + IgniteTypeFactory typeFactory = ctx.getTypeFactory(); + RowHandler rowHnd = ctx.rowHandler(); + + switch (type) { + case INNER: + return new InnerHashJoin<>(ctx, outRowType, info, rowHnd); + + case LEFT: + return new LeftHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType)); + + case RIGHT: + return new RightHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType)); + + case FULL: + return new FullOuterHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), + rowHnd.factory(typeFactory, rightRowType)); + + case SEMI: + return new SemiHashJoin<>(ctx, outRowType, info, rowHnd); + + case ANTI: + return new AntiHashJoin<>(ctx, outRowType, info, rowHnd); + + default: + throw new IllegalArgumentException("Join of type ''" + type + "' isn't supported yet."); + } + } + + /** */ + protected Collection lookup(Row row) { + Key row0 = extractKey(row, leftJoinPositions); + + // Key with null field can't be compared with other keys. + if (row0 == NULL_KEY) + return Collections.emptyList(); + + TouchedCollection found = hashStore.get(row0); + + if (found != null) { + found.touched = true; + + return found.items(); + } + + return Collections.emptyList(); + } + + /** */ + private static Iterator untouched(Map> entries) { + return new Iterator() { + private final Iterator> it = entries.values().iterator(); + + private Iterator innerIt = Collections.emptyIterator(); + + @Override public boolean hasNext() { + if (innerIt.hasNext()) + return true; + + advance(); + + return innerIt.hasNext(); + } + + @Override public RowT next() { + if (!hasNext()) + throw new NoSuchElementException(); + + return innerIt.next(); + } + + void advance() { + while (it.hasNext()) { + TouchedCollection coll = it.next(); + + if (!coll.touched && !coll.items().isEmpty()) { + innerIt = coll.items().iterator(); + + break; + } + } + } + }; + } + + /** {@inheritDoc} */ + @Override protected void pushRight(Row row) throws Exception { + assert downstream() != null; + assert waitingRight > 0; + + checkState(); + + waitingRight--; + + Key key = extractKey(row, rightJoinPositions); + + // No storing the row in hashStore, if it contains NULL. And we won't emit right part alone like in FULL OUTER and RIGHT joins. + if (keepRowsWithNull() || key != NULL_KEY) { + TouchedCollection raw = hashStore.computeIfAbsent(key, k -> new TouchedCollection<>()); + + raw.add(row); + } + + if (waitingRight == 0) + rightSource().request(waitingRight = IN_BUFFER_SIZE); + } + + /** */ + private Key extractKey(Row row, int[] mapping) { + RowHandler rowHnd = context().rowHandler(); + + for (int i : mapping) { + if (rowHnd.get(i, row) == null) + return NULL_KEY; + } + + return new RowWrapper<>(row, rowHnd, mapping); + } + + /** */ + protected void getMoreOrEnd() throws Exception { + if (waitingRight == 0) + rightSource().request(waitingRight = IN_BUFFER_SIZE); + + if (waitingLeft == 0 && leftInBuf.isEmpty()) + leftSource().request(waitingLeft = IN_BUFFER_SIZE); + + if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && leftInBuf.isEmpty() && left == null + && !rightIt.hasNext()) { + requested = 0; + + downstream().end(); + } + } + + /** + * Returns {@code true} if we need to store the row from right shoulder even if it contains NULL in any of join key position. + * + *

This is required for joins which emit unmatched part of the right shoulder, such as RIGHT JOIN and FULL OUTER JOIN. + * + * @return {@code true} when row must be stored in {@link #hashStore} unconditionally. + */ + protected boolean keepRowsWithNull() { + return false; + } + + /** */ + private static final class InnerHashJoin extends HashJoinNode { + /** + * Creates node for INNER JOIN. + * + * @param ctx Execution context. + * @param rowType Row type. + * @param info Join info. + * @param outRowHnd Output row factory. + */ + private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { + super(ctx, rowType, info, outRowHnd); + } + + /** {@inheritDoc} */ + @Override protected void join() throws Exception { + if (waitingRight == NOT_WAITING) { + inLoop = true; + + try { + while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + if (!rightIt.hasNext()) { + left = leftInBuf.remove(); + + rightIt = lookup(left).iterator(); + } + + while (rightIt.hasNext()) { + checkState(); + + RowT right = rightIt.next(); + + --requested; + + downstream().push(outRowHnd.concat(left, right)); + + if (requested == 0) + break; + } + + if (!rightIt.hasNext()) + left = null; + } + } + finally { + inLoop = false; + } + } + + getMoreOrEnd(); + } + } + + /** */ + private static final class LeftHashJoin extends HashJoinNode { + /** Right row factory. */ + private final RowHandler.RowFactory rightRowFactory; + + /** + * Creates node for LEFT OUTER JOIN. + * + * @param ctx Execution context. + * @param info Join info. + * @param rowType Row tyoe. + * @param outRowHnd Output row handler. + * @param rightRowFactory Right row factory. + */ + private LeftHashJoin( + ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + RowHandler.RowFactory rightRowFactory + ) { + super(ctx, rowType, info, outRowHnd); + + this.rightRowFactory = rightRowFactory; + } + + /** {@inheritDoc} */ + @Override protected void join() throws Exception { + if (waitingRight == NOT_WAITING) { + inLoop = true; + + try { + while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + checkState(); + + if (!rightIt.hasNext()) { + left = leftInBuf.remove(); + + Collection rightRows = lookup(left); + + if (rightRows.isEmpty()) { + requested--; + + downstream().push(outRowHnd.concat(left, rightRowFactory.create())); + } + + rightIt = rightRows.iterator(); + } + + while (rightIt.hasNext()) { + checkState(); + + RowT right = rightIt.next(); + + --requested; + + downstream().push(outRowHnd.concat(left, right)); + + if (requested == 0) + break; + } + + if (!rightIt.hasNext()) + left = null; + } + } + finally { + inLoop = false; + } + } + + getMoreOrEnd(); + } + } + + /** */ + private static final class RightHashJoin extends HashJoinNode { + /** Left row factory. */ + private final RowHandler.RowFactory leftRowFactory; + + /** + * Creates node for RIGHT OUTER JOIN. + * + * @param ctx Execution context. + * @param rowType Row type. + * @param info Join info. + * @param outRowHnd Output row factory. + * @param leftRowFactory Left row factory. + */ + private RightHashJoin( + ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + RowHandler.RowFactory leftRowFactory + ) { + super(ctx, rowType, info, outRowHnd); + + this.leftRowFactory = leftRowFactory; + } + + /** {@inheritDoc} */ + @Override protected void join() throws Exception { + if (waitingRight == NOT_WAITING) { + inLoop = true; + + try { + while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + checkState(); + + if (!rightIt.hasNext()) { + left = leftInBuf.remove(); + + rightIt = lookup(left).iterator(); + } + + while (rightIt.hasNext()) { + checkState(); + + RowT right = rightIt.next(); + + --requested; + + downstream().push(outRowHnd.concat(left, right)); + + if (requested == 0) + break; + } + + if (!rightIt.hasNext()) + left = null; + } + } finally { + inLoop = false; + } + } + + if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { + inLoop = true; + + try { + if (!rightIt.hasNext()) + rightIt = untouched(hashStore); + + RowT emptyLeft = leftRowFactory.create(); + + while (rightIt.hasNext()) { + checkState(); + + RowT right = rightIt.next(); + + RowT row = outRowHnd.concat(emptyLeft, right); + + --requested; + + downstream().push(row); + + if (requested == 0) + break; + } + } + finally { + inLoop = false; + } + } + + getMoreOrEnd(); + } + + /** {@inheritDoc} */ + @Override protected boolean keepRowsWithNull() { + return true; + } + } + + /** */ + private static class FullOuterHashJoin extends HashJoinNode { + /** Left row factory. */ + private final RowHandler.RowFactory leftRowFactory; + + /** Right row factory. */ + private final RowHandler.RowFactory rightRowFactory; + + /** + * Creates node for FULL OUTER JOIN. + * + * @param ctx Execution context. + * @param rowType Row type. + * @param info Join info. + * @param outRowHnd Output row handler. + * @param leftRowFactory Left row factory. + * @param rightRowFactory Right row factory. + */ + private FullOuterHashJoin( + ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + RowHandler.RowFactory leftRowFactory, + RowHandler.RowFactory rightRowFactory + ) { + super(ctx, rowType, info, outRowHnd); + + this.leftRowFactory = leftRowFactory; + this.rightRowFactory = rightRowFactory; + } + + /** {@inheritDoc} */ + @Override protected void join() throws Exception { + if (waitingRight == NOT_WAITING) { + inLoop = true; + + try { + while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + checkState(); + + if (!rightIt.hasNext()) { + left = leftInBuf.remove(); + + Collection rightRows = lookup(left); + + if (rightRows.isEmpty()) { + requested--; + + downstream().push(outRowHnd.concat(left, rightRowFactory.create())); + } + + rightIt = rightRows.iterator(); + } + + while (rightIt.hasNext()) { + checkState(); + + RowT right = rightIt.next(); + + --requested; + + downstream().push(outRowHnd.concat(left, right)); + + if (requested == 0) + break; + } + + if (!rightIt.hasNext()) + left = null; + + } + } + finally { + inLoop = false; + } + } + + if (left == null && !rightIt.hasNext() && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING + && requested > 0) { + inLoop = true; + + try { + if (!rightIt.hasNext()) + rightIt = untouched(hashStore); + + RowT emptyLeft = leftRowFactory.create(); + + while (rightIt.hasNext()) { + checkState(); + + RowT row = outRowHnd.concat(emptyLeft, rightIt.next()); + + --requested; + + downstream().push(row); + + if (requested == 0) + break; + } + } + finally { + inLoop = false; + } + } + + getMoreOrEnd(); + } + + /** {@inheritDoc} */ + @Override protected boolean keepRowsWithNull() { + return true; + } + } + + /** */ + private static final class SemiHashJoin extends HashJoinNode { + /** + * Creates node for SEMI JOIN operator. + * + * @param ctx Execution context. + * @param rowType Row type. + * @param info Join info. + * @param outRowHnd Output row handler. + */ + private SemiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { + super(ctx, rowType, info, outRowHnd); + } + + /** {@inheritDoc} */ + @Override protected void join() throws Exception { + if (waitingRight == NOT_WAITING) { + inLoop = true; + + try { + while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + checkState(); + + left = leftInBuf.remove(); + + Collection rightRows = lookup(left); + + if (!rightRows.isEmpty()) { + requested--; + + downstream().push(left); + + if (requested == 0) + break; + } + + left = null; + } + } + finally { + inLoop = false; + } + } + + getMoreOrEnd(); + } + } + + /** */ + private static final class AntiHashJoin extends HashJoinNode { + /** + * Creates node for ANTI JOIN. + * + * @param ctx Execution context. + * @param rowType Row type. + * @param info Join info. + * @param outRowHnd Output row handler. + */ + private AntiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { + super(ctx, rowType, info, outRowHnd); + } + + /** {@inheritDoc} */ + @Override protected void join() throws Exception { + if (waitingRight == NOT_WAITING) { + inLoop = true; + + try { + while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + checkState(); + + left = leftInBuf.remove(); + + Collection rightRows = lookup(left); + + if (rightRows.isEmpty()) { + requested--; + + downstream().push(left); + + if (requested == 0) + break; + } + + left = null; + } + } + finally { + inLoop = false; + } + } + + getMoreOrEnd(); + } + } + + /** Non-comparable key object. */ + private static class Key { + + } + + /** Comparable key object. */ + private static final class RowWrapper extends Key { + /** */ + private final RowT row; + + /** */ + private final RowHandler handler; + + /** */ + private final int[] items; + + /** */ + private RowWrapper(RowT row, RowHandler hnd, int[] items) { + this.row = row; + this.handler = hnd; + this.items = items; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int hashCode = 0; + + for (int i : items) + hashCode += Objects.hashCode(handler.get(i, row)); + + return hashCode; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + if (this == obj) + return true; + + if (obj == null || getClass() != obj.getClass()) + return false; + + RowWrapper row0 = (RowWrapper) obj; + + for (int i = 0; i < items.length; ++i) { + Object input = row0.handler.get(row0.items[i], row0.row); + Object cur = handler.get(items[i], row); + + boolean comp = Objects.equals(input, cur); + + if (!comp) + return comp; + } + + return true; + } + } + + /** */ + private static final class TouchedCollection { + /** */ + private final Collection coll; + + /** */ + private boolean touched; + + /** */ + private TouchedCollection() { + this.coll = new ArrayList<>(); + } + + /** */ + private void add(RowT row) { + coll.add(row); + } + + /** */ + private Collection items() { + return coll; + } + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java index 149645d0c4bec..0ee987ebafbd5 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashIndexSpool; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexBound; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexCount; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; @@ -144,6 +145,12 @@ private IgniteReceiver collect(IgniteReceiver receiver) { visit((IgniteRel)rel.getRight()))); } + /** {@inheritDoc} */ + @Override public IgniteRel visit(IgniteHashJoin rel) { + return rel.clone(cluster, F.asList(visit((IgniteRel)rel.getLeft()), + visit((IgniteRel)rel.getRight()))); + } + /** {@inheritDoc} */ @Override public IgniteRel visit(IgniteMergeJoin rel) { return rel.clone(cluster, F.asList(visit((IgniteRel)rel.getLeft()), diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java index a01198dcd9342..386091f2e57eb 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashIndexSpool; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexBound; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexCount; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; @@ -79,6 +80,11 @@ public class IgniteRelShuttle implements IgniteRelVisitor { return processNode(rel); } + /** {@inheritDoc} */ + @Override public IgniteRel visit(IgniteHashJoin rel) { + return processNode(rel); + } + /** {@inheritDoc} */ @Override public IgniteRel visit(IgniteCorrelatedNestedLoopJoin rel) { return processNode(rel); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java index 8e82d49404719..2edcc49953d26 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java @@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.query.calcite.rule.FilterSpoolMergeToHashIndexSpoolRule; import org.apache.ignite.internal.processors.query.calcite.rule.FilterSpoolMergeToSortedIndexSpoolRule; import org.apache.ignite.internal.processors.query.calcite.rule.HashAggregateConverterRule; +import org.apache.ignite.internal.processors.query.calcite.rule.HashJoinConverterRule; import org.apache.ignite.internal.processors.query.calcite.rule.IndexCountRule; import org.apache.ignite.internal.processors.query.calcite.rule.IndexMinMaxRule; import org.apache.ignite.internal.processors.query.calcite.rule.LogicalScanConverterRule; @@ -237,6 +238,7 @@ public enum PlannerPhase { CorrelatedNestedLoopJoinRule.INSTANCE, CorrelateToNestedLoopRule.INSTANCE, NestedLoopJoinConverterRule.INSTANCE, + HashJoinConverterRule.INSTANCE, ValuesConverterRule.INSTANCE, LogicalScanConverterRule.INDEX_SCAN, diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java new file mode 100644 index 0000000000000..d1a6a38bb0788 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java @@ -0,0 +1,123 @@ +/* + * 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.ignite.internal.processors.query.calcite.rel; + +import java.util.List; +import java.util.Set; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelInput; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.CorrelationId; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Util; +import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCost; +import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCostFactory; +import org.apache.ignite.internal.processors.query.calcite.util.Commons; + +/** Represent hash join. */ +public class IgniteHashJoin extends AbstractIgniteJoin { + /** */ + private static final String REL_TYPE = "HashJoin"; + + /** */ + public IgniteHashJoin( + RelOptCluster cluster, + RelTraitSet traitSet, + RelNode left, + RelNode right, + RexNode condition, + Set variablesSet, + JoinRelType joinType + ) { + super(cluster, traitSet, left, right, condition, variablesSet, joinType); + } + + /** Constructor. */ + public IgniteHashJoin(RelInput input) { + this( + input.getCluster(), + input.getTraitSet().replace(IgniteConvention.INSTANCE), + input.getInputs().get(0), + input.getInputs().get(1), + input.getExpression("condition"), + Set.copyOf(Commons.transform(input.getIntegerList("variablesSet"), CorrelationId::new)), + input.getEnum("joinType", JoinRelType.class) + ); + } + + /** {@inheritDoc} */ + @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) { + IgniteCostFactory costFactory = (IgniteCostFactory)planner.getCostFactory(); + + double leftRowCnt = mq.getRowCount(getLeft()); + + double rightRowCnt = mq.getRowCount(getRight()); + + if (Double.isInfinite(leftRowCnt) || Double.isInfinite(rightRowCnt)) + return planner.getCostFactory().makeInfiniteCost(); + + double rowCnt = leftRowCnt + rightRowCnt; + + int rightKeysSize = joinInfo.rightKeys.size(); + + double rightSize = rightRowCnt * IgniteCost.AVERAGE_FIELD_SIZE * getRight().getRowType().getFieldCount(); + + double distRightRows = Util.first(mq.getDistinctRowCount(right, ImmutableBitSet.of(joinInfo.rightKeys), null), 0.9 * rightRowCnt); + + rightSize += distRightRows * rightKeysSize * IgniteCost.AVERAGE_FIELD_SIZE; + + return costFactory.makeCost(rowCnt, rowCnt * IgniteCost.HASH_LOOKUP_COST, 0, rightSize, 0); + } + + /** {@inheritDoc} */ + @Override public Join copy( + RelTraitSet traitSet, + RexNode condition, + RelNode left, + RelNode right, + JoinRelType joinType, + boolean semiJoinDone + ) { + return new IgniteHashJoin(getCluster(), traitSet, left, right, condition, variablesSet, joinType); + } + + /** {@inheritDoc} */ + @Override public T accept(IgniteRelVisitor visitor) { + return visitor.visit(this); + } + + /** {@inheritDoc} */ + @Override public IgniteRel clone(RelOptCluster cluster, List inputs) { + return new IgniteHashJoin(cluster, getTraitSet(), inputs.get(0), inputs.get(1), getCondition(), + getVariablesSet(), getJoinType()); + } + + // TODO: needed? + /** {@inheritDoc} */ + @Override public String getRelTypeName() { + return REL_TYPE; + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java index ca9456e0b0bb6..9b8c157870289 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java @@ -54,6 +54,11 @@ public interface IgniteRelVisitor { */ T visit(IgniteNestedLoopJoin rel); + /** + * See {@link IgniteRelVisitor#visit(IgniteRel)}. + */ + T visit(IgniteHashJoin rel); + /** * See {@link IgniteRelVisitor#visit(IgniteRel)} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java new file mode 100644 index 0000000000000..835be136f253c --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java @@ -0,0 +1,88 @@ +/* + * 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.ignite.internal.processors.query.calcite.rule; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.PhysicalNode; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.util.Util; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; +import org.apache.ignite.internal.util.typedef.F; + +/** Hash join converter rule. */ +public class HashJoinConverterRule extends AbstractIgniteConverterRule { + /** */ + public static final RelOptRule INSTANCE = new HashJoinConverterRule(); + + /** Ctor. */ + public HashJoinConverterRule() { + super(LogicalJoin.class, "HashJoinConverter"); + } + + /** {@inheritDoc} */ + @Override public boolean matches(RelOptRuleCall call) { + LogicalJoin join = call.rel(0); + + return !F.isEmpty(join.analyzeCondition().pairs()) && join.analyzeCondition().isEqui() && checkConditions(join.getCondition()); + } + + /** */ + private static boolean checkConditions(RexNode node) { + RexVisitor v = new RexVisitorImpl<>(true) { + @Override public Void visitCall(RexCall call) { + if (call.getOperator().getKind() != SqlKind.EQUALS && call.getOperator().getKind() != SqlKind.AND) + throw Util.FoundOne.NULL; + + return super.visitCall(call); + } + }; + + try { + node.accept(v); + + return true; + } + catch (Util.FoundOne e) { + return false; + } + } + + /** {@inheritDoc} */ + @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, LogicalJoin rel) { + RelOptCluster cluster = rel.getCluster(); + RelTraitSet outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE); + RelTraitSet leftInTraits = cluster.traitSetOf(IgniteConvention.INSTANCE); + RelTraitSet rightInTraits = cluster.traitSetOf(IgniteConvention.INSTANCE); + RelNode left = convert(rel.getLeft(), leftInTraits); + RelNode right = convert(rel.getRight(), rightInTraits); + + return new IgniteHashJoin(cluster, outTraits, left, right, rel.getCondition(), rel.getVariablesSet(), rel.getJoinType()); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java index 06051ccb0ffa8..a4726a0dacc88 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java @@ -83,7 +83,7 @@ public void testValidIndexExpressions() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter" ); System.out.println("+++ " + RelOptUtil.toString(phys)); @@ -153,7 +153,7 @@ public void testInvalidIndexExpressions() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeRule" + "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeRule", "HashJoinConverter" ); assertNotNull(phys); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java new file mode 100644 index 0000000000000..4f72b072d00fc --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java @@ -0,0 +1,228 @@ +/* + * 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.ignite.internal.processors.query.calcite.planner; + +import java.util.List; +import org.apache.calcite.plan.RelOptPlanner.CannotPlanException; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.RelFieldCollation; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort; +import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; +import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; +import org.apache.ignite.internal.util.typedef.F; +import org.jetbrains.annotations.Nullable; +import org.junit.Test; + +import static org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; + +/** */ +public class HashJoinPlannerTest extends AbstractPlannerTest { + /** */ + private static final String[] DISABLED_RULES = {"NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", "MergeJoinConverter", + "JoinCommuteRule"}; + + /** */ + private static final String[] JOIN_TYPES = {"LEFT", "RIGHT", "INNER", "FULL OUTER"}; + + /** */ + private static int tableId = 1; + + /** */ + @Test + public void testHashJoinKeepsLeftCollation() throws Exception { + TestTable tbl1 = createSimpleTable(); + TestTable tbl2 = createComplexTable(); + + IgniteSchema schema = createSchema(tbl1, tbl2); + + String sql = "select t1.ID, t2.ID1 " + + "from TEST_TBL_CMPLX t2 " + + "join TEST_TBL t1 on t1.id = t2.id1 " + + "order by t2.ID1 NULLS LAST, t2.ID2 NULLS LAST"; + + RelNode plan = physicalPlan(sql, schema, DISABLED_RULES); + + assertEquals(0, findNodes(plan, byClass(IgniteSort.class)).size()); + assertEquals(1, findNodes(plan, byClass(IgniteHashJoin.class)).size()); + assertNotNull(findFirstNode(plan, byClass(IgniteHashJoin.class))); + } + + /** */ + @Test + public void testHashJoinErasesRightCollation() throws Exception { + TestTable tbl1 = createSimpleTable(); + TestTable tbl2 = createComplexTable(); + + IgniteSchema schema = createSchema(tbl1, tbl2); + + String sql = "select t1.ID, t2.ID1 " + + "from TEST_TBL t1 " + + "join TEST_TBL_CMPLX t2 on t1.id = t2.id1 " + + "order by t2.ID1 NULLS LAST, t2.ID2 NULLS LAST"; + + IgniteRel plan = physicalPlan(sql, schema, DISABLED_RULES); + + assertNotNull(findFirstNode(plan, byClass(IgniteHashJoin.class))); + assertNotNull(sortOnTopOfJoin(plan)); + } + + /** */ + @Test + public void testHashJoinWinsOnSkewedLeftInput() throws Exception { + TestTable thinTblSortedPk = createSimpleTable("SMALL_TBL", 1000); + TestTable thickTblSortedPk = createSimpleTable("LARGE_TBL", 500_000); + + IgniteSchema schema = createSchema(thinTblSortedPk, thickTblSortedPk); + + assertPlan( + "select t1.ID, t1.ID2, t2.ID, t2.ID2 from LARGE_TBL t1 join SMALL_TBL t2 on t1.ID2 = t2.ID2", + schema, + nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class)), + "JoinCommuteRule" + ); + + assertPlan( + "select t1.ID, t1.ID2, t2.ID, t2.ID2 from SMALL_TBL t1 join LARGE_TBL t2 on t1.ID2 = t2.ID2", + schema, + nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).negate()), + "JoinCommuteRule" + ); + + // Merge join can consume less cpu resources. + assertPlan( + "select t1.ID, t1.ID2, t2.ID, t2.ID2 from SMALL_TBL t1 join LARGE_TBL t2 on t1.ID = t2.ID", + schema, + nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).negate()), + "JoinCommuteRule" + ); + } + + /** */ + private static @Nullable IgniteSort sortOnTopOfJoin(IgniteRel root) { + List sortNodes = findNodes(root, byClass(IgniteSort.class) + .and(node -> node.getInputs().size() == 1 && node.getInput(0) instanceof Join)); + + if (sortNodes.size() > 1) + throw new IllegalStateException("Unexpected count of sort nodes: exp<=1, act=" + sortNodes.size()); + + return sortNodes.isEmpty() ? null : sortNodes.get(0); + } + + /** */ + @Test + public void testHashJoinApplied() throws Exception { + for (List paramSet : joinAppliedParameters()) { + assert paramSet != null && paramSet.size() == 2; + + String sql = (String)paramSet.get(0); + boolean canBePlanned = (Boolean)paramSet.get(1); + + TestTable tbl = createTable("T1", IgniteDistributions.single(), "ID", Integer.class, "C1", Integer.class); + + IgniteSchema schema = createSchema(tbl); + + for (String type : JOIN_TYPES) { + String sql0 = String.format(sql, type); + + if (canBePlanned) + assertPlan(sql0, schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class)), DISABLED_RULES); + else { + assertThrows(null, () -> physicalPlan(sql0, schema, DISABLED_RULES), CannotPlanException.class, + "There are not enough rules"); + } + } + } + } + + /** */ + private static List> joinAppliedParameters() { + return F.asList( + F.asList("select t1.c1 from t1 %s join t1 t2 using(c1)", true), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1", true), + F.asList("select t1.c1 from t1 %s join t1 t2 ON t1.id is not distinct from t2.c1", false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = OCTET_LENGTH('TEST')", false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = LOG10(t1.c1)", false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1 and t1.ID > t2.ID", false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1 and t2.c1 = 1", false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1", false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = ?", false) + ); + } + + /** */ + private static TestTable createSimpleTable() { + return createSimpleTable("TEST_TBL", DEFAULT_TBL_SIZE); + } + + /** */ + private static TestTable createSimpleTable(String name, int size) { + return createTable( + name, + size, + IgniteDistributions.affinity(0, ++tableId, 0), + "ID", Integer.class, + "ID2", Integer.class, + "VAL", String.class + ).addIndex( + RelCollations.of(new RelFieldCollation(0, ASCENDING, RelFieldCollation.NullDirection.LAST)), + "PK" + ); + } + + /** */ + private static TestTable createComplexTable() { + return createTable( + "TEST_TBL_CMPLX", + DEFAULT_TBL_SIZE, + IgniteDistributions.affinity(ImmutableIntList.of(0, 1), ++tableId, 0), + "ID1", Integer.class, + "ID2", Integer.class, + "VAL", String.class + ).addIndex( + RelCollations.of( + new RelFieldCollation(0, ASCENDING, RelFieldCollation.NullDirection.LAST), + new RelFieldCollation(1, ASCENDING, RelFieldCollation.NullDirection.LAST) + ), + "PK" + ); + } + +// /** */ +// static IgniteTable createSimpleTableHashPk(String tableName, int size) { +// return TestBuilders.table() +// .name(tableName) +// .size(size) +// .distribution(someAffinity()) +// .addColumn("ID", NativeTypes.INT32) +// .addColumn("ID2", NativeTypes.INT32) +// .addColumn("VAL", NativeTypes.STRING) +// .hashIndex() +// .name("PK") +// .addColumn("ID") +// .end() +// .build(); +// } + + +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/MergeJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/MergeJoinPlannerTest.java index 5cc79b4ef0523..fcffb24ca7938 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/MergeJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/MergeJoinPlannerTest.java @@ -41,7 +41,8 @@ public class MergeJoinPlannerTest extends AbstractPlannerTest { "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", "FilterSpoolMergeRule", - "JoinCommuteRule" + "JoinCommuteRule", + "HashJoinConverter" }; /** diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java index 53ff68e605e69..bc669842709d6 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/PlannerTestSuite.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.query.calcite.planner.DataTypesPlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.HashAggregatePlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.HashIndexSpoolPlannerTest; +import org.apache.ignite.internal.processors.query.calcite.planner.HashJoinPlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.IndexRebuildPlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.IndexSearchBoundsPlannerTest; import org.apache.ignite.internal.processors.query.calcite.planner.InlineIndexScanPlannerTest; @@ -72,6 +73,7 @@ JoinCommutePlannerTest.class, LimitOffsetPlannerTest.class, MergeJoinPlannerTest.class, + HashJoinPlannerTest.class, StatisticsPlannerTest.class, CorrelatedSubqueryPlannerTest.class, JoinWithUsingPlannerTest.class, From 2d435024ba79f1a4cef7e3c9f0482ea901e46031 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 23 Dec 2024 17:19:10 +0300 Subject: [PATCH 02/28] fixed tests. + memory tracking --- docs/_docs/SQL/sql-calcite.adoc | 8 +- .../AbstractRightMaterializedJoinNode.java | 2 +- .../query/calcite/exec/rel/HashJoinNode.java | 7 +- .../calcite/exec/rel/NestedLoopJoinNode.java | 167 +----------------- .../query/calcite/hint/HintDefinition.java | 26 +++ .../query/calcite/rel/IgniteHashJoin.java | 9 - .../rule/AbstractIgniteJoinConverterRule.java | 3 + .../calcite/rule/HashJoinConverterRule.java | 7 +- .../calcite/CalciteQueryProcessorTest.java | 3 +- .../query/calcite/exec/rel/ExecutionTest.java | 125 ++++++++++++- ...iteBasicSecondaryIndexIntegrationTest.java | 1 + .../integration/JoinIntegrationTest.java | 17 +- .../MemoryQuotasIntegrationTest.java | 22 ++- .../CorrelatedNestedLoopJoinPlannerTest.java | 2 +- .../planner/HashIndexSpoolPlannerTest.java | 6 +- .../calcite/planner/HashJoinPlannerTest.java | 40 ++--- .../planner/JoinColocationPlannerTest.java | 72 ++++---- .../planner/JoinCommutePlannerTest.java | 27 ++- .../planner/SortAggregatePlannerTest.java | 1 + .../planner/SortedIndexSpoolPlannerTest.java | 6 +- .../hints/JoinTypeHintPlannerTest.java | 80 ++++++--- .../src/test/sql/sqlite/join/join1.test | 8 +- 22 files changed, 338 insertions(+), 301 deletions(-) diff --git a/docs/_docs/SQL/sql-calcite.adoc b/docs/_docs/SQL/sql-calcite.adoc index 2e3ddc766e8b2..19311e0a603f3 100644 --- a/docs/_docs/SQL/sql-calcite.adoc +++ b/docs/_docs/SQL/sql-calcite.adoc @@ -371,10 +371,10 @@ SELECT /*+ ENFORCE_JOIN_ORDER */ T1.V1, T2.V1, T2.V2, T3.V1, T3.V2, T3.V3 FROM T SELECT t1.v1, t3.v2 FROM TBL1 t1 JOIN TBL3 t3 on t1.v3=t3.v3 WHERE t1.v2 in (SELECT /*+ ENFORCE_JOIN_ORDER */ t2.v2 FROM TBL2 t2 JOIN TBL3 t3 ON t2.v1=t3.v1) ---- -==== MERGE_JOIN, NL_JOIN, CNL_JOIN -Forces certain join type: Merge, Nested Loop and Correlated Nested Loop respectively. +==== MERGE_JOIN, NL_JOIN, CNL_JOIN, HASH_JOIN +Forces certain join type: Merge, Nested Loop, Correlated Nested Loop and Hash Join respectively. -Every of those has the negation like 'NO_INDEX': CNL_JOIN, NO_CNL_JOIN. The negation hint disables certain join type. +Every of those has the negation like 'NO_INDEX': CNL_JOIN, NO_CNL_JOIN, NO_HASH_JOIN. The negation hint disables certain join type. ===== Parameters: * Empty. To force or disable certain join type for every join. @@ -389,7 +389,7 @@ SELECT /*+ NL_JOIN(TBL3,TBL1) */ t4.v1, t2.v2 FROM TBL1 t4 JOIN TBL2 t2 on t1.v3 SELECT t1.v1, t2.v2 FROM TBL2 t1 JOIN TBL1 t2 on t1.v3=t2.v3 WHERE t2.v3 in (SELECT /*+ NO_CNL_JOIN(TBL4) */ t3.v3 FROM TBL3 t3 JOIN TBL4 t4 on t3.v1=t4.v1) -SELECT t4.v1, t2.v2 FROM TBL1 t4 JOIN TBL2 t2 on t1.v3=t2.v3 WHERE t2.v1 in (SELECT t3.v3 FROM TBL3 t3 JOIN TBL1 /*+ NL_JOIN */ t4 on t3.v2=t4.v2) +SELECT t4.v1, t2.v2 FROM TBL1 t4 JOIN TBL2 t2 on t1.v3=t2.v3 WHERE t2.v1 in (SELECT t3.v3 FROM TBL3 t3 JOIN TBL1 /*+ HASH_JOIN */ t4 on t3.v2=t4.v2) ---- ==== EXPAND_DISTINCT_AGG diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java index 1d7f403cfc8b1..5989e099869f9 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -25,7 +25,7 @@ import org.jetbrains.annotations.Nullable; /** Right-part materialized join node. Holds data from the right part locally. */ -public abstract class AbstractRightMaterializedJoinNode extends AbstractNode { +public abstract class AbstractRightMaterializedJoinNode extends MemoryTrackingNode { /** Special flag which marks that all the rows are received. */ protected static final int NOT_WAITING = -1; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index ccb5730265d07..df5ae17e8bde6 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -183,6 +183,8 @@ void advance() { checkState(); + nodeMemoryTracker.onRowAdded(row); + waitingRight--; Key key = extractKey(row, rightJoinPositions); @@ -422,7 +424,8 @@ private RightHashJoin( if (!rightIt.hasNext()) left = null; } - } finally { + } + finally { inLoop = false; } } @@ -717,7 +720,7 @@ private RowWrapper(RowT row, RowHandler hnd, int[] items) { if (obj == null || getClass() != obj.getClass()) return false; - RowWrapper row0 = (RowWrapper) obj; + RowWrapper row0 = (RowWrapper)obj; for (int i = 0; i < items.length; ++i) { Object input = row0.handler.get(row0.items[i], row0.row); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index 581b7366b1ee0..2ef56141cd259 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -17,48 +17,27 @@ package org.apache.ignite.internal.processors.query.calcite.exec.rel; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.BitSet; -import java.util.Deque; import java.util.List; import java.util.function.BiPredicate; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; -import org.apache.ignite.internal.util.typedef.F; import org.jetbrains.annotations.NotNull; /** */ -public abstract class NestedLoopJoinNode extends MemoryTrackingNode { - /** Special value to highlights that all row were received and we are not waiting any more. */ - protected static final int NOT_WAITING = -1; - +public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJoinNode { /** */ protected final BiPredicate cond; /** */ protected final RowHandler handler; - /** */ - protected int requested; - - /** */ - protected int waitingLeft; - - /** */ - protected int waitingRight; - /** */ protected final List rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE); - /** */ - protected final Deque leftInBuf = new ArrayDeque<>(IN_BUFFER_SIZE); - - /** */ - protected boolean inLoop; - /** * @param ctx Execution context. * @param cond Join expression. @@ -70,144 +49,30 @@ private NestedLoopJoinNode(ExecutionContext ctx, RelDataType rowType, BiPre handler = ctx.rowHandler(); } - /** {@inheritDoc} */ - @Override public void request(int rowsCnt) throws Exception { - assert !F.isEmpty(sources()) && sources().size() == 2; - assert rowsCnt > 0 && requested == 0; - - checkState(); - - requested = rowsCnt; - - if (!inLoop) - context().execute(this::doJoin, this::onError); - } - - /** */ - private void doJoin() throws Exception { - checkState(); - - join(); - } - /** {@inheritDoc} */ @Override protected void rewindInternal() { - requested = 0; - waitingLeft = 0; - waitingRight = 0; - rightMaterialized.clear(); - leftInBuf.clear(); - } - /** {@inheritDoc} */ - @Override protected Downstream requestDownstream(int idx) { - if (idx == 0) - return new Downstream() { - /** {@inheritDoc} */ - @Override public void push(Row row) throws Exception { - pushLeft(row); - } - - /** {@inheritDoc} */ - @Override public void end() throws Exception { - endLeft(); - } - - /** {@inheritDoc} */ - @Override public void onError(Throwable e) { - NestedLoopJoinNode.this.onError(e); - } - }; - else if (idx == 1) - return new Downstream() { - /** {@inheritDoc} */ - @Override public void push(Row row) throws Exception { - pushRight(row); - } - - /** {@inheritDoc} */ - @Override public void end() throws Exception { - endRight(); - } - - /** {@inheritDoc} */ - @Override public void onError(Throwable e) { - NestedLoopJoinNode.this.onError(e); - } - }; - - throw new IndexOutOfBoundsException(); + super.rewindInternal(); } - /** */ - private void pushLeft(Row row) throws Exception { - assert downstream() != null; - assert waitingLeft > 0; - - checkState(); - - waitingLeft--; - - leftInBuf.add(row); - - join(); - } - - /** */ - private void pushRight(Row row) throws Exception { + /** {@inheritDoc} */ + @Override protected void pushRight(Row row) throws Exception { assert downstream() != null; assert waitingRight > 0; checkState(); + nodeMemoryTracker.onRowAdded(row); + waitingRight--; rightMaterialized.add(row); - nodeMemoryTracker.onRowAdded(row); - if (waitingRight == 0) rightSource().request(waitingRight = IN_BUFFER_SIZE); } - /** */ - private void endLeft() throws Exception { - assert downstream() != null; - assert waitingLeft > 0; - - checkState(); - - waitingLeft = NOT_WAITING; - - join(); - } - - /** */ - private void endRight() throws Exception { - assert downstream() != null; - assert waitingRight > 0; - - checkState(); - - waitingRight = NOT_WAITING; - - join(); - } - - /** */ - protected Node leftSource() { - return sources().get(0); - } - - /** */ - protected Node rightSource() { - return sources().get(1); - } - - /** */ - protected abstract void join() throws Exception; - /** */ @NotNull public static NestedLoopJoinNode create(ExecutionContext ctx, RelDataType outputRowType, RelDataType leftRowType, RelDataType rightRowType, JoinRelType joinType, BiPredicate cond) { @@ -247,9 +112,6 @@ protected Node rightSource() { /** */ private static class InnerJoin extends NestedLoopJoinNode { - /** */ - private Row left; - /** */ private int rightIdx; @@ -263,7 +125,6 @@ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate extends NestedLoopJoinNode { /** Whether current left row was matched or not. */ private boolean matched; - /** */ - private Row left; - /** */ private int rightIdx; @@ -345,7 +203,6 @@ public LeftJoin( /** */ @Override protected void rewindInternal() { matched = false; - left = null; rightIdx = 0; super.rewindInternal(); @@ -422,9 +279,6 @@ private static class RightJoin extends NestedLoopJoinNode { /** */ private int lastPushedInd; - /** */ - private Row left; - /** */ private int rightIdx; @@ -445,7 +299,6 @@ public RightJoin( /** {@inheritDoc} */ @Override protected void rewindInternal() { - left = null; rightNotMatchedIndexes.clear(); lastPushedInd = 0; rightIdx = 0; @@ -689,9 +542,6 @@ public FullOuterJoin( /** */ private static class SemiJoin extends NestedLoopJoinNode { - /** */ - private Row left; - /** */ private int rightIdx; @@ -705,7 +555,6 @@ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate extends NestedLoopJoinNode { - /** */ - private Row left; - /** */ private int rightIdx; @@ -771,7 +617,6 @@ public AntiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate { +public class HashJoinConverterRule extends AbstractIgniteJoinConverterRule { /** */ public static final RelOptRule INSTANCE = new HashJoinConverterRule(); /** Ctor. */ public HashJoinConverterRule() { - super(LogicalJoin.class, "HashJoinConverter"); + super("HashJoinConverter", HintDefinition.HASH_JOIN); } /** {@inheritDoc} */ - @Override public boolean matches(RelOptRuleCall call) { + @Override public boolean matchesJoin(RelOptRuleCall call) { LogicalJoin join = call.rel(0); return !F.isEmpty(join.analyzeCondition().pairs()) && join.analyzeCondition().isEqui() && checkConditions(join.getCondition()); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java index 9493fcb970159..5c54c4cad4252 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java @@ -217,7 +217,8 @@ public void testCountWithJoin() throws Exception { awaitPartitionMapExchange(true, true, null); - List joinConverters = Arrays.asList("CorrelatedNestedLoopJoin", "MergeJoinConverter", "NestedLoopJoinConverter"); + List joinConverters = Arrays.asList("CorrelatedNestedLoopJoin", "MergeJoinConverter", "NestedLoopJoinConverter", + "HashJoinConverter"); // CorrelatedNestedLoopJoin skipped intentionally since it takes too long to finish // the query with only CNLJ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java index 6ca45427497c8..fc8cf3dc5c1ff 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java @@ -23,8 +23,10 @@ import java.util.UUID; import com.google.common.collect.ImmutableSet; import org.apache.calcite.rel.core.CorrelationId; +import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; @@ -289,10 +291,7 @@ public void testRightJoin() { assertEquals(4, rows.size()); - Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, rows.get(2)); - Assert.assertArrayEquals(new Object[] {2, "Ivan", null}, rows.get(3)); + checkDepJoinEmpResults(RIGHT, rows); } /** @@ -409,10 +408,7 @@ public void testSemiJoin() { while (node.hasNext()) rows.add(node.next()); - assertEquals(2, rows.size()); - - Assert.assertArrayEquals(new Object[] {"Core"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {"SQL"}, rows.get(1)); + checkDepJoinEmpResults(SEMI, rows); } /** @@ -469,7 +465,7 @@ public void testAntiJoin() { assertEquals(1, rows.size()); - Assert.assertArrayEquals(new Object[] {"QA"}, rows.get(0)); + checkDepJoinEmpResults(ANTI, rows); } /** @@ -538,6 +534,117 @@ public void testCorrelatedNestedLoopJoin() { } } + /** */ + @Test + public void testHashJoin() { + // select e.id, e.name, d.name as dep_name from DEP d join EMP e on e.DEPNO = d.DEPNO + for (JoinRelType joinType : F.asList(LEFT, INNER, RIGHT, FULL, SEMI, ANTI)) { + ExecutionContext ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0); + IgniteTypeFactory tf = ctx.getTypeFactory(); + + RelDataType rowType = TypeUtils.createRowType(tf, int.class, String.class); + ScanNode leftDeps = new ScanNode<>(ctx, rowType, Arrays.asList( + new Object[] {1, "Core"}, + new Object[] {2, "SQL"}, + new Object[] {3, "QA"} + )); + + rowType = TypeUtils.createRowType(tf, int.class, String.class, Integer.class); + ScanNode rightEmps = new ScanNode<>(ctx, rowType, Arrays.asList( + new Object[] {0, "Igor", 1}, + new Object[] {1, "Roman", 2}, + new Object[] {2, "Ivan", null}, + new Object[] {3, "Alexey", 1} + )); + + RelDataType leftType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, String.class); + RelDataType rightType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, String.class, Integer.class); + RelDataType outType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, String.class, int.class, + String.class, Integer.class); + + rowType = TypeUtils.createRowType(tf, int.class, String.class, String.class); + + HashJoinNode join = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, + JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(2))); + + join.register(F.asList(leftDeps, rightEmps)); + + ProjectNode project = new ProjectNode<>(ctx, rowType, + r -> joinType == SEMI || joinType == ANTI ? new Object[] {r[1]} : new Object[] {r[2], r[3], r[1]}); + + project.register(join); + + RootNode rootScan = new RootNode<>(ctx, rowType); + rootScan.register(project); + + assert rootScan.hasNext(); + + ArrayList rows = new ArrayList<>(); + + while (rootScan.hasNext()) + rows.add(rootScan.next()); + + checkDepJoinEmpResults(joinType, rows); + } + } + + /** */ + private void checkDepJoinEmpResults(JoinRelType joinType, ArrayList results) { + switch (joinType) { + case LEFT: + assertEquals(4, results.size()); + + Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); + Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); + Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); + Assert.assertArrayEquals(new Object[] {null, null, "QA"}, results.get(3)); + break; + + case INNER: + assertEquals(3, results.size()); + + Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); + Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); + Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); + break; + + case RIGHT: + assertEquals(4, results.size()); + + Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); + Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); + Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); + Assert.assertArrayEquals(new Object[] {2, "Ivan", null}, results.get(3)); + break; + + case FULL: + assertEquals(5, results.size()); + + Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); + Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); + Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); + Assert.assertArrayEquals(new Object[] {null, null, "QA"}, results.get(3)); + Assert.assertArrayEquals(new Object[] {2, "Ivan", null}, results.get(4)); + break; + + case SEMI: + assertEquals(2, results.size()); + + Assert.assertArrayEquals(new Object[] {"Core"}, results.get(0)); + Assert.assertArrayEquals(new Object[] {"SQL"}, results.get(1)); + break; + + case ANTI: + assertEquals(1, results.size()); + + Assert.assertArrayEquals(new Object[] {"QA"}, results.get(0)); + break; + + default: + throw new IllegalArgumentException("Unknows join type."); + } + } + /** */ @Test public void testMergeJoin() throws IgniteCheckedException { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java index 281463152981a..9c0ead688e297 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java @@ -276,6 +276,7 @@ public void testIndexLoopJoin() { .check(); } + // TODO: hash join /** */ @Test public void testMergeJoin() { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java index 603a7123b363b..b367506f9e285 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java @@ -842,21 +842,32 @@ enum JoinType { NESTED_LOOP( "CorrelatedNestedLoopJoin", "JoinCommuteRule", - "MergeJoinConverter" + "MergeJoinConverter", + "HashJoinConverter" ), /** */ MERGE( "CorrelatedNestedLoopJoin", "JoinCommuteRule", - "NestedLoopJoinConverter" + "NestedLoopJoinConverter", + "HashJoinConverter" ), /** */ CORRELATED( "MergeJoinConverter", "JoinCommuteRule", - "NestedLoopJoinConverter" + "NestedLoopJoinConverter", + "HashJoinConverter" + ), + + /** */ + HASH( + "MergeJoinConverter", + "JoinCommuteRule", + "NestedLoopJoinConverter", + "CorrelatedNestedLoopJoin" ); /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java index 99cd26fd9c2fb..f3c5a1f32ad24 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java @@ -17,12 +17,14 @@ package org.apache.ignite.internal.processors.query.calcite.integration; +import java.util.List; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.calcite.CalciteQueryEngineConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.SqlConfiguration; import org.apache.ignite.internal.processors.query.calcite.QueryChecker; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Test; @@ -234,22 +236,28 @@ public void testTableSpoolNode() { /** */ @Test - public void testNestedLoopJoinNode() { + public void testJoins() { sql("CREATE TABLE tbl2 (id INT, b VARBINARY) WITH TEMPLATE=PARTITIONED"); for (int i = 0; i < 800; i++) sql("INSERT INTO tbl2 VALUES (?, ?)", i, new byte[1000]); - assertQuery("SELECT /*+ NL_JOIN */ tbl.id, tbl.b, tbl2.id, tbl2.b FROM tbl JOIN tbl2 USING (id)") - .matches(QueryChecker.containsSubPlan("IgniteNestedLoopJoin")) - .resultSize(800) - .check(); + List> params = F.asList(F.asList("NL_JOIN", "NestedLoopJoin"), F.asList("HASH_JOIN", "IgniteHashJoin")); + + for (List paramSet : params) { + assertQuery("SELECT /*+ " + paramSet.get(0) + " */ tbl.id, tbl.b, tbl2.id, tbl2.b FROM tbl JOIN tbl2 USING (id)") + .matches(QueryChecker.containsSubPlan(paramSet.get(1))) + .resultSize(800) + .check(); + } for (int i = 800; i < 1000; i++) sql("INSERT INTO tbl2 VALUES (?, ?)", i, new byte[1000]); - assertThrows("SELECT /*+ NL_JOIN */ tbl.id, tbl.b, tbl2.id, tbl2.b FROM tbl JOIN tbl2 USING (id)", - IgniteException.class, "Query quota exceeded"); + for (List paramSet : params) { + assertThrows("SELECT /*+ " + paramSet.get(0) + " */ tbl.id, tbl.b, tbl2.id, tbl2.b FROM tbl JOIN tbl2 USING (id)", + IgniteException.class, "Query quota exceeded"); + } } /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java index a4726a0dacc88..0374001715c9c 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedNestedLoopJoinPlannerTest.java @@ -153,7 +153,7 @@ public void testInvalidIndexExpressions() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeRule", "HashJoinConverter" + "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeRule" ); assertNotNull(phys); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java index b0b0ccb447cf3..1d7f2e7438416 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java @@ -84,7 +84,7 @@ public void testSingleKey() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToSortedIndexSpoolRule" + "MergeJoinConverter", "HashJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToSortedIndexSpoolRule" ); System.out.println("+++\n" + RelOptUtil.toString(phys)); @@ -149,7 +149,7 @@ public void testMultipleKeys() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToSortedIndexSpoolRule" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter", "FilterSpoolMergeToSortedIndexSpoolRule" ); checkSplitAndSerialization(phys, publicSchema); @@ -212,7 +212,7 @@ public void testSourceWithoutCollation() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter" ); checkSplitAndSerialization(phys, publicSchema); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java index 4f72b072d00fc..c621ba7c3994e 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java @@ -90,28 +90,28 @@ public void testHashJoinErasesRightCollation() throws Exception { /** */ @Test public void testHashJoinWinsOnSkewedLeftInput() throws Exception { - TestTable thinTblSortedPk = createSimpleTable("SMALL_TBL", 1000); - TestTable thickTblSortedPk = createSimpleTable("LARGE_TBL", 500_000); + TestTable smallTbl = createSimpleTable("SMALL_TBL", 1000); + TestTable largeTbl = createSimpleTable("LARGE_TBL", 500_000); - IgniteSchema schema = createSchema(thinTblSortedPk, thickTblSortedPk); + IgniteSchema schema = createSchema(smallTbl, largeTbl); assertPlan( - "select t1.ID, t1.ID2, t2.ID, t2.ID2 from LARGE_TBL t1 join SMALL_TBL t2 on t1.ID2 = t2.ID2", + "select t1.ID, t1.INT_VAL, t2.ID, t2.INT_VAL from LARGE_TBL t1 join SMALL_TBL t2 on t1.INT_VAL = t2.INT_VAL", schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class)), "JoinCommuteRule" ); assertPlan( - "select t1.ID, t1.ID2, t2.ID, t2.ID2 from SMALL_TBL t1 join LARGE_TBL t2 on t1.ID2 = t2.ID2", + "select t1.ID, t1.INT_VAL, t2.ID, t2.INT_VAL from SMALL_TBL t1 join LARGE_TBL t2 on t1.INT_VAL = t2.INT_VAL", schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).negate()), "JoinCommuteRule" ); - // Merge join can consume less cpu resources. + // Merge join can consume less CPU resources. assertPlan( - "select t1.ID, t1.ID2, t2.ID, t2.ID2 from SMALL_TBL t1 join LARGE_TBL t2 on t1.ID = t2.ID", + "select t1.ID, t1.INT_VAL, t2.ID, t2.INT_VAL from SMALL_TBL t1 join LARGE_TBL t2 on t1.ID = t2.ID", schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).negate()), "JoinCommuteRule" @@ -124,7 +124,7 @@ public void testHashJoinWinsOnSkewedLeftInput() throws Exception { .and(node -> node.getInputs().size() == 1 && node.getInput(0) instanceof Join)); if (sortNodes.size() > 1) - throw new IllegalStateException("Unexpected count of sort nodes: exp<=1, act=" + sortNodes.size()); + throw new IllegalStateException("Incorrect sort nodes number: expected 1, actual " + sortNodes.size()); return sortNodes.isEmpty() ? null : sortNodes.get(0); } @@ -182,8 +182,8 @@ private static TestTable createSimpleTable(String name, int size) { size, IgniteDistributions.affinity(0, ++tableId, 0), "ID", Integer.class, - "ID2", Integer.class, - "VAL", String.class + "INT_VAL", Integer.class, + "STR_VAL", String.class ).addIndex( RelCollations.of(new RelFieldCollation(0, ASCENDING, RelFieldCollation.NullDirection.LAST)), "PK" @@ -198,7 +198,7 @@ private static TestTable createComplexTable() { IgniteDistributions.affinity(ImmutableIntList.of(0, 1), ++tableId, 0), "ID1", Integer.class, "ID2", Integer.class, - "VAL", String.class + "STR_VAL", String.class ).addIndex( RelCollations.of( new RelFieldCollation(0, ASCENDING, RelFieldCollation.NullDirection.LAST), @@ -207,22 +207,4 @@ private static TestTable createComplexTable() { "PK" ); } - -// /** */ -// static IgniteTable createSimpleTableHashPk(String tableName, int size) { -// return TestBuilders.table() -// .name(tableName) -// .size(size) -// .distribution(someAffinity()) -// .addColumn("ID", NativeTypes.INT32) -// .addColumn("ID2", NativeTypes.INT32) -// .addColumn("VAL", NativeTypes.STRING) -// .hashIndex() -// .name("PK") -// .addColumn("ID") -// .end() -// .build(); -// } - - } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java index 616405c5e2d33..932a6e1bd2b34 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java @@ -21,6 +21,7 @@ import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.util.ImmutableIntList; +import org.apache.ignite.internal.processors.query.calcite.rel.AbstractIgniteJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMergeJoin; @@ -42,6 +43,9 @@ * Test suite to verify join colocation. */ public class JoinColocationPlannerTest extends AbstractPlannerTest { + /** */ + private static String[] DISABLED_RULES = new String[] {"HashJoinConverter", "MergeJoinConverter"}; + /** * Join of the same tables with a simple affinity is expected to be colocated. */ @@ -62,16 +66,18 @@ public void joinSameTableSimpleAff() throws Exception { "from TEST_TBL t1 " + "join TEST_TBL t2 on t1.id = t2.id"; - RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin"); + for (String disabledRule : DISABLED_RULES) { + RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", disabledRule); - IgniteMergeJoin join = findFirstNode(phys, byClass(IgniteMergeJoin.class)); + AbstractIgniteJoin join = findFirstNode(phys, byClass(AbstractIgniteJoin.class)); - String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); + String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); - assertThat(invalidPlanMsg, join, notNullValue()); - assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); - assertThat(invalidPlanMsg, join.getLeft(), instanceOf(IgniteIndexScan.class)); - assertThat(invalidPlanMsg, join.getRight(), instanceOf(IgniteIndexScan.class)); + assertThat(invalidPlanMsg, join, notNullValue()); + assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); + assertThat(invalidPlanMsg, join.getLeft(), instanceOf(IgniteIndexScan.class)); + assertThat(invalidPlanMsg, join.getRight(), instanceOf(IgniteIndexScan.class)); + } } /** @@ -95,16 +101,18 @@ public void joinSameTableComplexAff() throws Exception { "from TEST_TBL t1 " + "join TEST_TBL t2 on t1.id1 = t2.id1 and t1.id2 = t2.id2"; - RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin"); + for (String disabledRule : DISABLED_RULES) { + RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", disabledRule); - IgniteMergeJoin join = findFirstNode(phys, byClass(IgniteMergeJoin.class)); + AbstractIgniteJoin join = findFirstNode(phys, byClass(AbstractIgniteJoin.class)); - String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); + String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); - assertThat(invalidPlanMsg, join, notNullValue()); - assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); - assertThat(invalidPlanMsg, join.getLeft(), instanceOf(IgniteIndexScan.class)); - assertThat(invalidPlanMsg, join.getRight(), instanceOf(IgniteIndexScan.class)); + assertThat(invalidPlanMsg, join, notNullValue()); + assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); + assertThat(invalidPlanMsg, join.getLeft(), instanceOf(IgniteIndexScan.class)); + assertThat(invalidPlanMsg, join.getRight(), instanceOf(IgniteIndexScan.class)); + } } /** @@ -140,22 +148,24 @@ public void joinComplexToSimpleAff() throws Exception { "from COMPLEX_TBL t1 " + "join SIMPLE_TBL t2 on t1.id1 = t2.id"; - RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin"); + for (String disabledRule : DISABLED_RULES) { + RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", disabledRule); - IgniteMergeJoin join = findFirstNode(phys, byClass(IgniteMergeJoin.class)); + AbstractIgniteJoin join = findFirstNode(phys, byClass(AbstractIgniteJoin.class)); - String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); + String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); - assertThat(invalidPlanMsg, join, notNullValue()); - assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); + assertThat(invalidPlanMsg, join, notNullValue()); + assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); - List exchanges = findNodes(phys, node -> node instanceof IgniteExchange - && ((IgniteRel)node).distribution().function().affinity()); + List exchanges = findNodes(phys, node -> node instanceof IgniteExchange + && ((IgniteRel)node).distribution().function().affinity()); - assertThat(invalidPlanMsg, exchanges, hasSize(1)); - assertThat(invalidPlanMsg, exchanges.get(0).getInput(0), instanceOf(IgniteIndexScan.class)); - assertThat(invalidPlanMsg, exchanges.get(0).getInput(0) - .getTable().unwrap(TestTable.class), equalTo(complexTbl)); + assertThat(invalidPlanMsg, exchanges, hasSize(1)); + assertThat(invalidPlanMsg, exchanges.get(0).getInput(0), instanceOf(IgniteIndexScan.class)); + assertThat(invalidPlanMsg, exchanges.get(0).getInput(0) + .getTable().unwrap(TestTable.class), equalTo(complexTbl)); + } } /** @@ -189,13 +199,15 @@ public void joinComplexToComplexAffWithDifferentOrder() throws Exception { "from COMPLEX_TBL_DIRECT t1 " + "join COMPLEX_TBL_INDIRECT t2 on t1.id1 = t2.id1 and t1.id2 = t2.id2"; - RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin"); + for (String disabledRule : DISABLED_RULES) { + RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", disabledRule); - IgniteMergeJoin exchange = findFirstNode(phys, node -> node instanceof IgniteExchange - && ((IgniteRel)node).distribution().function().affinity()); + IgniteMergeJoin exchange = findFirstNode(phys, node -> node instanceof IgniteExchange + && ((IgniteRel)node).distribution().function().affinity()); - String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); + String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); - assertThat(invalidPlanMsg, exchange, nullValue()); + assertThat(invalidPlanMsg, exchange, nullValue()); + } } } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java index 93ea2f87579e8..f5207009a7108 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java @@ -81,8 +81,8 @@ public class JoinCommutePlannerTest extends AbstractPlannerTest { public void testOuterCommute() throws Exception { String sql = "SELECT COUNT(*) FROM SMALL s RIGHT JOIN HUGE h on h.id = s.id"; - IgniteRel phys = physicalPlan(sql, publicSchema, - "MergeJoinConverter", "CorrelatedNestedLoopJoin"); + IgniteRel phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", + "HashJoinConverter"); assertNotNull(phys); @@ -94,7 +94,7 @@ public void testOuterCommute() throws Exception { assertEquals(JoinRelType.LEFT, join.getJoinType()); - PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", + PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin"); RelOptPlanner pl = ctx.cluster().getPlanner(); @@ -105,8 +105,8 @@ public void testOuterCommute() throws Exception { assertNotNull(phys); - phys = physicalPlan(sql, publicSchema, - "MergeJoinConverter", "CorrelatedNestedLoopJoin", "JoinCommuteRule"); + phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", "HashJoinConverter", + "JoinCommuteRule"); join = findFirstNode(phys, byClass(IgniteNestedLoopJoin.class)); @@ -117,8 +117,8 @@ public void testOuterCommute() throws Exception { // no commute assertEquals(JoinRelType.RIGHT, join.getJoinType()); - ctx = plannerCtx(sql, publicSchema, - "MergeJoinConverter", "CorrelatedNestedLoopJoin", "JoinCommuteRule"); + ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", "HashJoinConverter", + "JoinCommuteRule"); pl = ctx.cluster().getPlanner(); @@ -134,8 +134,7 @@ public void testOuterCommute() throws Exception { public void testInnerCommute() throws Exception { String sql = "SELECT COUNT(*) FROM SMALL s JOIN HUGE h on h.id = s.id"; - IgniteRel phys = physicalPlan(sql, publicSchema, - "MergeJoinConverter", "CorrelatedNestedLoopJoin"); + IgniteRel phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin"); assertNotNull(phys); @@ -162,7 +161,7 @@ public void testInnerCommute() throws Exception { assertEquals(JoinRelType.INNER, join.getJoinType()); - PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", + PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin"); RelOptPlanner pl = ctx.cluster().getPlanner(); @@ -173,8 +172,8 @@ public void testInnerCommute() throws Exception { assertNotNull(phys); - phys = physicalPlan(sql, publicSchema, - "MergeJoinConverter", "CorrelatedNestedLoopJoin", "JoinCommuteRule"); + phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin", + "JoinCommuteRule"); join = findFirstNode(phys, byClass(IgniteNestedLoopJoin.class)); proj = findFirstNode(phys, byClass(IgniteProject.class)); @@ -200,8 +199,8 @@ public void testInnerCommute() throws Exception { // no commute assertEquals(JoinRelType.INNER, join.getJoinType()); - ctx = plannerCtx(sql, publicSchema, - "MergeJoinConverter", "CorrelatedNestedLoopJoin", "JoinCommuteRule"); + ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin", + "JoinCommuteRule"); pl = ctx.cluster().getPlanner(); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java index 9116092c0d7db..c0bbb910e5e44 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java @@ -94,6 +94,7 @@ public void testNoSortAppendingWithCorrectCollation() throws Exception { publicSchema, F.concat(HASH_AGG_RULES, "NestedLoopJoinConverter", + "HashConverter", "CorrelatedNestedLoopJoin", "CorrelateToNestedLoopRule" ) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java index ceed9c6d7d76c..82c2f4fa471ed 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java @@ -90,7 +90,7 @@ public void testNotColocatedEqJoin() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" ); checkSplitAndSerialization(phys, publicSchema); @@ -157,7 +157,7 @@ public void testPartialIndexForCondition() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" + "MergeJoinConverter", "HashJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" ); System.out.println("+++ \n" + RelOptUtil.toString(phys)); @@ -252,7 +252,7 @@ public void testRestoreCollation() throws Exception { .and(input(1, isInstanceOf(IgniteSortedIndexSpool.class) .and(spool -> spool.collation().getFieldCollations().get(0).getFieldIndex() == equalIdx) ))), - "MergeJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" + "MergeJoinConverter", "HashJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" ); } } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/JoinTypeHintPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/JoinTypeHintPlannerTest.java index b03ba932e72c5..baa47e25cf3e3 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/JoinTypeHintPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/hints/JoinTypeHintPlannerTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.query.calcite.planner.TestTable; import org.apache.ignite.internal.processors.query.calcite.rel.AbstractIgniteJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMergeJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin; import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; @@ -36,9 +37,11 @@ import org.junit.Test; import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.CNL_JOIN; +import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.HASH_JOIN; import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.MERGE_JOIN; import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.NL_JOIN; import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.NO_CNL_JOIN; +import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.NO_HASH_JOIN; import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.NO_MERGE_JOIN; import static org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition.NO_NL_JOIN; @@ -120,12 +123,12 @@ public void testHintsErrors() throws Exception { // Following hint must not override leading. lsnrLog.clearListeners(); - lsnr = LogListener.matches("Skipped hint '" + NL_JOIN.name() + "'") + lsnr = LogListener.matches("Skipped hint '" + HASH_JOIN.name() + "'") .andMatches("This join type is already disabled or forced to use before").build(); lsnrLog.registerListener(lsnr); - physicalPlan("SELECT /*+ " + MERGE_JOIN + "(TBL1)," + NL_JOIN + "(TBL1,TBL2) */ t1.v1, t2.v2 FROM TBL1 " + + physicalPlan("SELECT /*+ " + MERGE_JOIN + "(TBL1)," + HASH_JOIN + "(TBL1,TBL2) */ t1.v1, t2.v2 FROM TBL1 " + "t1 JOIN TBL2 t2 on t1.v3=t2.v3", schema); assertTrue(lsnr.check()); @@ -140,7 +143,7 @@ public void testHintsErrors() throws Exception { physicalPlan("SELECT /*+ " + NL_JOIN + "(TBL1) */ t1.v1, t2.v2 FROM TBL1 " + "t1 JOIN TBL2 t2 on t1.v3=t2.v3 where t2.v1 in " + - "(SELECT /*+ " + MERGE_JOIN + "(TBL1), " + CNL_JOIN + "(TBL1,TBL3) */ t3.v3 from TBL3 t3 JOIN TBL1 t4 " + + "(SELECT /*+ " + HASH_JOIN + "(TBL1), " + CNL_JOIN + "(TBL1,TBL3) */ t3.v3 from TBL3 t3 JOIN TBL1 t4 " + "on t3.v2=t4.v2)", schema, CORE_JOIN_REORDER_RULES); assertTrue(lsnr.check()); @@ -186,9 +189,9 @@ public void testHintsErrors() throws Exception { */ @Test public void testDisableNLJoin() throws Exception { - for (HintDefinition hint : Arrays.asList(NO_NL_JOIN, CNL_JOIN, MERGE_JOIN)) { + for (HintDefinition hint : Arrays.asList(NO_NL_JOIN, CNL_JOIN, MERGE_JOIN, HASH_JOIN)) { doTestDisableJoinTypeWith("TBL5", "TBL4", "INNER", IgniteNestedLoopJoin.class, - NO_NL_JOIN, "MergeJoinConverter"); + NO_NL_JOIN, "MergeJoinConverter", "HashJoinConverter"); doTestDisableJoinTypeWith("TBL3", "TBL1", "LEFT", IgniteNestedLoopJoin.class, NO_NL_JOIN); @@ -210,15 +213,34 @@ public void testDisableNLJoin() throws Exception { @Test public void testDisableMergeJoin() throws Exception { for (HintDefinition hint : Arrays.asList(NO_MERGE_JOIN, NL_JOIN, CNL_JOIN)) { - doTestDisableJoinTypeWith("TBL4", "TBL2", "INNER", IgniteMergeJoin.class, hint); + doTestDisableJoinTypeWith("TBL4", "TBL2", "INNER", IgniteMergeJoin.class, hint, "HashJoinConverter"); - doTestDisableJoinTypeWith("TBL4", "TBL2", "LEFT", IgniteMergeJoin.class, hint); + doTestDisableJoinTypeWith("TBL4", "TBL2", "LEFT", IgniteMergeJoin.class, hint, "HashJoinConverter"); // Correlated nested loop join supports only INNER and LEFT join types. if (hint != CNL_JOIN) { - doTestDisableJoinTypeWith("TBL4", "TBL2", "RIGHT", IgniteMergeJoin.class, hint); + doTestDisableJoinTypeWith("TBL4", "TBL2", "RIGHT", IgniteMergeJoin.class, hint, "HashJoinConverter"); - doTestDisableJoinTypeWith("TBL4", "TBL2", "FULL", IgniteMergeJoin.class, hint); + doTestDisableJoinTypeWith("TBL4", "TBL2", "FULL", IgniteMergeJoin.class, hint, "HashJoinConverter"); + } + } + } + + /** + * Tests hash join is disabled by hints. + */ + @Test + public void testDisableHashJoin() throws Exception { + for (HintDefinition hint : Arrays.asList(NO_HASH_JOIN, MERGE_JOIN, NL_JOIN, CNL_JOIN)) { + doTestDisableJoinTypeWith("TBL4", "TBL2", "INNER", IgniteHashJoin.class, hint); + + doTestDisableJoinTypeWith("TBL4", "TBL2", "LEFT", IgniteHashJoin.class, hint); + + // Correlated nested loop join supports only INNER and LEFT join types. + if (hint != CNL_JOIN) { + doTestDisableJoinTypeWith("TBL4", "TBL2", "RIGHT", IgniteHashJoin.class, hint); + + doTestDisableJoinTypeWith("TBL4", "TBL2", "FULL", IgniteHashJoin.class, hint); } } } @@ -241,6 +263,24 @@ public void testMergeJoinEnabled() throws Exception { MERGE_JOIN, IgniteMergeJoin.class, CORE_JOIN_REORDER_RULES); } + /** + * Tests the hash join is enabled by the hint instead of the other joins. + */ + @Test + public void testHashJoinEnabled() throws Exception { + doTestCertainJoinTypeEnabled("TBL1", "INNER", "TBL2", IgniteCorrelatedNestedLoopJoin.class, + HASH_JOIN, IgniteHashJoin.class); + + doTestCertainJoinTypeEnabled("TBL1", "RIGHT", "TBL2", IgniteNestedLoopJoin.class, + HASH_JOIN, IgniteHashJoin.class); + + doTestCertainJoinTypeEnabled("TBL1", "INNER", "TBL2", IgniteCorrelatedNestedLoopJoin.class, + HASH_JOIN, IgniteHashJoin.class, CORE_JOIN_REORDER_RULES); + + doTestCertainJoinTypeEnabled("TBL1", "RIGHT", "TBL2", IgniteNestedLoopJoin.class, + HASH_JOIN, IgniteHashJoin.class, CORE_JOIN_REORDER_RULES); + } + /** * Tests the nested loop join is enabled by the hint instead of the other joins. */ @@ -249,13 +289,13 @@ public void testNLJoinEnabled() throws Exception { doTestCertainJoinTypeEnabled("TBL2", "INNER", "TBL1", IgniteCorrelatedNestedLoopJoin.class, NL_JOIN, IgniteNestedLoopJoin.class); - doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteMergeJoin.class, + doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteHashJoin.class, NL_JOIN, IgniteNestedLoopJoin.class); doTestCertainJoinTypeEnabled("TBL1", "LEFT", "TBL2", IgniteCorrelatedNestedLoopJoin.class, NL_JOIN, IgniteNestedLoopJoin.class, CORE_JOIN_REORDER_RULES); - doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteMergeJoin.class, + doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteHashJoin.class, NL_JOIN, IgniteNestedLoopJoin.class, CORE_JOIN_REORDER_RULES); } @@ -281,7 +321,7 @@ public void testCNLJoinEnabled() throws Exception { doTestCertainJoinTypeEnabled("TBL2", "LEFT", "TBL1", IgniteNestedLoopJoin.class, CNL_JOIN, IgniteCorrelatedNestedLoopJoin.class); - doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteMergeJoin.class, + doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteHashJoin.class, CNL_JOIN, IgniteCorrelatedNestedLoopJoin.class); // Even CNL join doesn't support RIGHT join, join type and join inputs might be switched by Calcite. @@ -291,7 +331,7 @@ public void testCNLJoinEnabled() throws Exception { doTestCertainJoinTypeEnabled("TBL2", "LEFT", "TBL1", IgniteNestedLoopJoin.class, CNL_JOIN, IgniteCorrelatedNestedLoopJoin.class, CORE_JOIN_REORDER_RULES); - doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteMergeJoin.class, + doTestCertainJoinTypeEnabled("TBL5", "INNER", "TBL4", IgniteHashJoin.class, CNL_JOIN, IgniteCorrelatedNestedLoopJoin.class, CORE_JOIN_REORDER_RULES); } @@ -387,15 +427,15 @@ public void testTableHints() throws Exception { .and(input(1, nodeOrAnyChild(isTableScan("TBL1")))))), CORE_JOIN_REORDER_RULES); // Table hint has a bigger priority. Leading CNL_JOIN is ignored. - assertPlan(String.format(sqlTpl, "/*+ " + CNL_JOIN + " */", "/*+ " + NL_JOIN + " */", "/*+ " + MERGE_JOIN + " */"), + assertPlan(String.format(sqlTpl, "/*+ " + CNL_JOIN + " */", "/*+ " + NL_JOIN + " */", "/*+ " + HASH_JOIN + " */"), schema, nodeOrAnyChild(isInstanceOf(IgniteNestedLoopJoin.class).and(input(1, isTableScan("TBL3")))) - .and(nodeOrAnyChild(isInstanceOf(IgniteMergeJoin.class) + .and(nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class) .and(input(1, nodeOrAnyChild(isTableScan("TBL1")))))), CORE_JOIN_REORDER_RULES); // Leading query hint works only for the second join. - assertPlan(String.format(sqlTpl, "/*+ " + CNL_JOIN + " */", "/*+ " + NL_JOIN + " */", ""), schema, + assertPlan(String.format(sqlTpl, "/*+ " + HASH_JOIN + " */", "/*+ " + NL_JOIN + " */", ""), schema, nodeOrAnyChild(isInstanceOf(IgniteNestedLoopJoin.class).and(input(1, isTableScan("TBL3")))) - .and(nodeOrAnyChild(isInstanceOf(IgniteCorrelatedNestedLoopJoin.class) + .and(nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class) .and(input(1, nodeOrAnyChild(isTableScan("TBL1")))))), CORE_JOIN_REORDER_RULES); // Table hint with wrong table name is ignored. @@ -499,13 +539,13 @@ public void testSeveralDisables() throws Exception { assertPlan(String.format(sqlTpl, "/*+ " + NO_CNL_JOIN + ',' + NO_NL_JOIN + " */"), schema, nodeOrAnyChild(isInstanceOf(IgniteCorrelatedNestedLoopJoin.class)).negate() .and(nodeOrAnyChild(isInstanceOf(IgniteNestedLoopJoin.class)).negate()) - .and(nodeOrAnyChild(isInstanceOf(IgniteMergeJoin.class).and(hasNestedTableScan("TBL1")) + .and(nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).and(hasNestedTableScan("TBL1")) .and(hasNestedTableScan("TBL2")))), CORE_JOIN_REORDER_RULES); assertPlan(String.format(sqlTpl, "/*+ " + NO_CNL_JOIN + "(TBL1)," + NO_NL_JOIN + "(TBL2) */"), schema, nodeOrAnyChild(isInstanceOf(IgniteCorrelatedNestedLoopJoin.class)).negate() .and(nodeOrAnyChild(isInstanceOf(IgniteNestedLoopJoin.class)).negate()) - .and(nodeOrAnyChild(isInstanceOf(IgniteMergeJoin.class).and(hasNestedTableScan("TBL1")) + .and(nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).and(hasNestedTableScan("TBL1")) .and(hasNestedTableScan("TBL2")))), CORE_JOIN_REORDER_RULES); // Check with forcing in the middle. @@ -530,7 +570,7 @@ schema, nodeOrAnyChild(isInstanceOf(IgniteCorrelatedNestedLoopJoin.class)).negat assertPlan(String.format(sqlTpl, "/*+ " + NO_CNL_JOIN + ',' + NO_NL_JOIN + ',' + NO_MERGE_JOIN + " */"), schema, nodeOrAnyChild(isInstanceOf(IgniteCorrelatedNestedLoopJoin.class)).negate() .and(nodeOrAnyChild(isInstanceOf(IgniteNestedLoopJoin.class)).negate()) - .and(nodeOrAnyChild(isInstanceOf(IgniteMergeJoin.class).and(hasNestedTableScan("TBL1")) + .and(nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class).and(hasNestedTableScan("TBL1")) .and(hasNestedTableScan("TBL2")))), CORE_JOIN_REORDER_RULES); // Check many duplicated disables doesn't erase other disables. diff --git a/modules/calcite/src/test/sql/sqlite/join/join1.test b/modules/calcite/src/test/sql/sqlite/join/join1.test index d1f33fa306274..09b41b0b30ae3 100644 --- a/modules/calcite/src/test/sql/sqlite/join/join1.test +++ b/modules/calcite/src/test/sql/sqlite/join/join1.test @@ -24,7 +24,7 @@ statement ok INSERT INTO t2 VALUES(3,4,5); query III rowsort -SELECT /*+ CNL_JOIN */ t2.* FROM t2 NATURAL JOIN t1; +SELECT /*+ NL_JOIN */ t2.* FROM t2 NATURAL JOIN t1; ---- 2 3 4 3 4 5 @@ -41,6 +41,12 @@ SELECT /*+ MERGE_JOIN */ t2.* FROM t2 NATURAL JOIN t1; 2 3 4 3 4 5 +query III rowsort +SELECT /*+ HASH_JOIN */ t2.* FROM t2 NATURAL JOIN t1; +---- +2 3 4 +3 4 5 + query III rowsort SELECT t1.* FROM t2 NATURAL JOIN t1; ---- From c30cb4429be221f786fc42199d6c96a1a67b9e4a Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 24 Dec 2024 13:16:28 +0300 Subject: [PATCH 03/28] Minor manual review fixes --- .../calcite/exec/LogicalRelImplementor.java | 13 +- .../query/calcite/exec/rel/HashJoinNode.java | 47 +++--- .../calcite/exec/rel/NestedLoopJoinNode.java | 148 ++++++++++-------- .../query/calcite/rel/IgniteHashJoin.java | 1 - .../query/calcite/exec/rel/ExecutionTest.java | 16 +- ...iteBasicSecondaryIndexIntegrationTest.java | 67 ++++---- .../MemoryQuotasIntegrationTest.java | 12 +- .../calcite/planner/HashJoinPlannerTest.java | 15 +- .../planner/JoinColocationPlannerTest.java | 2 +- .../planner/JoinCommutePlannerTest.java | 15 +- .../planner/SortAggregatePlannerTest.java | 1 - .../planner/SortedIndexSpoolPlannerTest.java | 2 +- 12 files changed, 174 insertions(+), 165 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java index a3c0d17a6b7d6..82da94506e6d1 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java @@ -270,17 +270,10 @@ public LogicalRelImplementor( /** {@inheritDoc} */ @Override public Node visit(IgniteHashJoin rel) { - RelDataType outType = rel.getRowType(); - RelDataType leftType = rel.getLeft().getRowType(); - RelDataType rightType = rel.getRight().getRowType(); - JoinRelType joinType = rel.getJoinType(); - - Node node = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, rel.analyzeCondition()); - - Node leftInput = visit(rel.getLeft()); - Node rightInput = visit(rel.getRight()); + Node node = HashJoinNode.create(ctx, rel.getRowType(), rel.getLeft().getRowType(), rel.getRight().getRowType(), + rel.getJoinType(), rel.analyzeCondition()); - node.register(Arrays.asList(leftInput, rightInput)); + node.register(Arrays.asList(visit(rel.getLeft()), visit(rel.getRight()))); return node; } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index df5ae17e8bde6..dd426ddebdc00 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -32,7 +32,7 @@ import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; -/** HashJoin implementor. */ +/** Hash join implementor. */ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNode { /** */ private static final int INITIAL_CAPACITY = 128; @@ -41,10 +41,10 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod private static final Key NULL_KEY = new Key(); /** */ - private final int[] leftJoinPositions; + private final int[] leftKeys; /** */ - private final int[] rightJoinPositions; + private final int[] rightKeys; /** Output row handler. */ protected final RowHandler outRowHnd; @@ -66,21 +66,21 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod protected HashJoinNode(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { super(ctx, rowType); - leftJoinPositions = info.leftKeys.toIntArray(); - rightJoinPositions = info.rightKeys.toIntArray(); + leftKeys = info.leftKeys.toIntArray(); + rightKeys = info.rightKeys.toIntArray(); - assert leftJoinPositions.length == rightJoinPositions.length; + assert leftKeys.length == rightKeys.length; this.outRowHnd = outRowHnd; } /** {@inheritDoc} */ @Override protected void rewindInternal() { + super.rewindInternal(); + rightIt = Collections.emptyIterator(); hashStore.clear(); - - super.rewindInternal(); } /** Creates certain join node. */ @@ -116,13 +116,13 @@ public static HashJoinNode create( return new AntiHashJoin<>(ctx, outRowType, info, rowHnd); default: - throw new IllegalArgumentException("Join of type ''" + type + "' isn't supported yet."); + throw new IllegalArgumentException("Join of type '" + type + "' isn't supported."); } } /** */ protected Collection lookup(Row row) { - Key row0 = extractKey(row, leftJoinPositions); + Key row0 = extractKey(row, leftKeys); // Key with null field can't be compared with other keys. if (row0 == NULL_KEY) @@ -162,7 +162,7 @@ private static Iterator untouched(Map> return innerIt.next(); } - void advance() { + private void advance() { while (it.hasNext()) { TouchedCollection coll = it.next(); @@ -187,9 +187,9 @@ void advance() { waitingRight--; - Key key = extractKey(row, rightJoinPositions); + Key key = extractKey(row, rightKeys); - // No storing the row in hashStore, if it contains NULL. And we won't emit right part alone like in FULL OUTER and RIGHT joins. + // No storing in #hashStore, if the row contains NULL. And we won't emit right part alone like in FULL OUTER and RIGHT joins. if (keepRowsWithNull() || key != NULL_KEY) { TouchedCollection raw = hashStore.computeIfAbsent(key, k -> new TouchedCollection<>()); @@ -213,7 +213,7 @@ private Key extractKey(Row row, int[] mapping) { } /** */ - protected void getMoreOrEnd() throws Exception { + protected void requestMoreOrEnd() throws Exception { if (waitingRight == 0) rightSource().request(waitingRight = IN_BUFFER_SIZE); @@ -230,8 +230,7 @@ protected void getMoreOrEnd() throws Exception { /** * Returns {@code true} if we need to store the row from right shoulder even if it contains NULL in any of join key position. - * - *

This is required for joins which emit unmatched part of the right shoulder, such as RIGHT JOIN and FULL OUTER JOIN. + * This is required for joins which emit unmatched part of the right shoulder, such as RIGHT JOIN and FULL OUTER JOIN. * * @return {@code true} when row must be stored in {@link #hashStore} unconditionally. */ @@ -247,7 +246,7 @@ private static final class InnerHashJoin extends HashJoinNode { * @param ctx Execution context. * @param rowType Row type. * @param info Join info. - * @param outRowHnd Output row factory. + * @param outRowHnd Output row handler. */ private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { super(ctx, rowType, info, outRowHnd); @@ -288,7 +287,7 @@ private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo } } - getMoreOrEnd(); + requestMoreOrEnd(); } } @@ -363,7 +362,7 @@ private LeftHashJoin( } } - getMoreOrEnd(); + requestMoreOrEnd(); } } @@ -378,7 +377,7 @@ private static final class RightHashJoin extends HashJoinNode { * @param ctx Execution context. * @param rowType Row type. * @param info Join info. - * @param outRowHnd Output row factory. + * @param outRowHnd Output row handler. * @param leftRowFactory Left row factory. */ private RightHashJoin( @@ -459,7 +458,7 @@ private RightHashJoin( } } - getMoreOrEnd(); + requestMoreOrEnd(); } /** {@inheritDoc} */ @@ -574,7 +573,7 @@ private FullOuterHashJoin( } } - getMoreOrEnd(); + requestMoreOrEnd(); } /** {@inheritDoc} */ @@ -627,7 +626,7 @@ private SemiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo i } } - getMoreOrEnd(); + requestMoreOrEnd(); } } @@ -675,7 +674,7 @@ private AntiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo i } } - getMoreOrEnd(); + requestMoreOrEnd(); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index 2ef56141cd259..cf9d9e010f38b 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -25,7 +25,6 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; -import org.jetbrains.annotations.NotNull; /** */ public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJoinNode { @@ -33,27 +32,28 @@ public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJ protected final BiPredicate cond; /** */ - protected final RowHandler handler; + protected final RowHandler rowHnd; /** */ protected final List rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE); /** * @param ctx Execution context. + * @param rowType Row type. * @param cond Join expression. */ private NestedLoopJoinNode(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType); this.cond = cond; - handler = ctx.rowHandler(); + rowHnd = ctx.rowHandler(); } /** {@inheritDoc} */ @Override protected void rewindInternal() { - rightMaterialized.clear(); - super.rewindInternal(); + + rightMaterialized.clear(); } /** {@inheritDoc} */ @@ -74,8 +74,14 @@ private NestedLoopJoinNode(ExecutionContext ctx, RelDataType rowType, BiPre } /** */ - @NotNull public static NestedLoopJoinNode create(ExecutionContext ctx, RelDataType outputRowType, - RelDataType leftRowType, RelDataType rightRowType, JoinRelType joinType, BiPredicate cond) { + public static NestedLoopJoinNode create( + ExecutionContext ctx, + RelDataType outputRowType, + RelDataType leftRowType, + RelDataType rightRowType, + JoinRelType joinType, + BiPredicate cond + ) { switch (joinType) { case INNER: return new InnerJoin<>(ctx, outputRowType, cond); @@ -106,7 +112,7 @@ private NestedLoopJoinNode(ExecutionContext ctx, RelDataType rowType, BiPre return new AntiJoin<>(ctx, outputRowType, cond); default: - throw new IllegalStateException("Join type \"" + joinType + "\" is not supported yet"); + throw new IllegalArgumentException("Join type '" + joinType + "' is not supported."); } } @@ -115,25 +121,23 @@ private static class InnerJoin extends NestedLoopJoinNode { /** */ private int rightIdx; - /** - * @param ctx Execution context. - * @param cond Join expression. - */ + /** */ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); } /** {@inheritDoc} */ @Override protected void rewindInternal() { - rightIdx = 0; - super.rewindInternal(); + + rightIdx = 0; } - /** */ + /** {@inheritDoc} */ @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { if (left == null) @@ -146,12 +150,13 @@ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) { requested = 0; + downstream().end(); } } @@ -179,16 +185,13 @@ private static class LeftJoin extends NestedLoopJoinNode { /** Right row factory. */ private final RowHandler.RowFactory rightRowFactory; - /** Whether current left row was matched or not. */ + /** Shows whether current left row was matched. */ private boolean matched; /** */ private int rightIdx; - /** - * @param ctx Execution context. - * @param cond Join expression. - */ + /** */ public LeftJoin( ExecutionContext ctx, RelDataType rowType, @@ -200,18 +203,19 @@ public LeftJoin( this.rightRowFactory = rightRowFactory; } - /** */ + /** {@inheritDoc} */ @Override protected void rewindInternal() { + super.rewindInternal(); + matched = false; rightIdx = 0; - - super.rewindInternal(); } /** {@inheritDoc} */ @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { if (left == null) { @@ -227,10 +231,10 @@ public LeftJoin( continue; requested--; + matched = true; - Row row = handler.concat(left, rightMaterialized.get(rightIdx - 1)); - downstream().push(row); + downstream().push(rowHnd.concat(left, rightMaterialized.get(rightIdx - 1))); } if (rightIdx == rightMaterialized.size()) { @@ -238,13 +242,15 @@ public LeftJoin( if (!matched && requested > 0) { requested--; + wasPushed = true; - downstream().push(handler.concat(left, rightRowFactory.create())); + downstream().push(rowHnd.concat(left, rightRowFactory.create())); } if (matched || wasPushed) { left = null; + rightIdx = 0; } } @@ -263,6 +269,7 @@ public LeftJoin( if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) { requested = 0; + downstream().end(); } } @@ -270,7 +277,7 @@ public LeftJoin( /** */ private static class RightJoin extends NestedLoopJoinNode { - /** Right row factory. */ + /** Left row factory. */ private final RowHandler.RowFactory leftRowFactory; /** */ @@ -282,10 +289,7 @@ private static class RightJoin extends NestedLoopJoinNode { /** */ private int rightIdx; - /** - * @param ctx Execution context. - * @param cond Join expression. - */ + /** */ public RightJoin( ExecutionContext ctx, RelDataType rowType, @@ -299,11 +303,11 @@ public RightJoin( /** {@inheritDoc} */ @Override protected void rewindInternal() { + super.rewindInternal(); + rightNotMatchedIndexes.clear(); lastPushedInd = 0; rightIdx = 0; - - super.rewindInternal(); } /** {@inheritDoc} */ @@ -330,14 +334,15 @@ public RightJoin( continue; requested--; + rightNotMatchedIndexes.clear(rightIdx - 1); - Row joined = handler.concat(left, right); - downstream().push(joined); + downstream().push(rowHnd.concat(left, right)); } if (rightIdx == rightMaterialized.size()) { left = null; + rightIdx = 0; } } @@ -351,6 +356,7 @@ public RightJoin( assert lastPushedInd >= 0; inLoop = true; + try { for (lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd);; lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd + 1) @@ -360,11 +366,12 @@ public RightJoin( if (lastPushedInd < 0) break; - Row row = handler.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd)); + Row row = rowHnd.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd)); rightNotMatchedIndexes.clear(lastPushedInd); requested--; + downstream().push(row); if (lastPushedInd == Integer.MAX_VALUE || requested <= 0) @@ -382,9 +389,10 @@ public RightJoin( if (waitingLeft == 0 && leftInBuf.isEmpty()) leftSource().request(waitingLeft = IN_BUFFER_SIZE); - if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null - && leftInBuf.isEmpty() && rightNotMatchedIndexes.isEmpty()) { + if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty() + && rightNotMatchedIndexes.isEmpty()) { requested = 0; + downstream().end(); } } @@ -398,7 +406,7 @@ private static class FullOuterJoin extends NestedLoopJoinNode { /** Right row factory. */ private final RowHandler.RowFactory rightRowFactory; - /** Whether current left row was matched or not. */ + /** Shows whether current left row was matched. */ private boolean leftMatched; /** */ @@ -413,10 +421,7 @@ private static class FullOuterJoin extends NestedLoopJoinNode { /** */ private int rightIdx; - /** - * @param ctx Execution context. - * @param cond Join expression. - */ + /** */ public FullOuterJoin( ExecutionContext ctx, RelDataType rowType, @@ -432,13 +437,13 @@ public FullOuterJoin( /** {@inheritDoc} */ @Override protected void rewindInternal() { + super.rewindInternal(); + left = null; leftMatched = false; rightNotMatchedIndexes.clear(); lastPushedInd = 0; rightIdx = 0; - - super.rewindInternal(); } /** {@inheritDoc} */ @@ -451,6 +456,7 @@ public FullOuterJoin( } inLoop = true; + try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { if (left == null) { @@ -468,11 +474,12 @@ public FullOuterJoin( continue; requested--; + leftMatched = true; + rightNotMatchedIndexes.clear(rightIdx - 1); - Row joined = handler.concat(left, right); - downstream().push(joined); + downstream().push(rowHnd.concat(left, right)); } if (rightIdx == rightMaterialized.size()) { @@ -480,13 +487,15 @@ public FullOuterJoin( if (!leftMatched && requested > 0) { requested--; + wasPushed = true; - downstream().push(handler.concat(left, rightRowFactory.create())); + downstream().push(rowHnd.concat(left, rightRowFactory.create())); } if (leftMatched || wasPushed) { left = null; + rightIdx = 0; } } @@ -501,6 +510,7 @@ public FullOuterJoin( assert lastPushedInd >= 0; inLoop = true; + try { for (lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd);; lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd + 1) @@ -510,11 +520,12 @@ public FullOuterJoin( if (lastPushedInd < 0) break; - Row row = handler.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd)); + Row row = rowHnd.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd)); rightNotMatchedIndexes.clear(lastPushedInd); requested--; + downstream().push(row); if (lastPushedInd == Integer.MAX_VALUE || requested <= 0) @@ -532,9 +543,10 @@ public FullOuterJoin( if (waitingLeft == 0 && leftInBuf.isEmpty()) leftSource().request(waitingLeft = IN_BUFFER_SIZE); - if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null - && leftInBuf.isEmpty() && rightNotMatchedIndexes.isEmpty()) { + if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty() + && rightNotMatchedIndexes.isEmpty()) { requested = 0; + downstream().end(); } } @@ -545,19 +557,16 @@ private static class SemiJoin extends NestedLoopJoinNode { /** */ private int rightIdx; - /** - * @param ctx Execution context. - * @param cond Join expression. - */ + /** */ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); } /** {@inheritDoc} */ @Override protected void rewindInternal() { - rightIdx = 0; - super.rewindInternal(); + + rightIdx = 0; } /** {@inheritDoc} */ @@ -576,6 +585,7 @@ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null - && leftInBuf.isEmpty()) { + if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) { downstream().end(); + requested = 0; } } @@ -607,25 +618,23 @@ private static class AntiJoin extends NestedLoopJoinNode { /** */ private int rightIdx; - /** - * @param ctx Execution context. - * @param cond Join expression. - */ + /** */ public AntiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); } - /** */ + /** {@inheritDoc} */ @Override protected void rewindInternal() { - rightIdx = 0; - super.rewindInternal(); + + rightIdx = 0; } /** {@inheritDoc} */ @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { if (left == null) @@ -642,10 +651,12 @@ public AntiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) { requested = 0; + downstream().end(); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java index c6998dc0a9d67..3e012c551eace 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java @@ -15,7 +15,6 @@ * limitations under the License. */ - package org.apache.ignite.internal.processors.query.calcite.rel; import java.util.List; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java index fc8cf3dc5c1ff..9e85722332260 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java @@ -291,7 +291,7 @@ public void testRightJoin() { assertEquals(4, rows.size()); - checkDepJoinEmpResults(RIGHT, rows); + checkDepartmentsJoinEmployeesResults(RIGHT, rows); } /** @@ -408,7 +408,7 @@ public void testSemiJoin() { while (node.hasNext()) rows.add(node.next()); - checkDepJoinEmpResults(SEMI, rows); + checkDepartmentsJoinEmployeesResults(SEMI, rows); } /** @@ -465,7 +465,7 @@ public void testAntiJoin() { assertEquals(1, rows.size()); - checkDepJoinEmpResults(ANTI, rows); + checkDepartmentsJoinEmployeesResults(ANTI, rows); } /** @@ -534,15 +534,15 @@ public void testCorrelatedNestedLoopJoin() { } } - /** */ + /** Tests 'Select e.id, e.name, d.name as dep_name from DEP d join EMP e on e.DEPNO = d.DEPNO'. */ @Test public void testHashJoin() { - // select e.id, e.name, d.name as dep_name from DEP d join EMP e on e.DEPNO = d.DEPNO for (JoinRelType joinType : F.asList(LEFT, INNER, RIGHT, FULL, SEMI, ANTI)) { ExecutionContext ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0); IgniteTypeFactory tf = ctx.getTypeFactory(); RelDataType rowType = TypeUtils.createRowType(tf, int.class, String.class); + ScanNode leftDeps = new ScanNode<>(ctx, rowType, Arrays.asList( new Object[] {1, "Core"}, new Object[] {2, "SQL"}, @@ -550,6 +550,7 @@ public void testHashJoin() { )); rowType = TypeUtils.createRowType(tf, int.class, String.class, Integer.class); + ScanNode rightEmps = new ScanNode<>(ctx, rowType, Arrays.asList( new Object[] {0, "Igor", 1}, new Object[] {1, "Roman", 2}, @@ -575,6 +576,7 @@ public void testHashJoin() { project.register(join); RootNode rootScan = new RootNode<>(ctx, rowType); + rootScan.register(project); assert rootScan.hasNext(); @@ -584,12 +586,12 @@ public void testHashJoin() { while (rootScan.hasNext()) rows.add(rootScan.next()); - checkDepJoinEmpResults(joinType, rows); + checkDepartmentsJoinEmployeesResults(joinType, rows); } } /** */ - private void checkDepJoinEmpResults(JoinRelType joinType, ArrayList results) { + private void checkDepartmentsJoinEmployeesResults(JoinRelType joinType, ArrayList results) { switch (joinType) { case LEFT: assertEquals(4, results.size()); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java index 9c0ead688e297..be6a7496e0104 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java @@ -18,6 +18,7 @@ import java.sql.Date; import java.util.LinkedHashMap; +import java.util.List; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; @@ -276,40 +277,42 @@ public void testIndexLoopJoin() { .check(); } - // TODO: hash join /** */ @Test - public void testMergeJoin() { - assertQuery("" + - "SELECT /*+ " + HintDefinition.MERGE_JOIN + " */ d1.name, d2.name FROM Developer d1, Developer d2 " + - "WHERE d1.depId = d2.depId") - .matches(containsSubPlan("IgniteMergeJoin")) - .returns("Bach", "Bach") - .returns("Beethoven", "Beethoven") - .returns("Beethoven", "Strauss") - .returns("Mozart", "Mozart") - .returns("Strauss", "Strauss") - .returns("Strauss", "Beethoven") - .returns("Vagner", "Vagner") - .returns("Chaikovsky", "Chaikovsky") - .returns("Verdy", "Verdy") - .returns("Stravinsky", "Stravinsky") - .returns("Rahmaninov", "Rahmaninov") - .returns("Shubert", "Shubert") - .returns("Glinka", "Glinka") - .returns("Arnalds", "Arnalds") - .returns("Glass", "Glass") - .returns("O'Halloran", "O'Halloran") - .returns("Prokofiev", "Prokofiev") - .returns("Yiruma", "Yiruma") - .returns("Cacciapaglia", "Cacciapaglia") - .returns("Einaudi", "Einaudi") - .returns("Hasaishi", "Hasaishi") - .returns("Marradi", "Marradi") - .returns("Musorgskii", "Musorgskii") - .returns("Rihter", "Rihter") - .returns("Zimmer", "Zimmer") - .check(); + public void testMergeAndHashJoin() { + for (List params : F.asList(F.asList(HintDefinition.MERGE_JOIN.name(), "IgniteMergeJoin"), + F.asList(HintDefinition.HASH_JOIN.name(), "IgniteHashJoin"))) { + assertQuery("" + + "SELECT /*+ " + params.get(0) + " */ d1.name, d2.name FROM Developer d1, Developer d2 " + + "WHERE d1.depId = d2.depId") + .matches(containsSubPlan(params.get(1))) + .returns("Bach", "Bach") + .returns("Beethoven", "Beethoven") + .returns("Beethoven", "Strauss") + .returns("Mozart", "Mozart") + .returns("Strauss", "Strauss") + .returns("Strauss", "Beethoven") + .returns("Vagner", "Vagner") + .returns("Chaikovsky", "Chaikovsky") + .returns("Verdy", "Verdy") + .returns("Stravinsky", "Stravinsky") + .returns("Rahmaninov", "Rahmaninov") + .returns("Shubert", "Shubert") + .returns("Glinka", "Glinka") + .returns("Arnalds", "Arnalds") + .returns("Glass", "Glass") + .returns("O'Halloran", "O'Halloran") + .returns("Prokofiev", "Prokofiev") + .returns("Yiruma", "Yiruma") + .returns("Cacciapaglia", "Cacciapaglia") + .returns("Einaudi", "Einaudi") + .returns("Hasaishi", "Hasaishi") + .returns("Marradi", "Marradi") + .returns("Musorgskii", "Musorgskii") + .returns("Rihter", "Rihter") + .returns("Zimmer", "Zimmer") + .check(); + } } // ===== No filter ===== diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java index f3c5a1f32ad24..167480432754f 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/MemoryQuotasIntegrationTest.java @@ -24,6 +24,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.SqlConfiguration; import org.apache.ignite.internal.processors.query.calcite.QueryChecker; +import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Test; @@ -236,17 +237,18 @@ public void testTableSpoolNode() { /** */ @Test - public void testJoins() { + public void testRightMeterializedJoins() { sql("CREATE TABLE tbl2 (id INT, b VARBINARY) WITH TEMPLATE=PARTITIONED"); for (int i = 0; i < 800; i++) sql("INSERT INTO tbl2 VALUES (?, ?)", i, new byte[1000]); - List> params = F.asList(F.asList("NL_JOIN", "NestedLoopJoin"), F.asList("HASH_JOIN", "IgniteHashJoin")); + List> params = F.asList(F.asList(HintDefinition.NL_JOIN.name(), "NestedLoopJoin"), + F.asList(HintDefinition.HASH_JOIN.name(), "IgniteHashJoin")); - for (List paramSet : params) { - assertQuery("SELECT /*+ " + paramSet.get(0) + " */ tbl.id, tbl.b, tbl2.id, tbl2.b FROM tbl JOIN tbl2 USING (id)") - .matches(QueryChecker.containsSubPlan(paramSet.get(1))) + for (List params0 : params) { + assertQuery("SELECT /*+ " + params0.get(0) + " */ tbl.id, tbl.b, tbl2.id, tbl2.b FROM tbl JOIN tbl2 USING (id)") + .matches(QueryChecker.containsSubPlan(params0.get(1))) .resultSize(800) .check(); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java index c621ba7c3994e..41c8ab0b24071 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema; import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.jetbrains.annotations.Nullable; import org.junit.Test; @@ -43,10 +44,7 @@ public class HashJoinPlannerTest extends AbstractPlannerTest { "JoinCommuteRule"}; /** */ - private static final String[] JOIN_TYPES = {"LEFT", "RIGHT", "INNER", "FULL OUTER"}; - - /** */ - private static int tableId = 1; + private static final String[] JOIN_TYPES = {"LEFT", "RIGHT", "INNER", "FULL"}; /** */ @Test @@ -132,10 +130,11 @@ public void testHashJoinWinsOnSkewedLeftInput() throws Exception { /** */ @Test public void testHashJoinApplied() throws Exception { - for (List paramSet : joinAppliedParameters()) { + for (List paramSet : testJoinIsAppliedParameters()) { assert paramSet != null && paramSet.size() == 2; String sql = (String)paramSet.get(0); + boolean canBePlanned = (Boolean)paramSet.get(1); TestTable tbl = createTable("T1", IgniteDistributions.single(), "ID", Integer.class, "C1", Integer.class); @@ -156,7 +155,7 @@ public void testHashJoinApplied() throws Exception { } /** */ - private static List> joinAppliedParameters() { + private static List> testJoinIsAppliedParameters() { return F.asList( F.asList("select t1.c1 from t1 %s join t1 t2 using(c1)", true), F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1", true), @@ -180,7 +179,7 @@ private static TestTable createSimpleTable(String name, int size) { return createTable( name, size, - IgniteDistributions.affinity(0, ++tableId, 0), + IgniteDistributions.affinity(0, CU.cacheId("default"), 0), "ID", Integer.class, "INT_VAL", Integer.class, "STR_VAL", String.class @@ -195,7 +194,7 @@ private static TestTable createComplexTable() { return createTable( "TEST_TBL_CMPLX", DEFAULT_TBL_SIZE, - IgniteDistributions.affinity(ImmutableIntList.of(0, 1), ++tableId, 0), + IgniteDistributions.affinity(ImmutableIntList.of(0, 1), CU.cacheId("default"), 0), "ID1", Integer.class, "ID2", Integer.class, "STR_VAL", String.class diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java index 932a6e1bd2b34..f8b41dc0a1ec0 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java @@ -44,7 +44,7 @@ */ public class JoinColocationPlannerTest extends AbstractPlannerTest { /** */ - private static String[] DISABLED_RULES = new String[] {"HashJoinConverter", "MergeJoinConverter"}; + private static final String[] DISABLED_RULES = new String[] {"HashJoinConverter", "MergeJoinConverter"}; /** * Join of the same tables with a simple affinity is expected to be colocated. diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java index f5207009a7108..95ea557b5a6f1 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java @@ -94,8 +94,8 @@ public void testOuterCommute() throws Exception { assertEquals(JoinRelType.LEFT, join.getJoinType()); - PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", - "CorrelatedNestedLoopJoin"); + PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", + "HashJoinConverter"); RelOptPlanner pl = ctx.cluster().getPlanner(); @@ -134,7 +134,8 @@ public void testOuterCommute() throws Exception { public void testInnerCommute() throws Exception { String sql = "SELECT COUNT(*) FROM SMALL s JOIN HUGE h on h.id = s.id"; - IgniteRel phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin"); + IgniteRel phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", + "HashJoinConverter"); assertNotNull(phys); @@ -161,8 +162,8 @@ public void testInnerCommute() throws Exception { assertEquals(JoinRelType.INNER, join.getJoinType()); - PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", - "CorrelatedNestedLoopJoin"); + PlanningContext ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", + "HashJoinConverter"); RelOptPlanner pl = ctx.cluster().getPlanner(); @@ -172,7 +173,7 @@ public void testInnerCommute() throws Exception { assertNotNull(phys); - phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin", + phys = physicalPlan(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", "HashJoinConverter", "JoinCommuteRule"); join = findFirstNode(phys, byClass(IgniteNestedLoopJoin.class)); @@ -199,7 +200,7 @@ public void testInnerCommute() throws Exception { // no commute assertEquals(JoinRelType.INNER, join.getJoinType()); - ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "HashJoinConverter", "CorrelatedNestedLoopJoin", + ctx = plannerCtx(sql, publicSchema, "MergeJoinConverter", "CorrelatedNestedLoopJoin", "HashJoinConverter", "JoinCommuteRule"); pl = ctx.cluster().getPlanner(); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java index c0bbb910e5e44..9116092c0d7db 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortAggregatePlannerTest.java @@ -94,7 +94,6 @@ public void testNoSortAppendingWithCorrectCollation() throws Exception { publicSchema, F.concat(HASH_AGG_RULES, "NestedLoopJoinConverter", - "HashConverter", "CorrelatedNestedLoopJoin", "CorrelateToNestedLoopRule" ) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java index 82c2f4fa471ed..4f64ff3aa2ab6 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java @@ -157,7 +157,7 @@ public void testPartialIndexForCondition() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "HashJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" ); System.out.println("+++ \n" + RelOptUtil.toString(phys)); From e43c971f8709033c2a22af097be5f8592d7518ab Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 24 Dec 2024 14:03:30 +0300 Subject: [PATCH 04/28] Minorities --- .../query/calcite/planner/HashIndexSpoolPlannerTest.java | 2 +- .../query/calcite/planner/SortedIndexSpoolPlannerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java index 1d7f2e7438416..d83844f183bb2 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashIndexSpoolPlannerTest.java @@ -84,7 +84,7 @@ public void testSingleKey() throws Exception { IgniteRel phys = physicalPlan( sql, publicSchema, - "MergeJoinConverter", "HashJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToSortedIndexSpoolRule" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter", "FilterSpoolMergeToSortedIndexSpoolRule" ); System.out.println("+++\n" + RelOptUtil.toString(phys)); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java index 4f64ff3aa2ab6..63ed1c6a8d6b5 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SortedIndexSpoolPlannerTest.java @@ -252,7 +252,7 @@ public void testRestoreCollation() throws Exception { .and(input(1, isInstanceOf(IgniteSortedIndexSpool.class) .and(spool -> spool.collation().getFieldCollations().get(0).getFieldIndex() == equalIdx) ))), - "MergeJoinConverter", "HashJoinConverter", "NestedLoopJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" + "MergeJoinConverter", "NestedLoopJoinConverter", "HashJoinConverter", "FilterSpoolMergeToHashIndexSpoolRule" ); } } From 91c9fecd6c2fb09e7eebc41bb3c9937855b1a42b Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Fri, 6 Jun 2025 22:54:07 +0300 Subject: [PATCH 05/28] fix --- .../query/calcite/exec/rel/HashJoinNode.java | 53 +++- .../exec/rel/AbstractExecutionTest.java | 11 + .../query/calcite/exec/rel/ExecutionTest.java | 2 +- .../exec/rel/HashJoinExecutionTest.java | 279 ++++++++++++++++++ 4 files changed, 334 insertions(+), 11 deletions(-) create mode 100644 modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index dd426ddebdc00..2c894080e58bd 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -259,12 +259,14 @@ private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { + // Proceed with next left row, if previous was fully processed. if (!rightIt.hasNext()) { left = leftInBuf.remove(); rightIt = lookup(left).iterator(); } + // Emits matched rows. while (rightIt.hasNext()) { checkState(); @@ -326,6 +328,7 @@ private LeftHashJoin( while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { checkState(); + // Proceed with next left row, if previous was fully processed. if (!rightIt.hasNext()) { left = leftInBuf.remove(); @@ -340,6 +343,7 @@ private LeftHashJoin( rightIt = rightRows.iterator(); } + // Emit unmatched left row. while (rightIt.hasNext()) { checkState(); @@ -371,6 +375,9 @@ private static final class RightHashJoin extends HashJoinNode { /** Left row factory. */ private final RowHandler.RowFactory leftRowFactory; + /** */ + private boolean drainMaterialization; + /** * Creates node for RIGHT OUTER JOIN. * @@ -401,12 +408,14 @@ private RightHashJoin( while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { checkState(); + // Proceed with next left row, if previous was fully processed. if (!rightIt.hasNext()) { left = leftInBuf.remove(); rightIt = lookup(left).iterator(); } + // Emits matched rows. while (rightIt.hasNext()) { checkState(); @@ -429,12 +438,17 @@ private RightHashJoin( } } + // Emit unmatched right rows. if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; try { - if (!rightIt.hasNext()) + if (!rightIt.hasNext() && !drainMaterialization) { + // Prevent scanning store more than once. + drainMaterialization = true; + rightIt = untouched(hashStore); + } RowT emptyLeft = leftRowFactory.create(); @@ -465,6 +479,13 @@ private RightHashJoin( @Override protected boolean keepRowsWithNull() { return true; } + + /** {@inheritDoc} */ + @Override protected void rewindInternal() { + drainMaterialization = false; + + super.rewindInternal(); + } } /** */ @@ -475,6 +496,9 @@ private static class FullOuterHashJoin extends HashJoinNode { /** Right row factory. */ private final RowHandler.RowFactory rightRowFactory; + /** */ + private boolean drainMaterialization; + /** * Creates node for FULL OUTER JOIN. * @@ -508,11 +532,13 @@ private FullOuterHashJoin( while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { checkState(); + // Proceed with next left row, if previous was fully processed. if (!rightIt.hasNext()) { left = leftInBuf.remove(); Collection rightRows = lookup(left); + // Emit unmatched left row. if (rightRows.isEmpty()) { requested--; @@ -522,6 +548,7 @@ private FullOuterHashJoin( rightIt = rightRows.iterator(); } + // Emits matched rows. while (rightIt.hasNext()) { checkState(); @@ -545,13 +572,17 @@ private FullOuterHashJoin( } } - if (left == null && !rightIt.hasNext() && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING - && requested > 0) { + // Emit unmatched right rows.Add commentMore actions + if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; try { - if (!rightIt.hasNext()) + if (!rightIt.hasNext() && !drainMaterialization) { + // Prevent scanning store more than once. + drainMaterialization = true; + rightIt = untouched(hashStore); + } RowT emptyLeft = leftRowFactory.create(); @@ -580,6 +611,13 @@ private FullOuterHashJoin( @Override protected boolean keepRowsWithNull() { return true; } + + /** {@inheritDoc} */ + @Override protected void rewindInternal() { + drainMaterialization = false; + + super.rewindInternal(); + } } /** */ @@ -609,13 +647,11 @@ private SemiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo i Collection rightRows = lookup(left); + // Emit matched row. if (!rightRows.isEmpty()) { requested--; downstream().push(left); - - if (requested == 0) - break; } left = null; @@ -661,9 +697,6 @@ private AntiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo i requested--; downstream().push(left); - - if (requested == 0) - break; } left = null; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java index de24b8de4d0d0..791c33815e331 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -328,6 +329,16 @@ protected QueryTaskExecutor taskExecutor(UUID nodeId) { return taskExecutors.get(nodeId); } + /** */ + protected ExecutionContext executionContext() { + return executionContext(nodes.get(new Random().nextInt(nodesCnt)), UUID.randomUUID(), 0); + } + + /** */ + protected ExecutionContext executionContext(UUID nodeId) { + return executionContext(nodeId, UUID.randomUUID(), 0); + } + /** */ protected ExecutionContext executionContext(UUID nodeId, UUID qryId, long fragmentId) { FragmentDescription fragmentDesc = new FragmentDescription(fragmentId, null, null, null); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java index 970ae80227511..70f7512f992b1 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java @@ -26,7 +26,7 @@ import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; -import org.apache.ignite.IgniteCheckedException; +import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java new file mode 100644 index 0000000000000..8ebc22679013d --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java @@ -0,0 +1,279 @@ +/* + * 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.ignite.internal.processors.query.calcite.exec.rel; + +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; +import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; +import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.junit.Test; + +import static org.apache.calcite.rel.core.JoinRelType.ANTI; +import static org.apache.calcite.rel.core.JoinRelType.FULL; +import static org.apache.calcite.rel.core.JoinRelType.RIGHT; +import static org.apache.calcite.rel.core.JoinRelType.SEMI; + +/** */ +@SuppressWarnings("TypeMayBeWeakened") +public class HashJoinExecutionTest extends AbstractExecutionTest { + /** */ + @Test + public void testEquiJoinWithDifferentBufferSize() { + for (JoinRelType joinType : JoinRelType.values()) { + validateEquiJoin(executionContext(), joinType, 0, 0); + validateEquiJoin(executionContext(), joinType, 0, 1); + validateEquiJoin(executionContext(), joinType, 0, 10); + validateEquiJoin(executionContext(), joinType, 1, 0); + validateEquiJoin(executionContext(), joinType, 1, 1); + validateEquiJoin(executionContext(), joinType, 1, 10); + validateEquiJoin(executionContext(), joinType, 10, 0); + validateEquiJoin(executionContext(), joinType, 10, 1); + validateEquiJoin(executionContext(), joinType, 10, 10); + + int testSize = IN_BUFFER_SIZE; + + validateEquiJoin(executionContext(), joinType, 0, testSize - 1); + validateEquiJoin(executionContext(), joinType, 0, testSize); + validateEquiJoin(executionContext(), joinType, 0, testSize + 1); + + validateEquiJoin(executionContext(), joinType, testSize - 1, 0); + validateEquiJoin(executionContext(), joinType, testSize - 1, testSize - 1); + validateEquiJoin(executionContext(), joinType, testSize - 1, testSize); + validateEquiJoin(executionContext(), joinType, testSize - 1, testSize + 1); + + validateEquiJoin(executionContext(), joinType, testSize, 0); + validateEquiJoin(executionContext(), joinType, testSize, testSize - 1); + validateEquiJoin(executionContext(), joinType, testSize, testSize); + validateEquiJoin(executionContext(), joinType, testSize, testSize + 1); + + validateEquiJoin(executionContext(), joinType, testSize + 1, 0); + validateEquiJoin(executionContext(), joinType, testSize + 1, testSize - 1); + validateEquiJoin(executionContext(), joinType, testSize + 1, testSize); + validateEquiJoin(executionContext(), joinType, testSize + 1, testSize + 1); + + validateEquiJoin(executionContext(), joinType, 2 * testSize, 0); + validateEquiJoin(executionContext(), joinType, 0, 2 * testSize); + validateEquiJoin(executionContext(), joinType, 2 * testSize, 2 * testSize); + } + } + +// /** */ +// @Test +// public void nonEquiJoinWithDifferentBufferSize() { +// for (JoinRelType joinType : F.asList(INNER, SEMI)) { +// validateNonEquiJoin(executionContext(), joinType, 0, 0); +// validateNonEquiJoin(executionContext(), joinType, 0, 1); +// validateNonEquiJoin(executionContext(), joinType, 0, 10); +// validateNonEquiJoin(executionContext(), joinType, 1, 0); +// validateNonEquiJoin(executionContext(), joinType, 1, 1); +// validateNonEquiJoin(executionContext(), joinType, 1, 10); +// validateNonEquiJoin(executionContext(), joinType, 10, 0); +// validateNonEquiJoin(executionContext(), joinType, 10, 1); +// validateNonEquiJoin(executionContext(), joinType, 10, 10); +// +// int testSize = IN_BUFFER_SIZE; +// +// validateNonEquiJoin(executionContext(), joinType, 0, testSize - 1); +// validateNonEquiJoin(executionContext(), joinType, 0, testSize); +// validateNonEquiJoin(executionContext(), joinType, 0, testSize + 1); +// +// validateNonEquiJoin(executionContext(), joinType, testSize - 1, 0); +// validateNonEquiJoin(executionContext(), joinType, testSize - 1, testSize - 1); +// validateNonEquiJoin(executionContext(), joinType, testSize - 1, testSize); +// validateNonEquiJoin(executionContext(), joinType, testSize - 1, testSize + 1); +// +// validateNonEquiJoin(executionContext(), joinType, testSize, 0); +// validateNonEquiJoin(executionContext(), joinType, testSize, testSize - 1); +// validateNonEquiJoin(executionContext(), joinType, testSize, testSize); +// validateNonEquiJoin(executionContext(), joinType, testSize, testSize + 1); +// +// validateNonEquiJoin(executionContext(), joinType, testSize + 1, 0); +// validateNonEquiJoin(executionContext(), joinType, testSize + 1, testSize - 1); +// validateNonEquiJoin(executionContext(), joinType, testSize + 1, testSize); +// validateNonEquiJoin(executionContext(), joinType, testSize + 1, testSize + 1); +// +// validateNonEquiJoin(executionContext(), joinType, 2 * testSize, 0); +// validateNonEquiJoin(executionContext(), joinType, 0, 2 * testSize); +// validateNonEquiJoin(executionContext(), joinType, 2 * testSize, 2 * testSize); +// } +// } + + /** */ + private void validateEquiJoin(ExecutionContext ctx, JoinRelType joinType, int leftSize, int rightSize) { + if (log.isInfoEnabled()) { + log.info("Testing eq. join with different buffer sizes. Join type: " + joinType + ", leftSz: " + + leftSize + ", rightSz: " + rightSize); + } + + IgniteTypeFactory tf = ctx.getTypeFactory(); + + { // Distinct inputs + Object[] person = {1, "name", 2}; + Object[] department = {1, "department"}; + + int resultSize = estimateResultSizeForDistinctInputs(joinType, leftSize, rightSize); + + RelDataType leftType = TypeUtils.createRowType(tf, Integer.class, String.class, Integer.class); + RelDataType righType = TypeUtils.createRowType(tf, Integer.class, String.class); + + validate( + ctx, + joinType, + () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), + leftType, + () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), + righType, + resultSize + ); + } + +// { // Matching inputs +// Object[] person = {1, "name", 2}; +// Object[] department = {2, "department"}; +// +// int resultSize = estimateResultSizeForEqualInputs(joinType, leftSize, rightSize); +// +// validate( +// ctx, +// joinType, +// () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), +// () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), +// resultSize +// ); +// } + } + +// /** */ +// protected void validateNonEquiJoin(ExecutionContext ctx, JoinRelType joinType, int leftSize, int rightSize) { +// Object[] person = {1, "name", 2}; +// Object[] department = {2, "department"}; +// +// int resultSize = estimateResultSizeForEqualInputs(joinType, leftSize, rightSize); +// +// validate( +// ctx, +// joinType, +// () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), +// () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), +// resultSize +// ); +// +// validate( +// ctx, +// joinType, +// () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), +// () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), +// 0 +// ); +// } + + /** */ + private static int estimateResultSizeForDistinctInputs(JoinRelType joinType, int leftSize, int rightSize) { + switch (joinType) { + case SEMI: // Fallthrough + case INNER: + return 0; + case ANTI: // Fallthrough + case LEFT: + return leftSize; + case RIGHT: + return rightSize; + case FULL: + return leftSize + rightSize; + default: + throw new IllegalArgumentException("Unsupported join type: " + joinType); + } + } + + /** */ + private static int estimateResultSizeForEqualInputs( + JoinRelType joinType, + int leftSize, + int rightSize + ) { + switch (joinType) { + case SEMI: + return rightSize == 0 ? 0 : leftSize; + case ANTI: + return rightSize == 0 ? leftSize : 0; + case LEFT: + return rightSize == 0 ? leftSize : leftSize * rightSize; + case RIGHT: + return leftSize == 0 ? rightSize : leftSize * rightSize; + case FULL: + return leftSize == 0 ? rightSize : rightSize == 0 ? leftSize : leftSize * rightSize; + case INNER: + return leftSize * rightSize; + default: + throw new IllegalArgumentException("Unsupported join type: " + joinType); + } + } + + /** */ + private void validate( + ExecutionContext ctx, + JoinRelType joinType, + Iterable leftSrc, + RelDataType leftType, + Iterable rightSrc, + RelDataType rightType, + int resultSize + ) { + ScanNode left = new ScanNode<>(ctx, leftType, leftSrc); + ScanNode right = new ScanNode<>(ctx, rightType, rightSrc); + + AbstractRightMaterializedJoinNode join = createJoinNode(ctx, joinType); + + join.register(F.asList(left, right)); + + RootNode node = new RootNode<>(ctx, join.rowType()); + + node.register(join); + + long cnt = StreamSupport.stream(Spliterators.spliteratorUnknownSize(node, Spliterator.ORDERED), false).count(); + + assertEquals(resultSize, cnt); + } + + /** */ + private static HashJoinNode createJoinNode(ExecutionContext ctx, JoinRelType joinType) { + IgniteTypeFactory tf = ctx.getTypeFactory(); + + RelDataType leftType = TypeUtils.createRowType(tf, tf.createSqlType(SqlTypeName.INTEGER), + tf.createSqlType(SqlTypeName.VARCHAR)); + + RelDataType rightType = TypeUtils.createRowType(tf, tf.createSqlType(SqlTypeName.INTEGER), + tf.createSqlType(SqlTypeName.VARCHAR), tf.createSqlType(SqlTypeName.INTEGER)); + + RelDataType outType = (joinType == ANTI || joinType == SEMI) + ? leftType + : TypeUtils.combinedRowType(tf, leftType, rightType); + + return HashJoinNode.create(ctx, outType, leftType, rightType, joinType, + JoinInfo.of(ImmutableIntList.of(2), ImmutableIntList.of(0))); + } +} From 72987eac367153ab62e955e050d42a0b8ad5ce6b Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 14 Jun 2025 19:17:23 +0300 Subject: [PATCH 06/28] +IGNITE-24675, +IGNITE-23970, +IGNITE-24050 --- .../calcite/exec/LogicalRelImplementor.java | 20 +- .../query/calcite/exec/rel/HashJoinNode.java | 109 ++++- .../calcite/rule/HashJoinConverterRule.java | 45 +- .../query/calcite/exec/rel/ExecutionTest.java | 2 +- .../exec/rel/HashJoinExecutionTest.java | 411 ++++++++++++------ .../calcite/planner/HashJoinPlannerTest.java | 46 +- .../query/calcite/planner/PlannerTest.java | 4 +- 7 files changed, 430 insertions(+), 207 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java index 60e1270648a23..64e64eaea8011 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java @@ -37,6 +37,7 @@ import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexUtil; import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.query.QueryUtils; @@ -273,8 +274,23 @@ public LogicalRelImplementor( /** {@inheritDoc} */ @Override public Node visit(IgniteHashJoin rel) { - Node node = HashJoinNode.create(ctx, rel.getRowType(), rel.getLeft().getRowType(), rel.getRight().getRowType(), - rel.getJoinType(), rel.analyzeCondition()); + RelDataType outType = rel.getRowType(); + RelDataType leftType = rel.getLeft().getRowType(); + RelDataType rightType = rel.getRight().getRowType(); + JoinRelType joinType = rel.getJoinType(); + + RexNode nonEquiConditionExpression = RexUtil.composeConjunction(Commons.emptyCluster().getRexBuilder(), + rel.analyzeCondition().nonEquiConditions, true); + + BiPredicate nonEquiCondition = null; + + if (nonEquiConditionExpression != null) { + RelDataType rowType = combinedRowType(ctx.getTypeFactory(), leftType, rightType); + + nonEquiCondition = expressionFactory.biPredicate(rel.getCondition(), rowType); + } + + Node node = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, rel.analyzeCondition(), nonEquiCondition); node.register(Arrays.asList(visit(rel.getLeft()), visit(rel.getRight()))); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 2c894080e58bd..ed569bec41098 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -25,12 +25,14 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.function.BiPredicate; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; +import org.jetbrains.annotations.Nullable; /** Hash join implementor. */ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNode { @@ -55,6 +57,9 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod /** */ protected Iterator rightIt = Collections.emptyIterator(); + /** */ + @Nullable protected final BiPredicate nonEqCond; + /** * Creates hash join node. * @@ -62,8 +67,15 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod * @param rowType Row type. * @param info Join info. * @param outRowHnd Output row handler. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ - protected HashJoinNode(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { + protected HashJoinNode( + ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + @Nullable BiPredicate nonEqCond + ) { super(ctx, rowType); leftKeys = info.leftKeys.toIntArray(); @@ -72,6 +84,8 @@ protected HashJoinNode(ExecutionContext ctx, RelDataType rowType, JoinInfo assert leftKeys.length == rightKeys.length; this.outRowHnd = outRowHnd; + + this.nonEqCond = nonEqCond; } /** {@inheritDoc} */ @@ -90,30 +104,31 @@ public static HashJoinNode create( RelDataType leftRowType, RelDataType rightRowType, JoinRelType type, - JoinInfo info + JoinInfo info, + @Nullable BiPredicate nonEqCond ) { IgniteTypeFactory typeFactory = ctx.getTypeFactory(); RowHandler rowHnd = ctx.rowHandler(); switch (type) { case INNER: - return new InnerHashJoin<>(ctx, outRowType, info, rowHnd); + return new InnerHashJoin<>(ctx, outRowType, info, rowHnd, nonEqCond); case LEFT: - return new LeftHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType)); + return new LeftHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType), nonEqCond); case RIGHT: - return new RightHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType)); + return new RightHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), nonEqCond); case FULL: return new FullOuterHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), - rowHnd.factory(typeFactory, rightRowType)); + rowHnd.factory(typeFactory, rightRowType), nonEqCond); case SEMI: - return new SemiHashJoin<>(ctx, outRowType, info, rowHnd); + return new SemiHashJoin<>(ctx, outRowType, info, rowHnd, nonEqCond); case ANTI: - return new AntiHashJoin<>(ctx, outRowType, info, rowHnd); + return new AntiHashJoin<>(ctx, outRowType, info, rowHnd, nonEqCond); default: throw new IllegalArgumentException("Join of type '" + type + "' isn't supported."); @@ -224,6 +239,8 @@ protected void requestMoreOrEnd() throws Exception { && !rightIt.hasNext()) { requested = 0; + hashStore.clear(); + downstream().end(); } } @@ -247,9 +264,15 @@ private static final class InnerHashJoin extends HashJoinNode { * @param rowType Row type. * @param info Join info. * @param outRowHnd Output row handler. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ - private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { - super(ctx, rowType, info, outRowHnd); + private InnerHashJoin(ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + @Nullable BiPredicate nonEqCond + ) { + super(ctx, rowType, info, outRowHnd, nonEqCond); } /** {@inheritDoc} */ @@ -272,6 +295,9 @@ private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo RowT right = rightIt.next(); + if (nonEqCond != null && !nonEqCond.test(left, right)) + continue; + --requested; downstream().push(outRowHnd.concat(left, right)); @@ -306,15 +332,19 @@ private static final class LeftHashJoin extends HashJoinNode { * @param rowType Row tyoe. * @param outRowHnd Output row handler. * @param rightRowFactory Right row factory. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private LeftHashJoin( ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd, - RowHandler.RowFactory rightRowFactory + RowHandler.RowFactory rightRowFactory, + @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd); + super(ctx, rowType, info, outRowHnd, nonEqCond); + + assert nonEqCond == null : "Non equi condition is not supported in LEFT join"; this.rightRowFactory = rightRowFactory; } @@ -386,15 +416,19 @@ private static final class RightHashJoin extends HashJoinNode { * @param info Join info. * @param outRowHnd Output row handler. * @param leftRowFactory Left row factory. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private RightHashJoin( ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd, - RowHandler.RowFactory leftRowFactory + RowHandler.RowFactory leftRowFactory, + @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd); + super(ctx, rowType, info, outRowHnd, nonEqCond); + + assert nonEqCond == null : "Non equi condition is not supported in RIGHT join"; this.leftRowFactory = leftRowFactory; } @@ -508,6 +542,7 @@ private static class FullOuterHashJoin extends HashJoinNode { * @param outRowHnd Output row handler. * @param leftRowFactory Left row factory. * @param rightRowFactory Right row factory. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private FullOuterHashJoin( ExecutionContext ctx, @@ -515,9 +550,12 @@ private FullOuterHashJoin( JoinInfo info, RowHandler outRowHnd, RowHandler.RowFactory leftRowFactory, - RowHandler.RowFactory rightRowFactory + RowHandler.RowFactory rightRowFactory, + @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd); + super(ctx, rowType, info, outRowHnd, nonEqCond); + + assert nonEqCond == null : "Non equi condition is not supported in FULL OUTER join"; this.leftRowFactory = leftRowFactory; this.rightRowFactory = rightRowFactory; @@ -629,9 +667,16 @@ private static final class SemiHashJoin extends HashJoinNode { * @param rowType Row type. * @param info Join info. * @param outRowHnd Output row handler. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ - private SemiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { - super(ctx, rowType, info, outRowHnd); + private SemiHashJoin( + ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + @Nullable BiPredicate nonEqCond + ) { + super(ctx, rowType, info, outRowHnd, nonEqCond); } /** {@inheritDoc} */ @@ -647,8 +692,21 @@ private SemiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo i Collection rightRows = lookup(left); - // Emit matched row. - if (!rightRows.isEmpty()) { + boolean anyMatched = !rightRows.isEmpty(); + + if (anyMatched && nonEqCond != null) { + anyMatched = false; + + for (RowT right : rightRows) { + if (nonEqCond.test(left, right)) { + anyMatched = true; + + break; + } + } + } + + if (anyMatched) { requested--; downstream().push(left); @@ -675,9 +733,16 @@ private static final class AntiHashJoin extends HashJoinNode { * @param rowType Row type. * @param info Join info. * @param outRowHnd Output row handler. + * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ - private AntiHashJoin(ExecutionContext ctx, RelDataType rowType, JoinInfo info, RowHandler outRowHnd) { - super(ctx, rowType, info, outRowHnd); + private AntiHashJoin( + ExecutionContext ctx, + RelDataType rowType, + JoinInfo info, + RowHandler outRowHnd, + @Nullable BiPredicate nonEqCond + ) { + super(ctx, rowType, info, outRowHnd, nonEqCond); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java index c628e60174a08..434b52f59f16c 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java @@ -17,21 +17,20 @@ package org.apache.ignite.internal.processors.query.calcite.rule; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.PhysicalNode; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexVisitor; -import org.apache.calcite.rex.RexVisitorImpl; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.util.Util; import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; @@ -42,8 +41,11 @@ public class HashJoinConverterRule extends AbstractIgniteJoinConverterRule { /** */ public static final RelOptRule INSTANCE = new HashJoinConverterRule(); + /** */ + private static final EnumSet NON_EQ_CONDITIONS_SUPPORT = EnumSet.of(JoinRelType.INNER, JoinRelType.SEMI); + /** Ctor. */ - public HashJoinConverterRule() { + private HashJoinConverterRule() { super("HashJoinConverter", HintDefinition.HASH_JOIN); } @@ -51,28 +53,21 @@ public HashJoinConverterRule() { @Override public boolean matchesJoin(RelOptRuleCall call) { LogicalJoin join = call.rel(0); - return !F.isEmpty(join.analyzeCondition().pairs()) && join.analyzeCondition().isEqui() && checkConditions(join.getCondition()); - } - - /** */ - private static boolean checkConditions(RexNode node) { - RexVisitor v = new RexVisitorImpl<>(true) { - @Override public Void visitCall(RexCall call) { - if (call.getOperator().getKind() != SqlKind.EQUALS && call.getOperator().getKind() != SqlKind.AND) - throw Util.FoundOne.NULL; + if (F.isEmpty(join.analyzeCondition().pairs())) + return false; - return super.visitCall(call); - } - }; + List filterNulls = new ArrayList<>(); - try { - node.accept(v); + RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(), join.getCondition(), new ArrayList<>(), + new ArrayList<>(), filterNulls); - return true; - } - catch (Util.FoundOne e) { + // IS NOT DISTINCT currently not supported by HashJoin + if (filterNulls.stream().anyMatch(filter -> !filter)) return false; - } + + // Current limitation: unmatched products on left or right part requires special handling of non-equi condition + // on execution level. + return join.analyzeCondition().isEqui() || NON_EQ_CONDITIONS_SUPPORT.contains(join.getJoinType()); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java index 70f7512f992b1..959b7f2df45d2 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java @@ -565,7 +565,7 @@ public void testHashJoin() { rowType = TypeUtils.createRowType(tf, int.class, String.class, String.class); HashJoinNode join = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, - JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(2))); + JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(2)), null); join.register(F.asList(leftDeps, rightEmps)); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java index 8ebc22679013d..063da256de94e 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java @@ -17,179 +17,290 @@ package org.apache.ignite.internal.processors.query.calcite.exec.rel; -import java.util.Spliterator; -import java.util.Spliterators; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.function.BiPredicate; import java.util.stream.IntStream; -import java.util.stream.StreamSupport; +import java.util.stream.Stream; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils; import org.apache.ignite.internal.util.typedef.F; +import org.jetbrains.annotations.Nullable; import org.junit.Test; import static org.apache.calcite.rel.core.JoinRelType.ANTI; -import static org.apache.calcite.rel.core.JoinRelType.FULL; -import static org.apache.calcite.rel.core.JoinRelType.RIGHT; +import static org.apache.calcite.rel.core.JoinRelType.INNER; +import static org.apache.calcite.rel.core.JoinRelType.LEFT; import static org.apache.calcite.rel.core.JoinRelType.SEMI; /** */ @SuppressWarnings("TypeMayBeWeakened") public class HashJoinExecutionTest extends AbstractExecutionTest { + /** */ + @Test + public void testHashJoinRewind() { + ExecutionContext ctx = executionContext(); + + RelDataType leftType = TypeUtils.createRowType(ctx.getTypeFactory(), Integer.class, String.class); + RelDataType rightType = TypeUtils.createRowType(ctx.getTypeFactory(), Integer.class, String.class, Integer.class); + + ScanNode deps = new ScanNode<>( + ctx, + rightType, + Arrays.asList( + new Object[] {1, "Core"}, + new Object[] {2, "SQL"}, + new Object[] {3, "QA"} + )); + + ScanNode persons = new ScanNode<>( + ctx, + leftType, + Arrays.asList( + new Object[] {0, "Igor", 1}, + new Object[] {1, "Roman", 2}, + new Object[] {2, "Ivan", 5}, + new Object[] {3, "Alexey", 1} + )); + + HashJoinNode join = createJoinNode(ctx, LEFT, leftType, rightType, null); + + join.register(F.asList(persons, deps)); + + ProjectNode project = new ProjectNode<>(ctx, join.rowType(), r -> new Object[] {r[2], r[3], r[1]}); + project.register(join); + + RootRewindable node = new RootRewindable<>(ctx, project.rowType()); + + node.register(project); + + assert node.hasNext(); + + ArrayList rows = new ArrayList<>(); + + while (node.hasNext()) + rows.add(node.next()); + + Object[][] expected = { + {1, 1, "Igor"}, + {1, 1, "Alexey"}, + {2, 2, "Roman"}, + {5, null, "Ivan"}, + }; + + checkResults(expected, rows); + + node.rewind(); + + assert node.hasNext(); + + rows.clear(); + + while (node.hasNext()) + rows.add(node.next()); + + checkResults(expected, rows); + } + /** */ @Test public void testEquiJoinWithDifferentBufferSize() { for (JoinRelType joinType : JoinRelType.values()) { - validateEquiJoin(executionContext(), joinType, 0, 0); - validateEquiJoin(executionContext(), joinType, 0, 1); - validateEquiJoin(executionContext(), joinType, 0, 10); - validateEquiJoin(executionContext(), joinType, 1, 0); - validateEquiJoin(executionContext(), joinType, 1, 1); - validateEquiJoin(executionContext(), joinType, 1, 10); - validateEquiJoin(executionContext(), joinType, 10, 0); - validateEquiJoin(executionContext(), joinType, 10, 1); - validateEquiJoin(executionContext(), joinType, 10, 10); + validateEquiJoin(joinType, 0, 0); + validateEquiJoin(joinType, 0, 1); + validateEquiJoin(joinType, 0, 10); + validateEquiJoin(joinType, 1, 0); + validateEquiJoin(joinType, 1, 1); + validateEquiJoin(joinType, 1, 10); + validateEquiJoin(joinType, 10, 0); + validateEquiJoin(joinType, 10, 1); + validateEquiJoin(joinType, 10, 10); int testSize = IN_BUFFER_SIZE; - validateEquiJoin(executionContext(), joinType, 0, testSize - 1); - validateEquiJoin(executionContext(), joinType, 0, testSize); - validateEquiJoin(executionContext(), joinType, 0, testSize + 1); + validateEquiJoin(joinType, 0, testSize - 1); + validateEquiJoin(joinType, 0, testSize); + validateEquiJoin(joinType, 0, testSize + 1); - validateEquiJoin(executionContext(), joinType, testSize - 1, 0); - validateEquiJoin(executionContext(), joinType, testSize - 1, testSize - 1); - validateEquiJoin(executionContext(), joinType, testSize - 1, testSize); - validateEquiJoin(executionContext(), joinType, testSize - 1, testSize + 1); + validateEquiJoin(joinType, testSize - 1, 0); + validateEquiJoin(joinType, testSize - 1, testSize - 1); + validateEquiJoin(joinType, testSize - 1, testSize); + validateEquiJoin(joinType, testSize - 1, testSize + 1); - validateEquiJoin(executionContext(), joinType, testSize, 0); - validateEquiJoin(executionContext(), joinType, testSize, testSize - 1); - validateEquiJoin(executionContext(), joinType, testSize, testSize); - validateEquiJoin(executionContext(), joinType, testSize, testSize + 1); + validateEquiJoin(joinType, testSize, 0); + validateEquiJoin(joinType, testSize, testSize - 1); + validateEquiJoin(joinType, testSize, testSize); + validateEquiJoin(joinType, testSize, testSize + 1); - validateEquiJoin(executionContext(), joinType, testSize + 1, 0); - validateEquiJoin(executionContext(), joinType, testSize + 1, testSize - 1); - validateEquiJoin(executionContext(), joinType, testSize + 1, testSize); - validateEquiJoin(executionContext(), joinType, testSize + 1, testSize + 1); + validateEquiJoin(joinType, testSize + 1, 0); + validateEquiJoin(joinType, testSize + 1, testSize - 1); + validateEquiJoin(joinType, testSize + 1, testSize); + validateEquiJoin(joinType, testSize + 1, testSize + 1); - validateEquiJoin(executionContext(), joinType, 2 * testSize, 0); - validateEquiJoin(executionContext(), joinType, 0, 2 * testSize); - validateEquiJoin(executionContext(), joinType, 2 * testSize, 2 * testSize); + validateEquiJoin(joinType, 2 * testSize, 0); + validateEquiJoin(joinType, 0, 2 * testSize); + validateEquiJoin(joinType, 2 * testSize, 2 * testSize); } } -// /** */ -// @Test -// public void nonEquiJoinWithDifferentBufferSize() { -// for (JoinRelType joinType : F.asList(INNER, SEMI)) { -// validateNonEquiJoin(executionContext(), joinType, 0, 0); -// validateNonEquiJoin(executionContext(), joinType, 0, 1); -// validateNonEquiJoin(executionContext(), joinType, 0, 10); -// validateNonEquiJoin(executionContext(), joinType, 1, 0); -// validateNonEquiJoin(executionContext(), joinType, 1, 1); -// validateNonEquiJoin(executionContext(), joinType, 1, 10); -// validateNonEquiJoin(executionContext(), joinType, 10, 0); -// validateNonEquiJoin(executionContext(), joinType, 10, 1); -// validateNonEquiJoin(executionContext(), joinType, 10, 10); -// -// int testSize = IN_BUFFER_SIZE; -// -// validateNonEquiJoin(executionContext(), joinType, 0, testSize - 1); -// validateNonEquiJoin(executionContext(), joinType, 0, testSize); -// validateNonEquiJoin(executionContext(), joinType, 0, testSize + 1); -// -// validateNonEquiJoin(executionContext(), joinType, testSize - 1, 0); -// validateNonEquiJoin(executionContext(), joinType, testSize - 1, testSize - 1); -// validateNonEquiJoin(executionContext(), joinType, testSize - 1, testSize); -// validateNonEquiJoin(executionContext(), joinType, testSize - 1, testSize + 1); -// -// validateNonEquiJoin(executionContext(), joinType, testSize, 0); -// validateNonEquiJoin(executionContext(), joinType, testSize, testSize - 1); -// validateNonEquiJoin(executionContext(), joinType, testSize, testSize); -// validateNonEquiJoin(executionContext(), joinType, testSize, testSize + 1); -// -// validateNonEquiJoin(executionContext(), joinType, testSize + 1, 0); -// validateNonEquiJoin(executionContext(), joinType, testSize + 1, testSize - 1); -// validateNonEquiJoin(executionContext(), joinType, testSize + 1, testSize); -// validateNonEquiJoin(executionContext(), joinType, testSize + 1, testSize + 1); -// -// validateNonEquiJoin(executionContext(), joinType, 2 * testSize, 0); -// validateNonEquiJoin(executionContext(), joinType, 0, 2 * testSize); -// validateNonEquiJoin(executionContext(), joinType, 2 * testSize, 2 * testSize); -// } -// } + /** */ + @Test + public void testNonEquiJoinWithDifferentBufferSize() { + JoinRelType joinType = INNER; + + validateNonEquiJoin(joinType, 0, 0); + validateNonEquiJoin(joinType, 0, 1); + validateNonEquiJoin(joinType, 0, 10); + validateNonEquiJoin(joinType, 1, 0); + validateNonEquiJoin(joinType, 1, 1); + validateNonEquiJoin(joinType, 1, 10); + validateNonEquiJoin(joinType, 10, 0); + validateNonEquiJoin(joinType, 10, 1); + validateNonEquiJoin(joinType, 10, 10); + + int testSize = IN_BUFFER_SIZE; + + validateNonEquiJoin(joinType, 0, testSize - 1); + validateNonEquiJoin(joinType, 0, testSize); + validateNonEquiJoin(joinType, 0, testSize + 1); + + validateNonEquiJoin(joinType, testSize - 1, 0); + validateNonEquiJoin(joinType, testSize - 1, testSize - 1); + validateNonEquiJoin(joinType, testSize - 1, testSize); + validateNonEquiJoin(joinType, testSize - 1, testSize + 1); + + validateNonEquiJoin(joinType, testSize, 0); + validateNonEquiJoin(joinType, testSize, testSize - 1); + validateNonEquiJoin(joinType, testSize, testSize); + validateNonEquiJoin(joinType, testSize, testSize + 1); + + validateNonEquiJoin(joinType, testSize + 1, 0); + validateNonEquiJoin(joinType, testSize + 1, testSize - 1); + validateNonEquiJoin(joinType, testSize + 1, testSize); + validateNonEquiJoin(joinType, testSize + 1, testSize + 1); + + validateNonEquiJoin(joinType, 2 * testSize, 0); + validateNonEquiJoin(joinType, 0, 2 * testSize); + validateNonEquiJoin(joinType, 2 * testSize, 2 * testSize); + } + + /** */ + @Test + public void testInnerJoinWithPostFiltration() { + Object[][] persons = { + new Object[] {0, "Igor", 1}, + new Object[] {1, "Roman", 2}, + new Object[] {2, "Ivan", 5}, + new Object[] {3, "Alexey", 1} + }; + + Object[][] deps = { + new Object[] {1, "Core"}, + new Object[] {2, "SQL"}, + new Object[] {3, "QA"} + }; + + BiPredicate condition = (l, r) -> ((String)r[1]).length() > 3 && ((String)l[1]).length() > 4; + + Object[][] expected = {{3, "Alexey", 1, 1, "Core"}}; + + validate(INNER, Stream.of(persons)::iterator, Stream.of(deps)::iterator, expected, -1, condition); + } /** */ - private void validateEquiJoin(ExecutionContext ctx, JoinRelType joinType, int leftSize, int rightSize) { + @Test + public void testSemiJoinWithPostFiltration() { + Object[][] persons = { + new Object[] {0, "Igor", 1}, + new Object[] {1, "Roman", 2}, + new Object[] {2, "Ivan", 5}, + new Object[] {3, "Alexey", 1} + }; + + Object[][] deps = { + new Object[] {1, "Core"}, + new Object[] {2, "SQL"}, + new Object[] {3, "QA"} + }; + + BiPredicate condition = (l, r) -> ((String)r[1]).length() > 3 && ((String)l[1]).length() > 4; + + Object[][] expected = {{3, "Alexey", 1}}; + + validate(SEMI, Stream.of(persons)::iterator, Stream.of(deps)::iterator, expected, -1, condition); + } + + /** */ + private void validateEquiJoin(JoinRelType joinType, int leftSize, int rightSize) { if (log.isInfoEnabled()) { log.info("Testing eq. join with different buffer sizes. Join type: " + joinType + ", leftSz: " + leftSize + ", rightSz: " + rightSize); } - IgniteTypeFactory tf = ctx.getTypeFactory(); - { // Distinct inputs Object[] person = {1, "name", 2}; Object[] department = {1, "department"}; int resultSize = estimateResultSizeForDistinctInputs(joinType, leftSize, rightSize); - RelDataType leftType = TypeUtils.createRowType(tf, Integer.class, String.class, Integer.class); - RelDataType righType = TypeUtils.createRowType(tf, Integer.class, String.class); - validate( - ctx, joinType, () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), - leftType, () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), - righType, - resultSize + null, + resultSize, + null ); } -// { // Matching inputs -// Object[] person = {1, "name", 2}; -// Object[] department = {2, "department"}; -// -// int resultSize = estimateResultSizeForEqualInputs(joinType, leftSize, rightSize); -// -// validate( -// ctx, -// joinType, -// () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), -// () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), -// resultSize -// ); -// } + { // Matching inputs + Object[] person = {1, "name", 2}; + Object[] department = {2, "department"}; + + int resultSize = estimateResultSizeForEqualInputs(joinType, leftSize, rightSize); + + validate( + joinType, + () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), + () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), + null, + resultSize, + null + ); + } } -// /** */ -// protected void validateNonEquiJoin(ExecutionContext ctx, JoinRelType joinType, int leftSize, int rightSize) { -// Object[] person = {1, "name", 2}; -// Object[] department = {2, "department"}; -// -// int resultSize = estimateResultSizeForEqualInputs(joinType, leftSize, rightSize); -// -// validate( -// ctx, -// joinType, -// () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), -// () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), -// resultSize -// ); -// -// validate( -// ctx, -// joinType, -// () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), -// () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), -// 0 -// ); -// } + /** */ + protected void validateNonEquiJoin(JoinRelType joinType, int leftSize, int rightSize) { + Object[] person = {1, "name", 2}; + Object[] department = {2, "department"}; + + int resultSize = estimateResultSizeForEqualInputs(joinType, leftSize, rightSize); + + validate( + joinType, + () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), + () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), + null, + resultSize, + (l, r) -> true + ); + + validate( + joinType, + () -> IntStream.range(0, leftSize).mapToObj(i -> person).iterator(), + () -> IntStream.range(0, rightSize).mapToObj(i -> department).iterator(), + null, + 0, + (l, r) -> false + ); + } /** */ private static int estimateResultSizeForDistinctInputs(JoinRelType joinType, int leftSize, int rightSize) { @@ -235,18 +346,24 @@ private static int estimateResultSizeForEqualInputs( /** */ private void validate( - ExecutionContext ctx, JoinRelType joinType, Iterable leftSrc, - RelDataType leftType, Iterable rightSrc, - RelDataType rightType, - int resultSize + @Nullable Object[][] expected, + int resultSize, + @Nullable BiPredicate postCondition ) { + ExecutionContext ctx = executionContext(); + + IgniteTypeFactory tf = ctx.getTypeFactory(); + + RelDataType leftType = TypeUtils.createRowType(tf, Integer.class, String.class, Integer.class); + RelDataType rightType = TypeUtils.createRowType(tf, Integer.class, String.class); + ScanNode left = new ScanNode<>(ctx, leftType, leftSrc); ScanNode right = new ScanNode<>(ctx, rightType, rightSrc); - AbstractRightMaterializedJoinNode join = createJoinNode(ctx, joinType); + AbstractRightMaterializedJoinNode join = createJoinNode(ctx, joinType, leftType, rightType, postCondition); join.register(F.asList(left, right)); @@ -254,26 +371,50 @@ private void validate( node.register(join); - long cnt = StreamSupport.stream(Spliterators.spliteratorUnknownSize(node, Spliterator.ORDERED), false).count(); + ArrayList result = new ArrayList<>(); + + while (node.hasNext()) + result.add(node.next()); - assertEquals(resultSize, cnt); + if (resultSize >= 0) + assertEquals(resultSize, result.size()); + + if (expected != null) + checkResults(expected, result); } /** */ - private static HashJoinNode createJoinNode(ExecutionContext ctx, JoinRelType joinType) { + private static HashJoinNode createJoinNode( + ExecutionContext ctx, + JoinRelType joinType, + RelDataType leftType, + RelDataType rightType, + @Nullable BiPredicate postCondition + ) { IgniteTypeFactory tf = ctx.getTypeFactory(); - RelDataType leftType = TypeUtils.createRowType(tf, tf.createSqlType(SqlTypeName.INTEGER), - tf.createSqlType(SqlTypeName.VARCHAR)); - - RelDataType rightType = TypeUtils.createRowType(tf, tf.createSqlType(SqlTypeName.INTEGER), - tf.createSqlType(SqlTypeName.VARCHAR), tf.createSqlType(SqlTypeName.INTEGER)); - RelDataType outType = (joinType == ANTI || joinType == SEMI) ? leftType : TypeUtils.combinedRowType(tf, leftType, rightType); return HashJoinNode.create(ctx, outType, leftType, rightType, joinType, - JoinInfo.of(ImmutableIntList.of(2), ImmutableIntList.of(0))); + JoinInfo.of(ImmutableIntList.of(2), ImmutableIntList.of(0)), postCondition); + } + + /** */ + private static void checkResults(Object[][] expected, ArrayList actual) { + assertEquals(expected.length, actual.size()); + + actual.sort(Comparator.comparing(r -> (int)r[0])); + + int length = expected.length; + + for (int i = 0; i < length; ++i) { + Object[] exp = expected[i]; + Object[] act = actual.get(i); + + assertEquals(exp.length, act.length); + assertEquals(0, F.compareArrays(exp, act)); + } } } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java index 41c8ab0b24071..743ff954fb7e3 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java @@ -35,7 +35,13 @@ import org.junit.Test; import static org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING; +import static org.apache.calcite.sql.ddl.SqlDdlNodes.createSchema; +import static org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest.byClass; +import static org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest.createTable; +import static org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest.findFirstNode; +import static org.apache.ignite.internal.processors.query.calcite.planner.AbstractPlannerTest.findNodes; import static org.apache.ignite.testframework.GridTestUtils.assertThrows; +import static org.junit.Assert.assertNotNull; /** */ public class HashJoinPlannerTest extends AbstractPlannerTest { @@ -130,19 +136,34 @@ public void testHashJoinWinsOnSkewedLeftInput() throws Exception { /** */ @Test public void testHashJoinApplied() throws Exception { - for (List paramSet : testJoinIsAppliedParameters()) { - assert paramSet != null && paramSet.size() == 2; + List> testParams = F.asList( + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1", true, false), + F.asList("select t1.c1 from t1 %s join t1 t2 using(c1)", true, false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1", false, false), + F.asList("select t1.c1 from t1 %s join t1 t2 ON t1.id is not distinct from t2.c1", false, false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = ?", false, false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = OCTET_LENGTH('TEST')", false, false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = LOG10(t1.c1)", false, false), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1 and t1.ID > t2.ID", true, true), + F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1 and t2.c1 = 1", false, false) + ); - String sql = (String)paramSet.get(0); + for (List paramSet : testParams) { + assert paramSet != null && paramSet.size() == 3; + String sql = (String)paramSet.get(0); boolean canBePlanned = (Boolean)paramSet.get(1); + boolean onlyInnerOrSemi = (Boolean)paramSet.get(2); TestTable tbl = createTable("T1", IgniteDistributions.single(), "ID", Integer.class, "C1", Integer.class); IgniteSchema schema = createSchema(tbl); - for (String type : JOIN_TYPES) { - String sql0 = String.format(sql, type); + for (String joinType : JOIN_TYPES) { + if (onlyInnerOrSemi && !joinType.equals("INNER") && !joinType.equals("SEMI")) + continue; + + String sql0 = String.format(sql, joinType); if (canBePlanned) assertPlan(sql0, schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class)), DISABLED_RULES); @@ -154,21 +175,6 @@ public void testHashJoinApplied() throws Exception { } } - /** */ - private static List> testJoinIsAppliedParameters() { - return F.asList( - F.asList("select t1.c1 from t1 %s join t1 t2 using(c1)", true), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1", true), - F.asList("select t1.c1 from t1 %s join t1 t2 ON t1.id is not distinct from t2.c1", false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = OCTET_LENGTH('TEST')", false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = LOG10(t1.c1)", false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1 and t1.ID > t2.ID", false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1 and t2.c1 = 1", false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1", false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = ?", false) - ); - } - /** */ private static TestTable createSimpleTable() { return createSimpleTable("TEST_TBL", DEFAULT_TBL_SIZE); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java index 1627da8b9007f..d19e911c953f5 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java @@ -803,7 +803,7 @@ public void testJoinPushExpressionRule() throws Exception { /** */ @Test - public void testMergeJoinIsNotAppliedForNonEquiJoin() throws Exception { + public void testMergeJoinIsAppliedForNonEquiJoin() throws Exception { IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE); TestTable emp = new TestTable( @@ -847,7 +847,7 @@ public void testMergeJoinIsNotAppliedForNonEquiJoin() throws Exception { assertEquals("" + "IgniteSort(sort0=[$3], sort1=[$0], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first])\n" + " IgniteProject(DEPTNO=[$3], NAME=[$4], ID=[$0], NAME0=[$1])\n" + - " IgniteNestedLoopJoin(condition=[AND(=($3, $2), >=($1, $4))], joinType=[inner])\n" + + " IgniteHashJoin(condition=[AND(=($3, $2), >=($1, $4))], joinType=[inner])\n" + " IgniteTableScan(table=[[PUBLIC, EMP]])\n" + " IgniteTableScan(table=[[PUBLIC, DEPT]])\n", RelOptUtil.toString(phys)); From 3337e989a9055c987e8dad642b3eed8c50168f13 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 16 Jun 2025 15:10:33 +0300 Subject: [PATCH 07/28] +minor ai3 ticket --- .../processors/query/calcite/exec/rel/MergeJoinNode.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java index 94648e323ef5d..4c97c7a7e1e13 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java @@ -159,7 +159,8 @@ private void pushLeft(Row row) throws Exception { leftInBuf.add(row); - join(); + if (waitingLeft == 0 && waitingRight <= 0) + join(); } /** */ @@ -173,7 +174,8 @@ private void pushRight(Row row) throws Exception { rightInBuf.add(row); - join(); + if (waitingRight == 0 && waitingLeft <= 0) + join(); } /** */ From 48016a998999b2ec6f2bc34264f78920ed5a8c0e Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 16 Jun 2025 15:11:04 +0300 Subject: [PATCH 08/28] Revert "+minor ai3 ticket" This reverts commit 3337e989a9055c987e8dad642b3eed8c50168f13. --- .../processors/query/calcite/exec/rel/MergeJoinNode.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java index 4c97c7a7e1e13..94648e323ef5d 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/MergeJoinNode.java @@ -159,8 +159,7 @@ private void pushLeft(Row row) throws Exception { leftInBuf.add(row); - if (waitingLeft == 0 && waitingRight <= 0) - join(); + join(); } /** */ @@ -174,8 +173,7 @@ private void pushRight(Row row) throws Exception { rightInBuf.add(row); - if (waitingRight == 0 && waitingLeft <= 0) - join(); + join(); } /** */ From 2acd9b3cdd57a21bb4b2b34c5f54c36319d4a011 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 16 Jun 2025 17:04:14 +0300 Subject: [PATCH 09/28] + lost HashJoin exec test --- .../java/org/apache/ignite/testsuites/ExecutionTestSuite.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java index a9392bf7abb6f..f5d19bfe4d3ef 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateSingleGroupExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashIndexSpoolExecutionTest; +import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashJoinExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.LimitExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.MergeJoinExecutionTest; @@ -45,6 +46,7 @@ ContinuousExecutionTest.class, MergeJoinExecutionTest.class, NestedLoopJoinExecutionTest.class, + HashJoinExecutionTest.class, TableSpoolExecutionTest.class, SortedIndexSpoolExecutionTest.class, HashIndexSpoolExecutionTest.class, From cc4526c93f61f811121f21b1464355ae5ce3a33b Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Jul 2025 13:50:32 +0300 Subject: [PATCH 10/28] merged master --- .../processors/query/calcite/exec/rel/ExecutionTest.java | 1 + .../query/calcite/planner/JoinCommutePlannerTest.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java index 2129043575cc1..9f63cb8211df8 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java @@ -26,6 +26,7 @@ import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java index 6cbcf78c791a1..8e704c047c535 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinCommutePlannerTest.java @@ -341,7 +341,7 @@ public void testInnerCommute() throws Exception { // no commute assertEquals(JoinRelType.INNER, join.getJoinType()); - ctx = plannerCtx(sql, publicSchema, planLsnr "MergeJoinConverter", "CorrelatedNestedLoopJoin", "HashJoinConverter", + ctx = plannerCtx(sql, publicSchema, planLsnr, "MergeJoinConverter", "CorrelatedNestedLoopJoin", "HashJoinConverter", "JoinCommuteRule"); pl = ctx.cluster().getPlanner(); From 75eb1e3ad697f09b73a92926e0f51c67228bf5ca Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 16 Jul 2025 17:39:26 +0300 Subject: [PATCH 11/28] review fixes --- .../query/calcite/exec/rel/HashJoinNode.java | 38 ++--------- .../query/calcite/prepare/PlannerHelper.java | 2 + ...iteBasicSecondaryIndexIntegrationTest.java | 66 +++++++++---------- .../calcite/planner/HashJoinPlannerTest.java | 19 ++---- .../planner/JoinColocationPlannerTest.java | 15 ++--- 5 files changed, 49 insertions(+), 91 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index ed569bec41098..82c80410030ee 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -32,6 +32,8 @@ import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; /** Hash join implementor. */ @@ -52,7 +54,7 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod protected final RowHandler outRowHnd; /** */ - protected final Map> hashStore = new HashMap<>(INITIAL_CAPACITY); + protected final Map> hashStore = U.newHashMap(INITIAL_CAPACITY); /** */ protected Iterator rightIt = Collections.emptyIterator(); @@ -156,39 +158,7 @@ protected Collection lookup(Row row) { /** */ private static Iterator untouched(Map> entries) { - return new Iterator() { - private final Iterator> it = entries.values().iterator(); - - private Iterator innerIt = Collections.emptyIterator(); - - @Override public boolean hasNext() { - if (innerIt.hasNext()) - return true; - - advance(); - - return innerIt.hasNext(); - } - - @Override public RowT next() { - if (!hasNext()) - throw new NoSuchElementException(); - - return innerIt.next(); - } - - private void advance() { - while (it.hasNext()) { - TouchedCollection coll = it.next(); - - if (!coll.touched && !coll.items().isEmpty()) { - innerIt = coll.items().iterator(); - - break; - } - } - } - }; + return F.flat(F.iterator(entries.values(), TouchedCollection::items, true, v -> !v.touched)); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java index 12b9deed6e982..bf141af36e61c 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java @@ -164,6 +164,8 @@ public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteL if (sqlNode.isA(ImmutableSet.of(SqlKind.INSERT, SqlKind.UPDATE, SqlKind.MERGE))) igniteRel = new FixDependentModifyNodeShuttle().visit(igniteRel); + System.err.println("TEST | plan:\n" + RelOptUtil.toString(igniteRel)); + return igniteRel; } catch (Throwable ex) { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java index 4ad18b760cd8c..7016a65f3d1df 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java @@ -18,7 +18,6 @@ import java.sql.Date; import java.util.LinkedHashMap; -import java.util.List; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; @@ -279,40 +278,37 @@ public void testIndexLoopJoin() { /** */ @Test - public void testMergeAndHashJoin() { - for (List params : F.asList(F.asList(HintDefinition.MERGE_JOIN.name(), "IgniteMergeJoin"), - F.asList(HintDefinition.HASH_JOIN.name(), "IgniteHashJoin"))) { - assertQuery("" + - "SELECT /*+ " + params.get(0) + " */ d1.name, d2.name FROM Developer d1, Developer d2 " + - "WHERE d1.depId = d2.depId") - .matches(containsSubPlan(params.get(1))) - .returns("Bach", "Bach") - .returns("Beethoven", "Beethoven") - .returns("Beethoven", "Strauss") - .returns("Mozart", "Mozart") - .returns("Strauss", "Strauss") - .returns("Strauss", "Beethoven") - .returns("Vagner", "Vagner") - .returns("Chaikovsky", "Chaikovsky") - .returns("Verdy", "Verdy") - .returns("Stravinsky", "Stravinsky") - .returns("Rahmaninov", "Rahmaninov") - .returns("Shubert", "Shubert") - .returns("Glinka", "Glinka") - .returns("Arnalds", "Arnalds") - .returns("Glass", "Glass") - .returns("O'Halloran", "O'Halloran") - .returns("Prokofiev", "Prokofiev") - .returns("Yiruma", "Yiruma") - .returns("Cacciapaglia", "Cacciapaglia") - .returns("Einaudi", "Einaudi") - .returns("Hasaishi", "Hasaishi") - .returns("Marradi", "Marradi") - .returns("Musorgskii", "Musorgskii") - .returns("Rihter", "Rihter") - .returns("Zimmer", "Zimmer") - .check(); - } + public void testMergeJoin() { + assertQuery("" + + "SELECT /*+ " + HintDefinition.MERGE_JOIN + " */ d1.name, d2.name FROM Developer d1, Developer d2 " + + "WHERE d1.depId = d2.depId") + .matches(containsSubPlan("IgniteMergeJoin")) + .returns("Bach", "Bach") + .returns("Beethoven", "Beethoven") + .returns("Beethoven", "Strauss") + .returns("Mozart", "Mozart") + .returns("Strauss", "Strauss") + .returns("Strauss", "Beethoven") + .returns("Vagner", "Vagner") + .returns("Chaikovsky", "Chaikovsky") + .returns("Verdy", "Verdy") + .returns("Stravinsky", "Stravinsky") + .returns("Rahmaninov", "Rahmaninov") + .returns("Shubert", "Shubert") + .returns("Glinka", "Glinka") + .returns("Arnalds", "Arnalds") + .returns("Glass", "Glass") + .returns("O'Halloran", "O'Halloran") + .returns("Prokofiev", "Prokofiev") + .returns("Yiruma", "Yiruma") + .returns("Cacciapaglia", "Cacciapaglia") + .returns("Einaudi", "Einaudi") + .returns("Hasaishi", "Hasaishi") + .returns("Marradi", "Marradi") + .returns("Musorgskii", "Musorgskii") + .returns("Rihter", "Rihter") + .returns("Zimmer", "Zimmer") + .check(); } // ===== No filter ===== diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java index 743ff954fb7e3..8198d463d34e3 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java @@ -61,15 +61,12 @@ public void testHashJoinKeepsLeftCollation() throws Exception { IgniteSchema schema = createSchema(tbl1, tbl2); String sql = "select t1.ID, t2.ID1 " - + "from TEST_TBL_CMPLX t2 " - + "join TEST_TBL t1 on t1.id = t2.id1 " - + "order by t2.ID1 NULLS LAST, t2.ID2 NULLS LAST"; - - RelNode plan = physicalPlan(sql, schema, DISABLED_RULES); + + "from TEST_TBL_CMPLX t2 " + + "join TEST_TBL t1 on t1.id = t2.id1 " + + "order by t2.ID1 NULLS LAST, t2.ID2 NULLS LAST"; - assertEquals(0, findNodes(plan, byClass(IgniteSort.class)).size()); - assertEquals(1, findNodes(plan, byClass(IgniteHashJoin.class)).size()); - assertNotNull(findFirstNode(plan, byClass(IgniteHashJoin.class))); + assertPlan(sql, schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class)) + .and(nodeOrAnyChild(isInstanceOf(IgniteSort.class)).negate()), DISABLED_RULES); } /** */ @@ -85,10 +82,8 @@ public void testHashJoinErasesRightCollation() throws Exception { + "join TEST_TBL_CMPLX t2 on t1.id = t2.id1 " + "order by t2.ID1 NULLS LAST, t2.ID2 NULLS LAST"; - IgniteRel plan = physicalPlan(sql, schema, DISABLED_RULES); - - assertNotNull(findFirstNode(plan, byClass(IgniteHashJoin.class))); - assertNotNull(sortOnTopOfJoin(plan)); + assertPlan(sql, schema, nodeOrAnyChild(isInstanceOf(IgniteSort.class) + .and(input(isInstanceOf(IgniteHashJoin.class)))), DISABLED_RULES); } /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java index 0306c73b812b2..5c72d742ffb47 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/JoinColocationPlannerTest.java @@ -70,16 +70,11 @@ public void joinSameTableSimpleAff() throws Exception { "join TEST_TBL t2 on t1.id = t2.id"; for (String disabledRule : DISABLED_RULES) { - RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", disabledRule); - - AbstractIgniteJoin join = findFirstNode(phys, byClass(AbstractIgniteJoin.class)); - - String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys); - - assertThat(invalidPlanMsg, join, notNullValue()); - assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true)); - assertThat(invalidPlanMsg, join.getLeft(), instanceOf(IgniteIndexScan.class)); - assertThat(invalidPlanMsg, join.getRight(), instanceOf(IgniteIndexScan.class)); + assertPlan(sql, schema, nodeOrAnyChild(isInstanceOf(AbstractIgniteJoin.class) + .and(join -> join.distribution().function().affinity()) + .and(input(0, isInstanceOf(IgniteIndexScan.class))) + .and(input(1, isInstanceOf(IgniteIndexScan.class))) + ), "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin", disabledRule); } } From 354937b14eb30db3625a5686771dc4fd5f29fa20 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 17 Jul 2025 15:04:29 +0300 Subject: [PATCH 12/28] raw --- .../query/calcite/exec/RuntimeHashIndex.java | 41 ++++++++++----- .../query/calcite/exec/rel/HashJoinNode.java | 50 +++++++++++++------ .../calcite/exec/rel/IndexSpoolNode.java | 2 +- 3 files changed, 65 insertions(+), 28 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java index 82973c64a3707..6219b57cfa276 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.query.calcite.exec; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -27,6 +28,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * Runtime hash index based on on-heap hash map. @@ -45,11 +47,14 @@ public class RuntimeHashIndex implements RuntimeIndex { private final ImmutableBitSet keys; /** Rows. */ - private final HashMap> rows; + private final HashMap> rows; /** Allow NULL values. */ private final boolean allowNulls; + /** */ + private final Supplier> groupCollectionFactory; + /** * */ @@ -69,7 +74,7 @@ public RuntimeHashIndex( /** {@inheritDoc} */ @Override public void push(Row r) { - GroupKey key = key(r); + GroupKey key = key(r, keys); if (key == NULL_KEY) return; @@ -85,12 +90,12 @@ public RuntimeHashIndex( } /** */ - public Iterable scan(Supplier searchRow) { - return new IndexScan(searchRow); + public IndexScan scan(Supplier searchRow, @NotNull int[] keysToUse) { + return new IndexScan(searchRow, keysToUse); } /** */ - private GroupKey key(Row r) { + private GroupKey key(Row r, ImmutableBitSet keys) { GroupKey.Builder b = GroupKey.builder(keys.cardinality()); for (Integer field : keys) { @@ -108,27 +113,37 @@ private GroupKey key(Row r) { /** * */ - private class IndexScan implements Iterable { + public class IndexScan implements Iterable { /** Search row. */ private final Supplier searchRow; + /** */ + private final ImmutableBitSet keysToUse; + /** * @param searchRow Search row. + * @param remappedKeys Actual keys to use. If {@code null}, default {@code keys} are used.. */ - IndexScan(Supplier searchRow) { + private IndexScan(Supplier searchRow, @Nullable int[] remappedKeys) { this.searchRow = searchRow; + this.keysToUse = remappedKeys == null ? keys : ImmutableBitSet.of(remappedKeys); } - /** {@inheritDoc} */ - @NotNull @Override public Iterator iterator() { - GroupKey key = key(searchRow.get()); + /** */ + public @Nullable Collection get() { + GroupKey key = key(searchRow.get(), keysToUse); if (key == NULL_KEY) - return Collections.emptyIterator(); + return null; - List eqRows = rows.get(key); + return rows.get(key); + } + + /** {@inheritDoc} */ + @Override public Iterator iterator() { + Collection res = get(); - return eqRows == null ? Collections.emptyIterator() : eqRows.iterator(); + return res == null ? Collections.emptyIterator() : res.iterator(); } } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 82c80410030ee..7e77646f51cb2 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -20,17 +20,17 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Objects; import java.util.function.BiPredicate; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; +import org.apache.ignite.internal.processors.query.calcite.exec.RuntimeHashIndex; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -56,6 +56,9 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod /** */ protected final Map> hashStore = U.newHashMap(INITIAL_CAPACITY); + /** */ + protected final RuntimeHashIndex runtimeHashIdx; + /** */ protected Iterator rightIt = Collections.emptyIterator(); @@ -88,6 +91,8 @@ protected HashJoinNode( this.outRowHnd = outRowHnd; this.nonEqCond = nonEqCond; + + runtimeHashIdx = new RuntimeHashIndex<>(ctx, ImmutableBitSet.of(info.rightKeys), keepRowsWithNull()); } /** {@inheritDoc} */ @@ -97,6 +102,7 @@ protected HashJoinNode( rightIt = Collections.emptyIterator(); hashStore.clear(); + runtimeHashIdx.close(); } /** Creates certain join node. */ @@ -139,21 +145,28 @@ public static HashJoinNode create( /** */ protected Collection lookup(Row row) { - Key row0 = extractKey(row, leftKeys); + Collection res = runtimeHashIdx.scan(() -> row, leftKeys).get(); - // Key with null field can't be compared with other keys. - if (row0 == NULL_KEY) + if (res == null) return Collections.emptyList(); - TouchedCollection found = hashStore.get(row0); - - if (found != null) { - found.touched = true; - - return found.items(); - } - - return Collections.emptyList(); + return res; + +// Key row0 = extractKey(row, leftKeys); +// +// // Key with null field can't be compared with other keys. +// if (row0 == NULL_KEY) +// return Collections.emptyList(); +// +// TouchedCollection found = hashStore.get(row0); +// +// if (found != null) { +// found.touched = true; +// +// return found.items(); +// } +// +// return Collections.emptyList(); } /** */ @@ -179,6 +192,8 @@ private static Iterator untouched(Map> TouchedCollection raw = hashStore.computeIfAbsent(key, k -> new TouchedCollection<>()); raw.add(row); + + runtimeHashIdx.push(row); } if (waitingRight == 0) @@ -210,6 +225,7 @@ protected void requestMoreOrEnd() throws Exception { requested = 0; hashStore.clear(); + runtimeHashIdx.close(); downstream().end(); } @@ -803,6 +819,12 @@ private RowWrapper(RowT row, RowHandler hnd, int[] items) { } } + /** */ + private static final class TouchedList extends ArrayList { + /** */ + private boolean touched; + } + /** */ private static final class TouchedCollection { /** */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java index 998792f0f36c1..2b5ac7fc5e7d9 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java @@ -196,7 +196,7 @@ public static IndexSpoolNode createHashSpool( ScanNode scan = new ScanNode<>( ctx, rowType, - idx.scan(searchRow), + idx.scan(searchRow, null), filter, null ); From d7fe5277df9ddc5f70a9c2eb001abc5a663e1738 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sun, 27 Jul 2025 18:07:27 +0300 Subject: [PATCH 13/28] rawWithOldStore --- .../query/calcite/exec/RuntimeHashIndex.java | 97 ++++++++--- .../query/calcite/exec/rel/HashJoinNode.java | 157 +++++++++--------- .../calcite/exec/rel/IndexSpoolNode.java | 2 +- .../calcite/exec/rel/NestedLoopJoinNode.java | 10 +- .../exec/rel/HashJoinExecutionTest.java | 4 +- .../exec/rel/JoinBuffersExecutionTest.java | 37 +++-- 6 files changed, 189 insertions(+), 118 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java index 0e670e1da5f31..ff705549c037c 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java @@ -17,15 +17,13 @@ package org.apache.ignite.internal.processors.query.calcite.exec; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; -import java.util.List; import java.util.function.Supplier; import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey; -import org.apache.ignite.internal.util.typedef.F; -import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; /** @@ -39,26 +37,51 @@ public class RuntimeHashIndex implements RuntimeIndex { private final RowHandler keysRowHnd; /** Rows. */ - private final HashMap, List> rows; + private final HashMap, Collection> rows; + + /** */ + private final Supplier> collectionFactory; /** Allow NULL values. */ private final boolean allowNulls; - /** - * - */ + /** Creates hash index with the default collection supplier. */ + public RuntimeHashIndex(ExecutionContext ectx, ImmutableBitSet keys, boolean allowNulls) { + this(ectx, keys, allowNulls, -1, null); + } + + /** */ public RuntimeHashIndex( ExecutionContext ectx, ImmutableBitSet keys, - boolean allowNulls + boolean allowNulls, + int initCapacity, + @Nullable Supplier> collectionFactory + ) { + this( + ectx, + allowNulls, + new MappingRowHandler<>(ectx.rowHandler(), keys), + initCapacity >= 0 ? new HashMap<>(initCapacity) : new HashMap<>(), + collectionFactory + ); + } + + /** Fields setting constructor. */ + private RuntimeHashIndex( + ExecutionContext ectx, + boolean allowNulls, + RowHandler keysRowHnd, + HashMap, Collection> rows, + @Nullable Supplier> collectionFactory ) { this.ectx = ectx; this.allowNulls = allowNulls; - assert !F.isEmpty(keys); + this.keysRowHnd = keysRowHnd; + this.rows = rows; - keysRowHnd = new MappingRowHandler<>(ectx.rowHandler(), keys); - rows = new HashMap<>(); + this.collectionFactory = collectionFactory == null ? ArrayList::new : collectionFactory; } /** {@inheritDoc} */ @@ -68,7 +91,7 @@ public RuntimeHashIndex( if (key == null) return; - List eqRows = rows.computeIfAbsent(key, k -> new ArrayList<>()); + Collection eqRows = rows.computeIfAbsent(key, k -> collectionFactory.get()); eqRows.add(r); } @@ -79,7 +102,12 @@ public RuntimeHashIndex( } /** */ - public Iterable scan(Supplier searchRow) { + public Collection> rowSets() { + return Collections.unmodifiableCollection(rows.values()); + } + + /** */ + public IndexScan scan(Supplier searchRow) { return new IndexScan(searchRow); } @@ -99,30 +127,59 @@ public Iterable scan(Supplier searchRow) { return new GroupKey<>(r, keysRowHnd); } + /** */ + public RuntimeHashIndex remappedSearcher(ImmutableBitSet remappedKeys) { + return new RemappedSearcher<>(this, remappedKeys); + } + + /** */ + private static class RemappedSearcher extends RuntimeHashIndex { + /** */ + private final RuntimeHashIndex origin; + + /** */ + private RemappedSearcher(RuntimeHashIndex origin, ImmutableBitSet remappedKeys){ + super(origin.ectx, origin.allowNulls, new MappingRowHandler<>(origin.ectx.rowHandler(), remappedKeys), + origin.rows, origin.collectionFactory); + + this.origin = origin; + } + + /** {@inheritDoc} */ + @Override public void push(Row r) { + origin.push(r); + } + } + /** * */ - private class IndexScan implements Iterable { + public class IndexScan implements Iterable { /** Search row. */ private final Supplier searchRow; /** * @param searchRow Search row. */ - IndexScan(Supplier searchRow) { + private IndexScan(Supplier searchRow) { this.searchRow = searchRow; } - /** {@inheritDoc} */ - @NotNull @Override public Iterator iterator() { + /** */ + public @Nullable Collection get() { GroupKey key = key(searchRow.get()); if (key == null) - return Collections.emptyIterator(); + return Collections.emptyList(); - List eqRows = rows.get(key); + return rows.get(key); + } + + /** {@inheritDoc} */ + @Override public Iterator iterator() { + Collection collection = get(); - return eqRows == null ? Collections.emptyIterator() : eqRows.iterator(); + return collection == null ? Collections.emptyIterator() : collection.iterator(); } } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 7e77646f51cb2..0142d9d0bcc57 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -55,9 +55,12 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod /** */ protected final Map> hashStore = U.newHashMap(INITIAL_CAPACITY); - - /** */ - protected final RuntimeHashIndex runtimeHashIdx; + protected final boolean keepRowsWithNull; +// /** */ +// protected final RuntimeHashIndex hashIdx; +// +// /** */ +// protected final RuntimeHashIndex remappedHashIdx; /** */ protected Iterator rightIt = Collections.emptyIterator(); @@ -72,6 +75,9 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod * @param rowType Row type. * @param info Join info. * @param outRowHnd Output row handler. + * @param keepRowsWithNull {@code True} if we need to store the row from right shoulder even if it contains NULL in + * any of join key position. This is required for joins which emit unmatched part + * of the right shoulder, such as RIGHT JOIN and FULL OUTER JOIN. * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ protected HashJoinNode( @@ -79,6 +85,7 @@ protected HashJoinNode( RelDataType rowType, JoinInfo info, RowHandler outRowHnd, + boolean keepRowsWithNull, @Nullable BiPredicate nonEqCond ) { super(ctx, rowType); @@ -92,7 +99,11 @@ protected HashJoinNode( this.nonEqCond = nonEqCond; - runtimeHashIdx = new RuntimeHashIndex<>(ctx, ImmutableBitSet.of(info.rightKeys), keepRowsWithNull()); + this.keepRowsWithNull = keepRowsWithNull; + +// hashIdx = new RuntimeHashIndex<>(ctx, ImmutableBitSet.of(info.rightKeys), keepRowsWithNull, INITIAL_CAPACITY, +// TouchedArrayList::new); +// remappedHashIdx = hashIdx.remappedSearcher(ImmutableBitSet.of(info.leftKeys)); } /** {@inheritDoc} */ @@ -102,7 +113,7 @@ protected HashJoinNode( rightIt = Collections.emptyIterator(); hashStore.clear(); - runtimeHashIdx.close(); +// hashIdx.close(); } /** Creates certain join node. */ @@ -145,34 +156,43 @@ public static HashJoinNode create( /** */ protected Collection lookup(Row row) { - Collection res = runtimeHashIdx.scan(() -> row, leftKeys).get(); - - if (res == null) - return Collections.emptyList(); - - return res; - -// Key row0 = extractKey(row, leftKeys); +// Collection res = remappedHashIdx.scan(() -> row).get(); // -// // Key with null field can't be compared with other keys. -// if (row0 == NULL_KEY) +// if (res == null) // return Collections.emptyList(); // -// TouchedCollection found = hashStore.get(row0); -// -// if (found != null) { -// found.touched = true; +// assert res instanceof TouchedArrayList; // -// return found.items(); -// } +// ((TouchedArrayList)res).touched = true; // -// return Collections.emptyList(); +// return res == null ? Collections.emptyList() : res; +// TODO: remove + Key row0 = extractKey(row, leftKeys); + + // Key with null field can't be compared with other keys. + if (row0 == NULL_KEY) + return Collections.emptyList(); + + TouchedCollection found = hashStore.get(row0); + + if (found != null) { + found.touched = true; + + return found.items(); + } + + return Collections.emptyList(); } /** */ - private static Iterator untouched(Map> entries) { - return F.flat(F.iterator(entries.values(), TouchedCollection::items, true, v -> !v.touched)); + protected Iterator untouched() { + return F.flat(F.iterator(hashStore.values(), TouchedCollection::items, true, v -> !v.touched)); } +// +// /** */ +// protected Iterator untouched() { +// return F.flat(F.iterator(hashIdx.rowSets(), c0 -> c0, true, c1 -> !((TouchedArrayList)c1).touched)); +// } /** {@inheritDoc} */ @Override protected void pushRight(Row row) throws Exception { @@ -185,15 +205,15 @@ private static Iterator untouched(Map> waitingRight--; + //hashIdx.push(row); + Key key = extractKey(row, rightKeys); - // No storing in #hashStore, if the row contains NULL. And we won't emit right part alone like in FULL OUTER and RIGHT joins. - if (keepRowsWithNull() || key != NULL_KEY) { + // No storing in #hashIndex, if the row contains NULL. And we won't emit right part alone like in FULL OUTER and RIGHT joins. + if (keepRowsWithNull || key != NULL_KEY) { TouchedCollection raw = hashStore.computeIfAbsent(key, k -> new TouchedCollection<>()); raw.add(row); - - runtimeHashIdx.push(row); } if (waitingRight == 0) @@ -225,22 +245,12 @@ protected void requestMoreOrEnd() throws Exception { requested = 0; hashStore.clear(); - runtimeHashIdx.close(); +// hashIdx.close(); downstream().end(); } } - /** - * Returns {@code true} if we need to store the row from right shoulder even if it contains NULL in any of join key position. - * This is required for joins which emit unmatched part of the right shoulder, such as RIGHT JOIN and FULL OUTER JOIN. - * - * @return {@code true} when row must be stored in {@link #hashStore} unconditionally. - */ - protected boolean keepRowsWithNull() { - return false; - } - /** */ private static final class InnerHashJoin extends HashJoinNode { /** @@ -258,7 +268,7 @@ private InnerHashJoin(ExecutionContext ctx, RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, nonEqCond); + super(ctx, rowType, info, outRowHnd, false, nonEqCond); } /** {@inheritDoc} */ @@ -328,7 +338,7 @@ private LeftHashJoin( RowHandler.RowFactory rightRowFactory, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, nonEqCond); + super(ctx, rowType, info, outRowHnd, false, nonEqCond); assert nonEqCond == null : "Non equi condition is not supported in LEFT join"; @@ -412,7 +422,7 @@ private RightHashJoin( RowHandler.RowFactory leftRowFactory, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, nonEqCond); + super(ctx, rowType, info, outRowHnd, true, nonEqCond); assert nonEqCond == null : "Non equi condition is not supported in RIGHT join"; @@ -467,7 +477,7 @@ private RightHashJoin( // Prevent scanning store more than once. drainMaterialization = true; - rightIt = untouched(hashStore); + rightIt = untouched(); } RowT emptyLeft = leftRowFactory.create(); @@ -495,11 +505,6 @@ private RightHashJoin( requestMoreOrEnd(); } - /** {@inheritDoc} */ - @Override protected boolean keepRowsWithNull() { - return true; - } - /** {@inheritDoc} */ @Override protected void rewindInternal() { drainMaterialization = false; @@ -539,7 +544,7 @@ private FullOuterHashJoin( RowHandler.RowFactory rightRowFactory, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, nonEqCond); + super(ctx, rowType, info, outRowHnd, true, nonEqCond); assert nonEqCond == null : "Non equi condition is not supported in FULL OUTER join"; @@ -562,33 +567,31 @@ private FullOuterHashJoin( Collection rightRows = lookup(left); - // Emit unmatched left row. if (rightRows.isEmpty()) { - requested--; - - downstream().push(outRowHnd.concat(left, rightRowFactory.create())); + // Emit empty right row for unmatched left row. + rightIt = Collections.singletonList(rightRowFactory.create()).iterator(); } - - rightIt = rightRows.iterator(); + else + rightIt = rightRows.iterator(); } - // Emits matched rows. - while (rightIt.hasNext()) { - checkState(); + if (rightIt.hasNext()) { + // Emits matched rows. + while (requested > 0 && rightIt.hasNext()) { + checkState(); - RowT right = rightIt.next(); + RowT right = rightIt.next(); - --requested; + --requested; - downstream().push(outRowHnd.concat(left, right)); + downstream().push(outRowHnd.concat(left, right)); + } - if (requested == 0) - break; + if (!rightIt.hasNext()) + left = null; } - - if (!rightIt.hasNext()) + else left = null; - } } finally { @@ -605,22 +608,21 @@ private FullOuterHashJoin( // Prevent scanning store more than once. drainMaterialization = true; - rightIt = untouched(hashStore); + rightIt = untouched(); } RowT emptyLeft = leftRowFactory.create(); - while (rightIt.hasNext()) { + while (requested > 0 && rightIt.hasNext()) { checkState(); - RowT row = outRowHnd.concat(emptyLeft, rightIt.next()); + RowT right = rightIt.next(); + + RowT row = outRowHnd.concat(emptyLeft, right); --requested; downstream().push(row); - - if (requested == 0) - break; } } finally { @@ -631,11 +633,6 @@ private FullOuterHashJoin( requestMoreOrEnd(); } - /** {@inheritDoc} */ - @Override protected boolean keepRowsWithNull() { - return true; - } - /** {@inheritDoc} */ @Override protected void rewindInternal() { drainMaterialization = false; @@ -662,7 +659,7 @@ private SemiHashJoin( RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, nonEqCond); + super(ctx, rowType, info, outRowHnd, false, nonEqCond); } /** {@inheritDoc} */ @@ -728,7 +725,7 @@ private AntiHashJoin( RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, nonEqCond); + super(ctx, rowType, info, outRowHnd, false, nonEqCond); } /** {@inheritDoc} */ @@ -820,7 +817,7 @@ private RowWrapper(RowT row, RowHandler hnd, int[] items) { } /** */ - private static final class TouchedList extends ArrayList { + private static final class TouchedArrayList extends ArrayList { /** */ private boolean touched; } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java index 2b5ac7fc5e7d9..998792f0f36c1 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java @@ -196,7 +196,7 @@ public static IndexSpoolNode createHashSpool( ScanNode scan = new ScanNode<>( ctx, rowType, - idx.scan(searchRow, null), + idx.scan(searchRow), filter, null ); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index 6271de8957786..cd49013d7fde6 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -28,7 +28,7 @@ /** */ public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJoinNode { - /** */ + /** TODO : use also in hash join */ private static final int HALF_BUF_SIZE = IN_BUFFER_SIZE >> 1; /** */ @@ -152,7 +152,7 @@ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate + JoinFactory joinFactory = (ctx, outType, leftType, rightType, joinType) -> MergeJoinNode.create(ctx, outType, leftType, rightType, joinType, Comparator.comparingInt(r -> (Integer)r[0]), true); Consumer> bufChecker = (node) -> { @@ -63,19 +64,35 @@ public void testMergeJoinBuffers() throws Exception { doTestJoinBuffer(joinFactory, bufChecker); } - /** Tests NL with input bigger that the buffer size. */ + /** */ @Test public void testNLJoinBuffers() throws Exception { - JoinFactory joinFactory = (ctx, outType, leftType, rightType, joinType, cond) -> + JoinFactory joinFactory = (ctx, outType, leftType, rightType, joinType) -> NestedLoopJoinNode.create(ctx, outType, leftType, rightType, joinType, (r1, r2) -> r1[0].equals(r2[0])); Consumer> bufChecker = (node) -> - assertTrue(((NestedLoopJoinNode)node).leftInBuf.size() <= IN_BUFFER_SIZE); + assertTrue(((AbstractRightMaterializedJoinNode)node).leftInBuf.size() <= IN_BUFFER_SIZE); + + doTestJoinBuffer(joinFactory, bufChecker); + } + + /** */ + @Test + public void testHashJoinBuffers() throws Exception { + JoinInfo joinInfo = JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(0)); + + JoinFactory joinFactory = (ctx, outType, leftType, rightType, joinType) -> + HashJoinNode.create(ctx, outType, leftType, rightType, joinType, joinInfo, null); + + Consumer> bufChecker = (node) -> + assertTrue(((AbstractRightMaterializedJoinNode)node).leftInBuf.size() <= IN_BUFFER_SIZE); doTestJoinBuffer(joinFactory, bufChecker); } /** + * Tests a join with input bigger that the buffer size. + * * @param joinFactory Creates certain join node. * @param joinBufChecker Finally check node after successfull run. */ @@ -83,7 +100,7 @@ private void doTestJoinBuffer( JoinFactory joinFactory, Consumer> joinBufChecker ) throws Exception { - for (JoinRelType joinType : F.asList(LEFT, INNER, RIGHT, FULL, SEMI, ANTI)) { + for (JoinRelType joinType : F.asList(FULL)) { if (log.isInfoEnabled()) log.info("Testing join of type '" + joinType + "'..."); @@ -106,8 +123,7 @@ private void doTestJoinBuffer( RelDataType outType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, int.class); - AbstractNode join = joinFactory.create(ctx, outType, leftType, rightType, joinType, - (r1, r2) -> r1[0].equals(r2[0])); + AbstractNode join = joinFactory.create(ctx, outType, leftType, rightType, joinType); join.register(F.asList(leftNode, rightNode)); @@ -231,8 +247,7 @@ AbstractNode create( RelDataType outType, RelDataType leftType, RelDataType rightType, - JoinRelType joinType, - BiPredicate cond + JoinRelType joinType ); } } From d493a914537d87c313b01d7524e0302165dcde96 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sun, 27 Jul 2025 18:29:59 +0300 Subject: [PATCH 14/28] raw with new storage. Some tests. --- .../calcite/exec/LogicalRelImplementor.java | 16 +- .../query/calcite/exec/MappingRowHandler.java | 17 +- .../query/calcite/exec/RuntimeHashIndex.java | 46 ++- .../query/calcite/exec/rel/HashJoinNode.java | 329 +++++---------- .../query/calcite/prepare/PlannerHelper.java | 2 - .../query/calcite/rel/IgniteJoinInfo.java | 97 +++++ .../calcite/rule/HashJoinConverterRule.java | 20 +- .../calcite/rule/MergeJoinConverterRule.java | 8 +- .../query/calcite/exec/rel/ExecutionTest.java | 128 +----- .../exec/rel/HashJoinExecutionTest.java | 4 +- .../exec/rel/JoinBuffersExecutionTest.java | 40 +- .../integration/JoinIntegrationTest.java | 382 +++++++++++++++++- .../calcite/planner/HashJoinPlannerTest.java | 39 +- .../query/calcite/planner/PlannerTest.java | 0 14 files changed, 706 insertions(+), 422 deletions(-) create mode 100644 modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java delete mode 100644 modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java index 64e64eaea8011..5a092e68ae7b3 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java @@ -82,6 +82,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexBound; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexCount; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteLimit; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMergeJoin; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin; @@ -120,8 +121,6 @@ import org.apache.ignite.internal.util.typedef.F; import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED; -import static org.apache.calcite.sql.SqlKind.IS_DISTINCT_FROM; -import static org.apache.calcite.sql.SqlKind.IS_NOT_DISTINCT_FROM; import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.combinedRowType; /** @@ -279,6 +278,8 @@ public LogicalRelImplementor( RelDataType rightType = rel.getRight().getRowType(); JoinRelType joinType = rel.getJoinType(); + IgniteJoinInfo joinInfo = IgniteJoinInfo.of(rel); + RexNode nonEquiConditionExpression = RexUtil.composeConjunction(Commons.emptyCluster().getRexBuilder(), rel.analyzeCondition().nonEquiConditions, true); @@ -290,7 +291,8 @@ public LogicalRelImplementor( nonEquiCondition = expressionFactory.biPredicate(rel.getCondition(), rowType); } - Node node = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, rel.analyzeCondition(), nonEquiCondition); + Node node = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, joinInfo, + nonEquiCondition); node.register(Arrays.asList(visit(rel.getLeft()), visit(rel.getRight()))); @@ -327,12 +329,18 @@ public LogicalRelImplementor( RelDataType rightType = rel.getRight().getRowType(); JoinRelType joinType = rel.getJoinType(); + IgniteJoinInfo info = IgniteJoinInfo.of(rel); + + assert !info.pairs().isEmpty() && info.isEqui(); + int pairsCnt = rel.analyzeCondition().pairs().size(); + // TODO : revise after https://issues.apache.org/jira/browse/IGNITE-26048 + assert !info.hasMatchingNulls() || info.matchingNullsCnt() == info.pairs().size(); Comparator comp = expressionFactory.comparator( rel.leftCollation().getFieldCollations().subList(0, pairsCnt), rel.rightCollation().getFieldCollations().subList(0, pairsCnt), - rel.getCondition().getKind() == IS_NOT_DISTINCT_FROM || rel.getCondition().getKind() == IS_DISTINCT_FROM + info.hasMatchingNulls() ); Node node = MergeJoinNode.create(ctx, outType, leftType, rightType, joinType, comp, hasExchange(rel)); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java index 6d5a9b3f61b25..ff79d8f887f67 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java @@ -19,6 +19,7 @@ import java.lang.reflect.Type; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.ImmutableIntList; /** * Read only handler to process subset of columns. @@ -28,22 +29,28 @@ public class MappingRowHandler implements RowHandler { private final RowHandler delegate; /** */ - private final int[] mapping; + private final ImmutableIntList mapping; - /** */ + /** Creates a mapping handler based on a bit set mapping. Keeps columns unique, but may change required colums order. */ public MappingRowHandler(RowHandler delegate, ImmutableBitSet requiredColumns) { this.delegate = delegate; - mapping = requiredColumns.toArray(); + mapping = ImmutableIntList.of(requiredColumns.toArray()); + } + + /** Creates a mapping handler based on a array mapping as is. */ + public MappingRowHandler(RowHandler delegate, ImmutableIntList requiredColumns) { + this.delegate = delegate; + mapping = requiredColumns; } /** {@inheritDoc} */ @Override public Object get(int field, Row row) { - return delegate.get(mapping[field], row); + return delegate.get(mapping.get(field), row); } /** {@inheritDoc} */ @Override public int columnCount(Row row) { - return mapping.length; + return mapping.size(); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java index ff705549c037c..21e1cf9effe1e 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java @@ -21,8 +21,10 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.Map; import java.util.function.Supplier; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey; import org.jetbrains.annotations.Nullable; @@ -30,6 +32,17 @@ * Runtime hash index based on on-heap hash map. */ public class RuntimeHashIndex implements RuntimeIndex { + /** Allowed key for null values but matching no any other key. */ + private static final GroupKey NON_MATCHING_NULLS_KEY = new GroupKey<>(null, null) { + @Override public boolean equals(Object o) { + throw new UnsupportedOperationException("Rows with null values must not be compared at all."); + } + + @Override public int hashCode() { + return 0; + } + }; + /** */ protected final ExecutionContext ectx; @@ -37,7 +50,7 @@ public class RuntimeHashIndex implements RuntimeIndex { private final RowHandler keysRowHnd; /** Rows. */ - private final HashMap, Collection> rows; + private final Map, Collection> rows; /** */ private final Supplier> collectionFactory; @@ -45,22 +58,27 @@ public class RuntimeHashIndex implements RuntimeIndex { /** Allow NULL values. */ private final boolean allowNulls; + /** */ + private final boolean nullsMatch; + /** Creates hash index with the default collection supplier. */ public RuntimeHashIndex(ExecutionContext ectx, ImmutableBitSet keys, boolean allowNulls) { - this(ectx, keys, allowNulls, -1, null); + this(ectx, ImmutableIntList.of(keys.toArray()), allowNulls, true, -1, null); } /** */ public RuntimeHashIndex( ExecutionContext ectx, - ImmutableBitSet keys, + ImmutableIntList keys, boolean allowNulls, + boolean nullsMatch, int initCapacity, @Nullable Supplier> collectionFactory ) { this( ectx, allowNulls, + nullsMatch, new MappingRowHandler<>(ectx.rowHandler(), keys), initCapacity >= 0 ? new HashMap<>(initCapacity) : new HashMap<>(), collectionFactory @@ -71,12 +89,14 @@ public RuntimeHashIndex( private RuntimeHashIndex( ExecutionContext ectx, boolean allowNulls, + boolean nullsMatch, RowHandler keysRowHnd, - HashMap, Collection> rows, + Map, Collection> rows, @Nullable Supplier> collectionFactory ) { this.ectx = ectx; this.allowNulls = allowNulls; + this.nullsMatch = allowNulls && nullsMatch; this.keysRowHnd = keysRowHnd; this.rows = rows; @@ -117,10 +137,10 @@ public IndexScan scan(Supplier searchRow) { * IS NOT DISTINCT FROM condition). */ private @Nullable GroupKey key(Row r) { - if (!allowNulls) { + if (!allowNulls || !nullsMatch) { for (int i = 0; i < keysRowHnd.columnCount(r); i++) { if (keysRowHnd.get(i, r) == null) - return null; + return allowNulls ? NON_MATCHING_NULLS_KEY : null; } } @@ -128,7 +148,7 @@ public IndexScan scan(Supplier searchRow) { } /** */ - public RuntimeHashIndex remappedSearcher(ImmutableBitSet remappedKeys) { + public RuntimeHashIndex remappedSearcher(int[] remappedKeys) { return new RemappedSearcher<>(this, remappedKeys); } @@ -138,11 +158,11 @@ private static class RemappedSearcher extends RuntimeHashIndex { private final RuntimeHashIndex origin; /** */ - private RemappedSearcher(RuntimeHashIndex origin, ImmutableBitSet remappedKeys){ - super(origin.ectx, origin.allowNulls, new MappingRowHandler<>(origin.ectx.rowHandler(), remappedKeys), - origin.rows, origin.collectionFactory); + private RemappedSearcher(RuntimeHashIndex o, int[] remappedKeys) { + super(o.ectx, o.allowNulls, o.nullsMatch, new MappingRowHandler<>(o.ectx.rowHandler(), ImmutableIntList.of(remappedKeys)), + o.rows, o.collectionFactory); - this.origin = origin; + this.origin = o; } /** {@inheritDoc} */ @@ -169,8 +189,8 @@ private IndexScan(Supplier searchRow) { public @Nullable Collection get() { GroupKey key = key(searchRow.get()); - if (key == null) - return Collections.emptyList(); + if (key == null || key == NON_MATCHING_NULLS_KEY) + return null; return rows.get(key); } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 0142d9d0bcc57..e639c997ec154 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -21,19 +21,17 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.Map; -import java.util.Objects; import java.util.function.BiPredicate; -import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; import org.apache.ignite.internal.processors.query.calcite.exec.RuntimeHashIndex; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; /** Hash join implementor. */ @@ -41,9 +39,6 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod /** */ private static final int INITIAL_CAPACITY = 128; - /** All keys with null-fields are mapped to this object. */ - private static final Key NULL_KEY = new Key(); - /** */ private final int[] leftKeys; @@ -53,14 +48,11 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod /** Output row handler. */ protected final RowHandler outRowHnd; - /** */ - protected final Map> hashStore = U.newHashMap(INITIAL_CAPACITY); - protected final boolean keepRowsWithNull; -// /** */ -// protected final RuntimeHashIndex hashIdx; -// -// /** */ -// protected final RuntimeHashIndex remappedHashIdx; + /** Right rows storage. */ + protected final RuntimeHashIndex rightHashStore; + + /** Uses the right keys for find matching left rows. */ + protected final RuntimeHashIndex remappedLeftSearcher; /** */ protected Iterator rightIt = Collections.emptyIterator(); @@ -72,7 +64,7 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod * Creates hash join node. * * @param ctx Execution context. - * @param rowType Row type. + * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. * @param keepRowsWithNull {@code True} if we need to store the row from right shoulder even if it contains NULL in @@ -83,13 +75,17 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod protected HashJoinNode( ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, boolean keepRowsWithNull, @Nullable BiPredicate nonEqCond ) { super(ctx, rowType); + // For IS NOT DISTINCT we have to keep rows with null values. + if (info.hasMatchingNulls()) + keepRowsWithNull = true; + leftKeys = info.leftKeys.toIntArray(); rightKeys = info.rightKeys.toIntArray(); @@ -99,11 +95,16 @@ protected HashJoinNode( this.nonEqCond = nonEqCond; - this.keepRowsWithNull = keepRowsWithNull; + rightHashStore = new RuntimeHashIndex<>(ctx, info.rightKeys, keepRowsWithNull, info.hasMatchingNulls(), + INITIAL_CAPACITY, TouchedArrayList::new); -// hashIdx = new RuntimeHashIndex<>(ctx, ImmutableBitSet.of(info.rightKeys), keepRowsWithNull, INITIAL_CAPACITY, -// TouchedArrayList::new); -// remappedHashIdx = hashIdx.remappedSearcher(ImmutableBitSet.of(info.leftKeys)); + // We remap right keys to search left rows because values order might differ. + Map remappedKeys = new LinkedHashMap<>(rightKeys.length, 1.0f); + + for (int i = 0; i < rightKeys.length; ++i) + remappedKeys.put(rightKeys[i], leftKeys[i]); + + remappedLeftSearcher = rightHashStore.remappedSearcher(remappedKeys.values().stream().mapToInt(i -> i).toArray()); } /** {@inheritDoc} */ @@ -112,42 +113,45 @@ protected HashJoinNode( rightIt = Collections.emptyIterator(); - hashStore.clear(); -// hashIdx.close(); + rightHashStore.close(); } /** Creates certain join node. */ public static HashJoinNode create( ExecutionContext ctx, - RelDataType outRowType, + RelDataType rowType, RelDataType leftRowType, RelDataType rightRowType, JoinRelType type, - JoinInfo info, + IgniteJoinInfo info, @Nullable BiPredicate nonEqCond ) { + assert !info.pairs().isEmpty() && (info.isEqui() || type == JoinRelType.INNER || type == JoinRelType.SEMI); + assert !info.hasMatchingNulls() || info.matchingNullsCnt() == info.leftKeys.size(); + IgniteTypeFactory typeFactory = ctx.getTypeFactory(); RowHandler rowHnd = ctx.rowHandler(); switch (type) { case INNER: - return new InnerHashJoin<>(ctx, outRowType, info, rowHnd, nonEqCond); + return new InnerHashJoin<>(ctx, rowType, info, rowHnd, nonEqCond); case LEFT: - return new LeftHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType), nonEqCond); + return new LeftHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType), nonEqCond); case RIGHT: - return new RightHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), nonEqCond); + return new RightHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), nonEqCond); - case FULL: - return new FullOuterHashJoin<>(ctx, outRowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), + case FULL: { + return new FullOuterHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), rowHnd.factory(typeFactory, rightRowType), nonEqCond); + } case SEMI: - return new SemiHashJoin<>(ctx, outRowType, info, rowHnd, nonEqCond); + return new SemiHashJoin<>(ctx, rowType, info, rowHnd, nonEqCond); case ANTI: - return new AntiHashJoin<>(ctx, outRowType, info, rowHnd, nonEqCond); + return new AntiHashJoin<>(ctx, rowType, info, rowHnd, nonEqCond); default: throw new IllegalArgumentException("Join of type '" + type + "' isn't supported."); @@ -156,43 +160,22 @@ public static HashJoinNode create( /** */ protected Collection lookup(Row row) { -// Collection res = remappedHashIdx.scan(() -> row).get(); -// -// if (res == null) -// return Collections.emptyList(); -// -// assert res instanceof TouchedArrayList; -// -// ((TouchedArrayList)res).touched = true; -// -// return res == null ? Collections.emptyList() : res; -// TODO: remove - Key row0 = extractKey(row, leftKeys); - - // Key with null field can't be compared with other keys. - if (row0 == NULL_KEY) - return Collections.emptyList(); + Collection res = remappedLeftSearcher.scan(() -> row).get(); - TouchedCollection found = hashStore.get(row0); + if (res == null) + return Collections.emptyList(); - if (found != null) { - found.touched = true; + assert res instanceof TouchedArrayList; - return found.items(); - } + ((TouchedArrayList)res).touched = true; - return Collections.emptyList(); + return res; } /** */ protected Iterator untouched() { - return F.flat(F.iterator(hashStore.values(), TouchedCollection::items, true, v -> !v.touched)); + return F.flat(F.iterator(rightHashStore.rowSets(), c0 -> c0, true, c1 -> !((TouchedArrayList)c1).touched)); } -// -// /** */ -// protected Iterator untouched() { -// return F.flat(F.iterator(hashIdx.rowSets(), c0 -> c0, true, c1 -> !((TouchedArrayList)c1).touched)); -// } /** {@inheritDoc} */ @Override protected void pushRight(Row row) throws Exception { @@ -205,33 +188,12 @@ protected Iterator untouched() { waitingRight--; - //hashIdx.push(row); - - Key key = extractKey(row, rightKeys); - - // No storing in #hashIndex, if the row contains NULL. And we won't emit right part alone like in FULL OUTER and RIGHT joins. - if (keepRowsWithNull || key != NULL_KEY) { - TouchedCollection raw = hashStore.computeIfAbsent(key, k -> new TouchedCollection<>()); - - raw.add(row); - } + rightHashStore.push(row); if (waitingRight == 0) rightSource().request(waitingRight = IN_BUFFER_SIZE); } - /** */ - private Key extractKey(Row row, int[] mapping) { - RowHandler rowHnd = context().rowHandler(); - - for (int i : mapping) { - if (rowHnd.get(i, row) == null) - return NULL_KEY; - } - - return new RowWrapper<>(row, rowHnd, mapping); - } - /** */ protected void requestMoreOrEnd() throws Exception { if (waitingRight == 0) @@ -244,8 +206,7 @@ protected void requestMoreOrEnd() throws Exception { && !rightIt.hasNext()) { requested = 0; - hashStore.clear(); -// hashIdx.close(); + rightHashStore.close(); downstream().end(); } @@ -257,14 +218,14 @@ private static final class InnerHashJoin extends HashJoinNode { * Creates node for INNER JOIN. * * @param ctx Execution context. - * @param rowType Row type. + * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private InnerHashJoin(ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { @@ -285,24 +246,25 @@ private InnerHashJoin(ExecutionContext ctx, rightIt = lookup(left).iterator(); } - // Emits matched rows. - while (rightIt.hasNext()) { - checkState(); + if (rightIt.hasNext()) { + // Emits matched rows. + while (requested > 0 && rightIt.hasNext()) { + checkState(); - RowT right = rightIt.next(); + RowT right = rightIt.next(); - if (nonEqCond != null && !nonEqCond.test(left, right)) - continue; + if (nonEqCond != null && !nonEqCond.test(left, right)) + continue; - --requested; + --requested; - downstream().push(outRowHnd.concat(left, right)); + downstream().push(outRowHnd.concat(left, right)); + } - if (requested == 0) - break; + if (!rightIt.hasNext()) + left = null; } - - if (!rightIt.hasNext()) + else left = null; } } @@ -325,7 +287,7 @@ private static final class LeftHashJoin extends HashJoinNode { * * @param ctx Execution context. * @param info Join info. - * @param rowType Row tyoe. + * @param rowType Out row type. * @param outRowHnd Output row handler. * @param rightRowFactory Right row factory. * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. @@ -333,7 +295,7 @@ private static final class LeftHashJoin extends HashJoinNode { private LeftHashJoin( ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, RowHandler.RowFactory rightRowFactory, @Nullable BiPredicate nonEqCond @@ -369,21 +331,22 @@ private LeftHashJoin( rightIt = rightRows.iterator(); } - // Emit unmatched left row. - while (rightIt.hasNext()) { - checkState(); + if (rightIt.hasNext()) { + // Emit unmatched left row. + while (requested > 0 && rightIt.hasNext()) { + checkState(); - RowT right = rightIt.next(); + RowT right = rightIt.next(); - --requested; + --requested; - downstream().push(outRowHnd.concat(left, right)); + downstream().push(outRowHnd.concat(left, right)); + } - if (requested == 0) - break; + if (!rightIt.hasNext()) + left = null; } - - if (!rightIt.hasNext()) + else left = null; } } @@ -408,7 +371,7 @@ private static final class RightHashJoin extends HashJoinNode { * Creates node for RIGHT OUTER JOIN. * * @param ctx Execution context. - * @param rowType Row type. + * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. * @param leftRowFactory Left row factory. @@ -417,7 +380,7 @@ private static final class RightHashJoin extends HashJoinNode { private RightHashJoin( ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, RowHandler.RowFactory leftRowFactory, @Nullable BiPredicate nonEqCond @@ -445,21 +408,22 @@ private RightHashJoin( rightIt = lookup(left).iterator(); } - // Emits matched rows. - while (rightIt.hasNext()) { - checkState(); + if (rightIt.hasNext()) { + // Emits matched rows. + while (requested > 0 && rightIt.hasNext()) { + checkState(); - RowT right = rightIt.next(); + RowT right = rightIt.next(); - --requested; + --requested; - downstream().push(outRowHnd.concat(left, right)); + downstream().push(outRowHnd.concat(left, right)); + } - if (requested == 0) - break; + if (!rightIt.hasNext()) + left = null; } - - if (!rightIt.hasNext()) + else left = null; } } @@ -482,7 +446,7 @@ private RightHashJoin( RowT emptyLeft = leftRowFactory.create(); - while (rightIt.hasNext()) { + while (requested > 0 && rightIt.hasNext()) { checkState(); RowT right = rightIt.next(); @@ -492,9 +456,6 @@ private RightHashJoin( --requested; downstream().push(row); - - if (requested == 0) - break; } } finally { @@ -530,7 +491,7 @@ private static class FullOuterHashJoin extends HashJoinNode { * @param ctx Execution context. * @param rowType Row type. * @param info Join info. - * @param outRowHnd Output row handler. + * @param outRowHnd Out row handler. * @param leftRowFactory Left row factory. * @param rightRowFactory Right row factory. * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. @@ -538,7 +499,7 @@ private static class FullOuterHashJoin extends HashJoinNode { private FullOuterHashJoin( ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, RowHandler.RowFactory leftRowFactory, RowHandler.RowFactory rightRowFactory, @@ -599,7 +560,7 @@ private FullOuterHashJoin( } } - // Emit unmatched right rows.Add commentMore actions + // Emit unmatched right rows. if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; @@ -647,7 +608,7 @@ private static final class SemiHashJoin extends HashJoinNode { * Creates node for SEMI JOIN operator. * * @param ctx Execution context. - * @param rowType Row type. + * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. @@ -655,7 +616,7 @@ private static final class SemiHashJoin extends HashJoinNode { private SemiHashJoin( ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { @@ -671,16 +632,20 @@ private SemiHashJoin( while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { checkState(); - left = leftInBuf.remove(); + // Proceed with next left row, if previous was fully processed. + if (!rightIt.hasNext()) { + left = leftInBuf.remove(); - Collection rightRows = lookup(left); + rightIt = lookup(left).iterator(); + } - boolean anyMatched = !rightRows.isEmpty(); + boolean anyMatched = rightIt.hasNext() && nonEqCond == null; - if (anyMatched && nonEqCond != null) { - anyMatched = false; + if (!anyMatched) { + // Find any matched row. + while (rightIt.hasNext()) { + RowT right = rightIt.next(); - for (RowT right : rightRows) { if (nonEqCond.test(left, right)) { anyMatched = true; @@ -693,9 +658,12 @@ private SemiHashJoin( requested--; downstream().push(left); + + rightIt = Collections.emptyIterator(); } - left = null; + if (!rightIt.hasNext()) + left = null; } } finally { @@ -713,7 +681,7 @@ private static final class AntiHashJoin extends HashJoinNode { * Creates node for ANTI JOIN. * * @param ctx Execution context. - * @param rowType Row type. + * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. @@ -721,7 +689,7 @@ private static final class AntiHashJoin extends HashJoinNode { private AntiHashJoin( ExecutionContext ctx, RelDataType rowType, - JoinInfo info, + IgniteJoinInfo info, RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { @@ -759,90 +727,9 @@ private AntiHashJoin( } } - /** Non-comparable key object. */ - private static class Key { - - } - - /** Comparable key object. */ - private static final class RowWrapper extends Key { - /** */ - private final RowT row; - - /** */ - private final RowHandler handler; - - /** */ - private final int[] items; - - /** */ - private RowWrapper(RowT row, RowHandler hnd, int[] items) { - this.row = row; - this.handler = hnd; - this.items = items; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int hashCode = 0; - - for (int i : items) - hashCode += Objects.hashCode(handler.get(i, row)); - - return hashCode; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object obj) { - if (this == obj) - return true; - - if (obj == null || getClass() != obj.getClass()) - return false; - - RowWrapper row0 = (RowWrapper)obj; - - for (int i = 0; i < items.length; ++i) { - Object input = row0.handler.get(row0.items[i], row0.row); - Object cur = handler.get(items[i], row); - - boolean comp = Objects.equals(input, cur); - - if (!comp) - return comp; - } - - return true; - } - } - /** */ private static final class TouchedArrayList extends ArrayList { /** */ private boolean touched; } - - /** */ - private static final class TouchedCollection { - /** */ - private final Collection coll; - - /** */ - private boolean touched; - - /** */ - private TouchedCollection() { - this.coll = new ArrayList<>(); - } - - /** */ - private void add(RowT row) { - coll.add(row); - } - - /** */ - private Collection items() { - return coll; - } - } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java index bf141af36e61c..12b9deed6e982 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerHelper.java @@ -164,8 +164,6 @@ public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteL if (sqlNode.isA(ImmutableSet.of(SqlKind.INSERT, SqlKind.UPDATE, SqlKind.MERGE))) igniteRel = new FixDependentModifyNodeShuttle().visit(igniteRel); - System.err.println("TEST | plan:\n" + RelOptUtil.toString(igniteRel)); - return igniteRel; } catch (Throwable ex) { diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java new file mode 100644 index 0000000000000..cddfba4fcd308 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java @@ -0,0 +1,97 @@ +/* + * 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.ignite.internal.processors.query.calcite.rel; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import com.google.common.collect.ImmutableList; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.ImmutableIntList; + +/** */ +public class IgniteJoinInfo extends JoinInfo { + /** */ + private final Set matchingNulls; + + /** */ + protected IgniteJoinInfo( + ImmutableIntList leftKeys, + ImmutableIntList rightKeys, + Set matchingNulls, + ImmutableList nonEquis + ) { + super(leftKeys, rightKeys, nonEquis); + + this.matchingNulls = matchingNulls; + } + + /** */ + public static IgniteJoinInfo of(Join join) { + List leftKeys = new ArrayList<>(); + List rightKeys = new ArrayList<>(); + List filteredNulls = new ArrayList<>(); + List nonEquis = new ArrayList<>(); + + RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(), join.getCondition(), leftKeys, rightKeys, + filteredNulls, nonEquis); + + Set bs = Collections.emptySet(); + + for (int i = 0; i < filteredNulls.size(); ++i) { + if (!filteredNulls.get(i)) { + if (bs == Collections.EMPTY_SET) + bs = new HashSet<>(); + + bs.add(i); + } + } + + return new IgniteJoinInfo( + ImmutableIntList.of(leftKeys.stream().mapToInt(i -> i).toArray()), + ImmutableIntList.of(rightKeys.stream().mapToInt(i -> i).toArray()), + bs, + ImmutableList.copyOf(nonEquis) + ); + } + + /** */ + public static IgniteJoinInfo of(ImmutableIntList leftKeys, ImmutableIntList rightKeys) { + return new IgniteJoinInfo(leftKeys, rightKeys, Collections.emptySet(), ImmutableList.of()); + } + + /** */ + public boolean hasMatchingNulls() { + return !matchingNulls.isEmpty(); + } + + /** */ + public int matchingNullsCnt() { + return matchingNulls.size(); + } + + /** */ + public boolean matchingNull(int equiPairIdx) { + return matchingNulls.contains(equiPairIdx); + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java index 434b52f59f16c..48a78d1e3d7eb 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java @@ -17,14 +17,11 @@ package org.apache.ignite.internal.processors.query.calcite.rule; -import java.util.ArrayList; import java.util.EnumSet; -import java.util.List; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.PhysicalNode; import org.apache.calcite.rel.RelNode; @@ -34,7 +31,7 @@ import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashJoin; -import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; /** Hash join converter rule. */ public class HashJoinConverterRule extends AbstractIgniteJoinConverterRule { @@ -53,21 +50,18 @@ private HashJoinConverterRule() { @Override public boolean matchesJoin(RelOptRuleCall call) { LogicalJoin join = call.rel(0); - if (F.isEmpty(join.analyzeCondition().pairs())) - return false; - - List filterNulls = new ArrayList<>(); + IgniteJoinInfo joinInfo = IgniteJoinInfo.of(join); - RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(), join.getCondition(), new ArrayList<>(), - new ArrayList<>(), filterNulls); + if (joinInfo.pairs().isEmpty()) + return false; - // IS NOT DISTINCT currently not supported by HashJoin - if (filterNulls.stream().anyMatch(filter -> !filter)) + // IS NOT DISTINCT is currently not supported simultaneously with equi conditions. + if (joinInfo.hasMatchingNulls() && joinInfo.matchingNullsCnt() != joinInfo.pairs().size()) return false; // Current limitation: unmatched products on left or right part requires special handling of non-equi condition // on execution level. - return join.analyzeCondition().isEqui() || NON_EQ_CONDITIONS_SUPPORT.contains(join.getJoinType()); + return joinInfo.isEqui() || NON_EQ_CONDITIONS_SUPPORT.contains(join.getJoinType()); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/MergeJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/MergeJoinConverterRule.java index 13288a808d705..aa0e11f46996e 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/MergeJoinConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/MergeJoinConverterRule.java @@ -30,6 +30,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMergeJoin; import org.apache.ignite.internal.util.typedef.F; @@ -49,11 +50,12 @@ public MergeJoinConverterRule() { /** {@inheritDoc} */ @Override public boolean matchesJoin(RelOptRuleCall call) { - LogicalJoin logicalJoin = call.rel(0); + LogicalJoin join = call.rel(0); - JoinInfo joinInfo = JoinInfo.of(logicalJoin.getLeft(), logicalJoin.getRight(), logicalJoin.getCondition()); + IgniteJoinInfo info = IgniteJoinInfo.of(join); - return !F.isEmpty(joinInfo.pairs()) && joinInfo.isEqui(); + // TODO : revise after https://issues.apache.org/jira/browse/IGNITE-26048 + return !F.isEmpty(info.pairs()) && info.isEqui() && (!info.hasMatchingNulls() || info.matchingNullsCnt() == info.pairs().size()); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java index 0122fcd66895a..6ab1ea04565a9 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ExecutionTest.java @@ -23,10 +23,8 @@ import java.util.UUID; import com.google.common.collect.ImmutableSet; import org.apache.calcite.rel.core.CorrelationId; -import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; @@ -48,7 +46,6 @@ import static org.apache.calcite.rel.core.JoinRelType.LEFT; import static org.apache.calcite.rel.core.JoinRelType.RIGHT; import static org.apache.calcite.rel.core.JoinRelType.SEMI; -import static org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode.IN_BUFFER_SIZE; import static org.apache.ignite.internal.processors.query.calcite.util.Commons.getFieldFromBiRows; /** @@ -291,7 +288,10 @@ public void testRightJoin() { assertEquals(4, rows.size()); - checkDepartmentsJoinEmployeesResults(RIGHT, rows); + Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, rows.get(0)); + Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, rows.get(1)); + Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, rows.get(2)); + Assert.assertArrayEquals(new Object[] {2, "Ivan", null}, rows.get(3)); } /** @@ -408,7 +408,10 @@ public void testSemiJoin() { while (node.hasNext()) rows.add(node.next()); - checkDepartmentsJoinEmployeesResults(SEMI, rows); + assertEquals(2, rows.size()); + + Assert.assertArrayEquals(new Object[] {"Core"}, rows.get(0)); + Assert.assertArrayEquals(new Object[] {"SQL"}, rows.get(1)); } /** @@ -465,7 +468,7 @@ public void testAntiJoin() { assertEquals(1, rows.size()); - checkDepartmentsJoinEmployeesResults(ANTI, rows); + Assert.assertArrayEquals(new Object[] {"QA"}, rows.get(0)); } /** @@ -534,119 +537,6 @@ public void testCorrelatedNestedLoopJoin() { } } - /** Tests 'Select e.id, e.name, d.name as dep_name from DEP d join EMP e on e.DEPNO = d.DEPNO'. */ - @Test - public void testHashJoin() { - for (JoinRelType joinType : F.asList(LEFT, INNER, RIGHT, FULL, SEMI, ANTI)) { - ExecutionContext ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0); - IgniteTypeFactory tf = ctx.getTypeFactory(); - - RelDataType rowType = TypeUtils.createRowType(tf, int.class, String.class); - - ScanNode leftDeps = new ScanNode<>(ctx, rowType, Arrays.asList( - new Object[] {1, "Core"}, - new Object[] {2, "SQL"}, - new Object[] {3, "QA"} - )); - - rowType = TypeUtils.createRowType(tf, int.class, String.class, Integer.class); - - ScanNode rightEmps = new ScanNode<>(ctx, rowType, Arrays.asList( - new Object[] {0, "Igor", 1}, - new Object[] {1, "Roman", 2}, - new Object[] {2, "Ivan", null}, - new Object[] {3, "Alexey", 1} - )); - - RelDataType leftType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, String.class); - RelDataType rightType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, String.class, Integer.class); - RelDataType outType = TypeUtils.createRowType(ctx.getTypeFactory(), int.class, String.class, int.class, - String.class, Integer.class); - - rowType = TypeUtils.createRowType(tf, int.class, String.class, String.class); - - HashJoinNode join = HashJoinNode.create(ctx, outType, leftType, rightType, joinType, - JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(2)), null); - - join.register(F.asList(leftDeps, rightEmps)); - - ProjectNode project = new ProjectNode<>(ctx, rowType, - r -> joinType == SEMI || joinType == ANTI ? new Object[] {r[1]} : new Object[] {r[2], r[3], r[1]}); - - project.register(join); - - RootNode rootScan = new RootNode<>(ctx, rowType); - - rootScan.register(project); - - assert rootScan.hasNext(); - - ArrayList rows = new ArrayList<>(); - - while (rootScan.hasNext()) - rows.add(rootScan.next()); - - checkDepartmentsJoinEmployeesResults(joinType, rows); - } - } - - /** */ - private void checkDepartmentsJoinEmployeesResults(JoinRelType joinType, ArrayList results) { - switch (joinType) { - case LEFT: - assertEquals(4, results.size()); - - Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); - Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); - Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); - Assert.assertArrayEquals(new Object[] {null, null, "QA"}, results.get(3)); - break; - - case INNER: - assertEquals(3, results.size()); - - Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); - Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); - Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); - break; - - case RIGHT: - assertEquals(4, results.size()); - - Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); - Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); - Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); - Assert.assertArrayEquals(new Object[] {2, "Ivan", null}, results.get(3)); - break; - - case FULL: - assertEquals(5, results.size()); - - Assert.assertArrayEquals(new Object[] {0, "Igor", "Core"}, results.get(0)); - Assert.assertArrayEquals(new Object[] {3, "Alexey", "Core"}, results.get(1)); - Assert.assertArrayEquals(new Object[] {1, "Roman", "SQL"}, results.get(2)); - Assert.assertArrayEquals(new Object[] {null, null, "QA"}, results.get(3)); - Assert.assertArrayEquals(new Object[] {2, "Ivan", null}, results.get(4)); - break; - - case SEMI: - assertEquals(2, results.size()); - - Assert.assertArrayEquals(new Object[] {"Core"}, results.get(0)); - Assert.assertArrayEquals(new Object[] {"SQL"}, results.get(1)); - break; - - case ANTI: - assertEquals(1, results.size()); - - Assert.assertArrayEquals(new Object[] {"QA"}, results.get(0)); - break; - - default: - throw new IllegalArgumentException("Unknows join type."); - } - } - /** */ @Test public void testMergeJoin() { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java index 9b8d5a849351c..f326fa71e7365 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java @@ -23,11 +23,11 @@ import java.util.function.BiPredicate; import java.util.stream.IntStream; import java.util.stream.Stream; -import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory; import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils; import org.apache.ignite.internal.util.typedef.F; @@ -400,7 +400,7 @@ private static HashJoinNode createJoinNode( : TypeUtils.combinedRowType(tf, leftType, rightType); return HashJoinNode.create(ctx, outType, leftType, rightType, joinType, - JoinInfo.of(ImmutableIntList.of(2), ImmutableIntList.of(0)), postCondition); + IgniteJoinInfo.of(ImmutableIntList.of(2), ImmutableIntList.of(0)), postCondition); } /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java index 5a59a9be55446..93eb7e679103a 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java @@ -25,11 +25,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.stream.IntStream; -import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.testframework.GridTestUtils; @@ -37,12 +37,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import static org.apache.calcite.rel.core.JoinRelType.ANTI; import static org.apache.calcite.rel.core.JoinRelType.FULL; -import static org.apache.calcite.rel.core.JoinRelType.INNER; -import static org.apache.calcite.rel.core.JoinRelType.LEFT; -import static org.apache.calcite.rel.core.JoinRelType.RIGHT; -import static org.apache.calcite.rel.core.JoinRelType.SEMI; /** * @@ -61,7 +56,7 @@ public void testMergeJoinBuffers() throws Exception { assertTrue(((MergeJoinNode)node).rightInBuf.size() <= IN_BUFFER_SIZE); }; - doTestJoinBuffer(joinFactory, bufChecker); + doTestJoinBuffer(joinFactory, bufChecker, false); } /** */ @@ -73,13 +68,13 @@ public void testNLJoinBuffers() throws Exception { Consumer> bufChecker = (node) -> assertTrue(((AbstractRightMaterializedJoinNode)node).leftInBuf.size() <= IN_BUFFER_SIZE); - doTestJoinBuffer(joinFactory, bufChecker); + doTestJoinBuffer(joinFactory, bufChecker, false); } /** */ @Test public void testHashJoinBuffers() throws Exception { - JoinInfo joinInfo = JoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(0)); + IgniteJoinInfo joinInfo = IgniteJoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(0)); JoinFactory joinFactory = (ctx, outType, leftType, rightType, joinType) -> HashJoinNode.create(ctx, outType, leftType, rightType, joinType, joinInfo, null); @@ -87,7 +82,7 @@ public void testHashJoinBuffers() throws Exception { Consumer> bufChecker = (node) -> assertTrue(((AbstractRightMaterializedJoinNode)node).leftInBuf.size() <= IN_BUFFER_SIZE); - doTestJoinBuffer(joinFactory, bufChecker); + doTestJoinBuffer(joinFactory, bufChecker, true); } /** @@ -95,17 +90,19 @@ public void testHashJoinBuffers() throws Exception { * * @param joinFactory Creates certain join node. * @param joinBufChecker Finally check node after successfull run. + * @param sortResults If {@code true}, sorts results before checking. */ private void doTestJoinBuffer( JoinFactory joinFactory, - Consumer> joinBufChecker + Consumer> joinBufChecker, + boolean sortResults ) throws Exception { for (JoinRelType joinType : F.asList(FULL)) { if (log.isInfoEnabled()) log.info("Testing join of type '" + joinType + "'..."); - int size = IN_BUFFER_SIZE * 2 + IN_BUFFER_SIZE / 2; - int intersect = Math.max(10, IN_BUFFER_SIZE / 10); + int size = IN_BUFFER_SIZE * 2 + 1; + int intersect = 10; int leftTo = size + intersect; int rightTo = size * 2; @@ -155,6 +152,23 @@ private void doTestJoinBuffer( assertTrue(GridTestUtils.waitForCondition(finished::get, getTestTimeout())); + // Sorting might be needed because join may not produce a sorted result. + if (sortResults) { + res.sort(new Comparator<>() { + @Override public int compare(Object[] row0, Object[] row1) { + assert row0.length == row1.length; + assert row0.length == 2; + assert row0[0] == row0[1] && row0[0] != null || (row0[0] != null || row0[1] != null); + assert row1[0] == row1[1] && row1[0] != null || (row1[0] != null || row1[1] != null); + + int v1 = (int)(row0[0] == null ? row0[1] : row0[0]); + int v2 = (int)(row1[0] == null ? row1[1] : row1[0]); + + return Integer.compare(v1, v2); + } + }); + } + switch (joinType) { case LEFT: assertEquals(size + intersect, res.size()); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java index 677d23fc57ec4..43d5d022c246c 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java @@ -50,14 +50,14 @@ public static List params() { @Override protected void init() throws Exception { super.init(); - executeSql("create table t1 (c1 int, c2 int, c3 int) WITH " + atomicity()); - executeSql("create table t2 (c1 int, c2 int, c3 int) WITH " + atomicity()); + sql("create table t1 (c1 int, c2 int, c3 int) WITH " + atomicity()); + sql("create table t2 (c1 int, c2 int, c3 int) WITH " + atomicity()); - executeSql("create index t1_idx on t1 (c3, c2, c1)"); - executeSql("create index t2_idx on t2 (c3, c2, c1)"); + sql("create index t1_idx on t1 (c3, c2, c1)"); + sql("create index t2_idx on t2 (c3, c2, c1)"); - executeSql("insert into t1 values (1, 1, 1), (2, null, 2), (2, 2, 2), (3, 3, null), (3, 3, 3), (4, 4, 4)"); - executeSql("insert into t2 values (1, 1, 1), (2, 2, null), (2, 2, 2), (3, null, 3), (3, 3, 3), (4, 4, 4)"); + sql("insert into t1 values (1, 1, 1), (2, null, 2), (2, 2, 2), (3, 3, null), (3, 3, 3), (4, 4, 4)"); + sql("insert into t2 values (1, 1, 1), (2, 2, null), (2, 2, 2), (3, null, 3), (3, 3, 3), (4, 4, 4)"); } /** {@inheritDoc} */ @@ -66,7 +66,7 @@ public static List params() { } /** - * Test verifies result of inner join with different ordering. + * Test verifies result of inner join. */ @Test public void testInnerJoin() { @@ -251,10 +251,84 @@ public void testInnerJoin() { .returns(4, 4, 4, 4, 4) .returns(null, 3, 3, 3, 3) .check(); + + assertQuery("select t1.c2, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 order by t1.c2, t2.c3") + .returns(1, 1) + .returns(2, 2) + .returns(3, 3) + .returns(3, 3) + .returns(3, 3) + .returns(3, 3) + .returns(4, 4) + .returns(null, null) + .check(); + + assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 is not distinct from t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") + .returns(1, 1, 1, 1) + .returns(2, 2, 2, 2) + .returns(3, 3, 3, 3) + .returns(3, 3, 3, 3) + .returns(4, 4, 4, 4) + .returns(null, 2, 2, null) + .check(); + + assertQuery("select t1.c2, t1.c3, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 and t1.c3 > 3") + .returns(4, 4, 4) + .check(); + + // HASH JOIN doesn't support: completely non-equi conditions, additional non-equi conditions (post filters) + // except INNER and SEMI joins, equi and IS NOT DISTINCT conditions simultaneously. + // MERGE JOIN doesn't support: non-equi conditions, equi and IS NOT DISTINCT conditions simultaneously. + if (joinType == JoinType.HASH || joinType == JoinType.MERGE) + return; + + assertQuery("select t1.c2, t1.c3, t2.c2+3 as t2c2, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 3" + + " order by t1.c2, t1.c3, t2.c3") + .returns(1, 1, null) + .returns(2, 2, null) + .returns(3, 3, null) + .returns(3, null, null) + .returns(4, 4, 4) + .returns(null, 2, null) + .check(); + + assertQuery("select t1.c2, t1.c3, t2.c2+3 as t2c2, t2.c3 from t1 left join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 3" + + " order by t1.c2, t1.c3, t2.c3") + .returns(4, 4, 4) + .returns(null, null, 1) + .returns(null, null, 2) + .returns(null, null, 3) + .returns(null, null, 3) + .returns(null, null, null) + .check(); + + assertQuery("select t1.c2, t1.c3, t2.c2+3 as t2c2, t2.c3 from t1 right join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 3" + + " order by t1.c2, t1.c3, t2.c3") + .returns(1, 1, null) + .returns(2, 2, null) + .returns(3, 3, null) + .returns(3, null, null) + .returns(4, 4, 4) + .returns(null, 2, null) + .returns(null, null, 1) + .returns(null, null, 2) + .returns(null, null, 3) + .returns(null, null, 3) + .returns(null, null, null) + .check(); + + assertQuery("select t1.c2, t1.c3, t2.c2+3 as t2c2, t2.c3 from t1 full join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3> testParams = F.asList( - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1", true, false), - F.asList("select t1.c1 from t1 %s join t1 t2 using(c1)", true, false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1", false, false), - F.asList("select t1.c1 from t1 %s join t1 t2 ON t1.id is not distinct from t2.c1", false, false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = ?", false, false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = OCTET_LENGTH('TEST')", false, false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = LOG10(t1.c1)", false, false), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = t2.c1 and t1.ID > t2.ID", true, true), - F.asList("select t1.c1 from t1 %s join t1 t2 on t1.c1 = 1 and t2.c1 = 1", false, false) + F.asList("select t1.c1 from t1 %s join t2 on t1.id = t2.id", true, false), + F.asList("select t1.c1 from t1 %s join t2 on t1.id = t2.id and t1.c1=t2.c1", true, false), + F.asList("select t1.c1 from t1 %s join t2 using(c1)", true, false), + F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = 1", false, false), + F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1", true, false), + F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 is not distinct from t2.id", + true, false), + F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 = t2.id", false, false), + F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 > t2.id", true, true), + F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = ?", false, false), + F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = OCTET_LENGTH('TEST')", false, false), + F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = LOG10(t1.c1)", false, false), + F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = t2.c1 and t1.ID > t2.ID", true, true), + F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = 1 and t2.c1 = 1", false, false) ); for (List paramSet : testParams) { @@ -150,9 +149,10 @@ public void testHashJoinApplied() throws Exception { boolean canBePlanned = (Boolean)paramSet.get(1); boolean onlyInnerOrSemi = (Boolean)paramSet.get(2); - TestTable tbl = createTable("T1", IgniteDistributions.single(), "ID", Integer.class, "C1", Integer.class); + TestTable tbl1 = createTable("T1", IgniteDistributions.single(), "ID", Integer.class, "C1", Integer.class); + TestTable tbl2 = createTable("T2", IgniteDistributions.single(), "ID", Integer.class, "C1", Integer.class); - IgniteSchema schema = createSchema(tbl); + IgniteSchema schema = createSchema(tbl1, tbl2); for (String joinType : JOIN_TYPES) { if (onlyInnerOrSemi && !joinType.equals("INNER") && !joinType.equals("SEMI")) @@ -160,6 +160,9 @@ public void testHashJoinApplied() throws Exception { String sql0 = String.format(sql, joinType); + if (log.isInfoEnabled()) + log.info("Testing query '" + sql0 + "' for join type " + joinType); + if (canBePlanned) assertPlan(sql0, schema, nodeOrAnyChild(isInstanceOf(IgniteHashJoin.class)), DISABLED_RULES); else { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java deleted file mode 100644 index e69de29bb2d1d..0000000000000 From 8ef4118376e85691a71b7331d666f6dc213eb67e Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 29 Jul 2025 22:11:06 +0300 Subject: [PATCH 15/28] self-review fixes --- .../query/calcite/rel/IgniteJoinInfo.java | 34 ++++++------------- .../exec/rel/AbstractExecutionTest.java | 5 --- 2 files changed, 11 insertions(+), 28 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java index cddfba4fcd308..1f164942cd17e 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java @@ -18,10 +18,7 @@ package org.apache.ignite.internal.processors.query.calcite.rel; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.Set; import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.core.Join; @@ -31,19 +28,19 @@ /** */ public class IgniteJoinInfo extends JoinInfo { - /** */ - private final Set matchingNulls; + /** Filtered nulls of equi conditions. It usually means presence of IS NOT DISTINCT. */ + private final int matchingNullsCnt; /** */ protected IgniteJoinInfo( ImmutableIntList leftKeys, ImmutableIntList rightKeys, - Set matchingNulls, + int matchingNullsCnt, ImmutableList nonEquis ) { super(leftKeys, rightKeys, nonEquis); - this.matchingNulls = matchingNulls; + this.matchingNullsCnt = matchingNullsCnt; } /** */ @@ -56,42 +53,33 @@ public static IgniteJoinInfo of(Join join) { RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(), join.getCondition(), leftKeys, rightKeys, filteredNulls, nonEquis); - Set bs = Collections.emptySet(); + int matchingNullsCnt = 0; for (int i = 0; i < filteredNulls.size(); ++i) { - if (!filteredNulls.get(i)) { - if (bs == Collections.EMPTY_SET) - bs = new HashSet<>(); - - bs.add(i); - } + if (!filteredNulls.get(i)) + ++matchingNullsCnt; } return new IgniteJoinInfo( ImmutableIntList.of(leftKeys.stream().mapToInt(i -> i).toArray()), ImmutableIntList.of(rightKeys.stream().mapToInt(i -> i).toArray()), - bs, + matchingNullsCnt, ImmutableList.copyOf(nonEquis) ); } /** */ public static IgniteJoinInfo of(ImmutableIntList leftKeys, ImmutableIntList rightKeys) { - return new IgniteJoinInfo(leftKeys, rightKeys, Collections.emptySet(), ImmutableList.of()); + return new IgniteJoinInfo(leftKeys, rightKeys, 0, ImmutableList.of()); } /** */ public boolean hasMatchingNulls() { - return !matchingNulls.isEmpty(); + return matchingNullsCnt != 0; } /** */ public int matchingNullsCnt() { - return matchingNulls.size(); - } - - /** */ - public boolean matchingNull(int equiPairIdx) { - return matchingNulls.contains(equiPairIdx); + return matchingNullsCnt; } } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java index de1cbfd0dc318..3b33a0a143e40 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java @@ -342,11 +342,6 @@ protected ExecutionContext executionContext() { return executionContext(nodes.get(new Random().nextInt(nodesCnt)), UUID.randomUUID(), 0); } - /** */ - protected ExecutionContext executionContext(UUID nodeId) { - return executionContext(nodeId, UUID.randomUUID(), 0); - } - /** */ protected ExecutionContext executionContext(UUID nodeId, UUID qryId, long fragmentId) { FragmentDescription fragmentDesc = new FragmentDescription(fragmentId, null, null, null); From 3fb141c18c7e7852541d6ff151c9fa93f3ab4730 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 30 Jul 2025 12:33:35 +0300 Subject: [PATCH 16/28] optimized inputs requesting --- .../rel/AbstractRightMaterializedJoinNode.java | 3 +++ .../query/calcite/exec/rel/HashJoinNode.java | 16 +++++++++------- .../calcite/exec/rel/NestedLoopJoinNode.java | 3 --- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java index 5989e099869f9..2695bbc4aadc0 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -26,6 +26,9 @@ /** Right-part materialized join node. Holds data from the right part locally. */ public abstract class AbstractRightMaterializedJoinNode extends MemoryTrackingNode { + /** */ + protected static final int HALF_BUF_SIZE = IN_BUFFER_SIZE >> 1; + /** Special flag which marks that all the rows are received. */ protected static final int NOT_WAITING = -1; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index e639c997ec154..42f47b7d452a0 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -196,20 +196,22 @@ protected Iterator untouched() { /** */ protected void requestMoreOrEnd() throws Exception { - if (waitingRight == 0) - rightSource().request(waitingRight = IN_BUFFER_SIZE); - - if (waitingLeft == 0 && leftInBuf.isEmpty()) - leftSource().request(waitingLeft = IN_BUFFER_SIZE); - if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && leftInBuf.isEmpty() && left == null - && !rightIt.hasNext()) { + && !rightIt.hasNext()) { requested = 0; rightHashStore.close(); downstream().end(); + + return; } + + if (waitingRight == 0 && requested > 0) + rightSource().request(waitingRight = IN_BUFFER_SIZE); + + if (waitingLeft == 0 && leftInBuf.size() <= HALF_BUF_SIZE) + leftSource().request(waitingLeft = IN_BUFFER_SIZE - leftInBuf.size()); } /** */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index cd49013d7fde6..f8560a1a9afc8 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -28,9 +28,6 @@ /** */ public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJoinNode { - /** TODO : use also in hash join */ - private static final int HALF_BUF_SIZE = IN_BUFFER_SIZE >> 1; - /** */ protected final BiPredicate cond; From a29c239e0ec26d907806f86ef0cd1301f771c8be Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 30 Jul 2025 14:23:14 +0300 Subject: [PATCH 17/28] self-review fixes --- .../query/calcite/exec/RuntimeHashIndex.java | 2 + .../query/calcite/exec/rel/HashJoinNode.java | 37 ++++------- .../query/calcite/rel/IgniteHashJoin.java | 2 +- .../query/calcite/rel/IgniteJoinInfo.java | 2 +- .../exec/rel/HashJoinExecutionTest.java | 5 +- .../exec/rel/JoinBuffersExecutionTest.java | 62 ++++++++++++------- .../integration/JoinIntegrationTest.java | 21 ++++--- 7 files changed, 71 insertions(+), 60 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java index 21e1cf9effe1e..69e292534c9b0 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java @@ -137,6 +137,8 @@ public IndexScan scan(Supplier searchRow) { * IS NOT DISTINCT FROM condition). */ private @Nullable GroupKey key(Row r) { + assert !nullsMatch || allowNulls; + if (!allowNulls || !nullsMatch) { for (int i = 0; i < keysRowHnd.columnCount(r); i++) { if (keysRowHnd.get(i, r) == null) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 42f47b7d452a0..07abb343c651a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -21,8 +21,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.Map; import java.util.function.BiPredicate; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; @@ -51,7 +49,7 @@ public abstract class HashJoinNode extends AbstractRightMaterializedJoinNod /** Right rows storage. */ protected final RuntimeHashIndex rightHashStore; - /** Uses the right keys for find matching left rows. */ + /** Uses keys of right hand to find matching left rows. */ protected final RuntimeHashIndex remappedLeftSearcher; /** */ @@ -98,13 +96,7 @@ protected HashJoinNode( rightHashStore = new RuntimeHashIndex<>(ctx, info.rightKeys, keepRowsWithNull, info.hasMatchingNulls(), INITIAL_CAPACITY, TouchedArrayList::new); - // We remap right keys to search left rows because values order might differ. - Map remappedKeys = new LinkedHashMap<>(rightKeys.length, 1.0f); - - for (int i = 0; i < rightKeys.length; ++i) - remappedKeys.put(rightKeys[i], leftKeys[i]); - - remappedLeftSearcher = rightHashStore.remappedSearcher(remappedKeys.values().stream().mapToInt(i -> i).toArray()); + remappedLeftSearcher = rightHashStore.remappedSearcher(leftKeys); } /** {@inheritDoc} */ @@ -128,6 +120,7 @@ public static HashJoinNode create( ) { assert !info.pairs().isEmpty() && (info.isEqui() || type == JoinRelType.INNER || type == JoinRelType.SEMI); assert !info.hasMatchingNulls() || info.matchingNullsCnt() == info.leftKeys.size(); + assert nonEqCond == null || type == JoinRelType.INNER || type == JoinRelType.SEMI; IgniteTypeFactory typeFactory = ctx.getTypeFactory(); RowHandler rowHnd = ctx.rowHandler(); @@ -137,10 +130,10 @@ public static HashJoinNode create( return new InnerHashJoin<>(ctx, rowType, info, rowHnd, nonEqCond); case LEFT: - return new LeftHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType), nonEqCond); + return new LeftHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, rightRowType)); case RIGHT: - return new RightHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), nonEqCond); + return new RightHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType)); case FULL: { return new FullOuterHashJoin<>(ctx, rowType, info, rowHnd, rowHnd.factory(typeFactory, leftRowType), @@ -292,17 +285,15 @@ private static final class LeftHashJoin extends HashJoinNode { * @param rowType Out row type. * @param outRowHnd Output row handler. * @param rightRowFactory Right row factory. - * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private LeftHashJoin( ExecutionContext ctx, RelDataType rowType, IgniteJoinInfo info, RowHandler outRowHnd, - RowHandler.RowFactory rightRowFactory, - @Nullable BiPredicate nonEqCond + RowHandler.RowFactory rightRowFactory ) { - super(ctx, rowType, info, outRowHnd, false, nonEqCond); + super(ctx, rowType, info, outRowHnd, false, null); assert nonEqCond == null : "Non equi condition is not supported in LEFT join"; @@ -377,17 +368,15 @@ private static final class RightHashJoin extends HashJoinNode { * @param info Join info. * @param outRowHnd Output row handler. * @param leftRowFactory Left row factory. - * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private RightHashJoin( ExecutionContext ctx, RelDataType rowType, IgniteJoinInfo info, RowHandler outRowHnd, - RowHandler.RowFactory leftRowFactory, - @Nullable BiPredicate nonEqCond + RowHandler.RowFactory leftRowFactory ) { - super(ctx, rowType, info, outRowHnd, true, nonEqCond); + super(ctx, rowType, info, outRowHnd, true, null); assert nonEqCond == null : "Non equi condition is not supported in RIGHT join"; @@ -477,7 +466,7 @@ private RightHashJoin( } /** */ - private static class FullOuterHashJoin extends HashJoinNode { + private static final class FullOuterHashJoin extends HashJoinNode { /** Left row factory. */ private final RowHandler.RowFactory leftRowFactory; @@ -496,7 +485,6 @@ private static class FullOuterHashJoin extends HashJoinNode { * @param outRowHnd Out row handler. * @param leftRowFactory Left row factory. * @param rightRowFactory Right row factory. - * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private FullOuterHashJoin( ExecutionContext ctx, @@ -507,7 +495,7 @@ private FullOuterHashJoin( RowHandler.RowFactory rightRowFactory, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, true, nonEqCond); + super(ctx, rowType, info, outRowHnd, true, null); assert nonEqCond == null : "Non equi condition is not supported in FULL OUTER join"; @@ -686,7 +674,6 @@ private static final class AntiHashJoin extends HashJoinNode { * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. - * @param nonEqCond If provided, only rows matching the predicate will be emitted as matched rows. */ private AntiHashJoin( ExecutionContext ctx, @@ -695,7 +682,7 @@ private AntiHashJoin( RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, false, nonEqCond); + super(ctx, rowType, info, outRowHnd, false, null); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java index 3e012c551eace..b609cc795d478 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java @@ -36,7 +36,7 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCostFactory; import org.apache.ignite.internal.processors.query.calcite.util.Commons; -/** Represent hash join. */ +/** */ public class IgniteHashJoin extends AbstractIgniteJoin { /** */ public IgniteHashJoin( diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java index 1f164942cd17e..47ce4894afb94 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java @@ -26,7 +26,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.util.ImmutableIntList; -/** */ +/** Extended {@link JoinInfo}. */ public class IgniteJoinInfo extends JoinInfo { /** Filtered nulls of equi conditions. It usually means presence of IS NOT DISTINCT. */ private final int matchingNullsCnt; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java index f326fa71e7365..cd8a4ca5f4c5c 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java @@ -42,7 +42,6 @@ import static org.apache.calcite.rel.core.JoinRelType.SEMI; /** */ -@SuppressWarnings("TypeMayBeWeakened") public class HashJoinExecutionTest extends AbstractExecutionTest { /** */ @Test @@ -208,7 +207,7 @@ public void testInnerJoinWithPostFiltration() { new Object[] {3, "QA"} }; - BiPredicate condition = (l, r) -> ((String)r[1]).length() > 3 && ((String)l[1]).length() > 4; + BiPredicate condition = (l, r) -> ((CharSequence)r[1]).length() > 3 && ((CharSequence)l[1]).length() > 4; Object[][] expected = {{3, "Alexey", 1, 1, "Core"}}; @@ -231,7 +230,7 @@ public void testSemiJoinWithPostFiltration() { new Object[] {3, "QA"} }; - BiPredicate condition = (l, r) -> ((String)r[1]).length() > 3 && ((String)l[1]).length() > 4; + BiPredicate condition = (l, r) -> ((CharSequence)r[1]).length() > 3 && ((CharSequence)l[1]).length() > 4; Object[][] expected = {{3, "Alexey", 1}}; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java index 93eb7e679103a..b7cf320a8ee26 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java @@ -37,11 +37,14 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import static org.apache.calcite.rel.core.JoinRelType.ANTI; import static org.apache.calcite.rel.core.JoinRelType.FULL; +import static org.apache.calcite.rel.core.JoinRelType.INNER; +import static org.apache.calcite.rel.core.JoinRelType.LEFT; +import static org.apache.calcite.rel.core.JoinRelType.RIGHT; +import static org.apache.calcite.rel.core.JoinRelType.SEMI; -/** - * - */ +/** Tests that buffers of join nodes are cleared at the join end and that a join node is not stuck. */ @RunWith(Parameterized.class) public class JoinBuffersExecutionTest extends AbstractExecutionTest { /** */ @@ -97,12 +100,12 @@ private void doTestJoinBuffer( Consumer> joinBufChecker, boolean sortResults ) throws Exception { - for (JoinRelType joinType : F.asList(FULL)) { + for (JoinRelType joinType : F.asList(LEFT, INNER, RIGHT, FULL, SEMI, ANTI)) { if (log.isInfoEnabled()) log.info("Testing join of type '" + joinType + "'..."); - int size = IN_BUFFER_SIZE * 2 + 1; - int intersect = 10; + int size = IN_BUFFER_SIZE * 2 + IN_BUFFER_SIZE / 2; + int intersect = Math.max(10, IN_BUFFER_SIZE / 10); int leftTo = size + intersect; int rightTo = size * 2; @@ -153,21 +156,8 @@ private void doTestJoinBuffer( assertTrue(GridTestUtils.waitForCondition(finished::get, getTestTimeout())); // Sorting might be needed because join may not produce a sorted result. - if (sortResults) { - res.sort(new Comparator<>() { - @Override public int compare(Object[] row0, Object[] row1) { - assert row0.length == row1.length; - assert row0.length == 2; - assert row0[0] == row0[1] && row0[0] != null || (row0[0] != null || row0[1] != null); - assert row1[0] == row1[1] && row1[0] != null || (row1[0] != null || row1[1] != null); - - int v1 = (int)(row0[0] == null ? row0[1] : row0[0]); - int v2 = (int)(row1[0] == null ? row1[1] : row1[0]); - - return Integer.compare(v1, v2); - } - }); - } + if (sortResults) + sortResults(res, joinType); switch (joinType) { case LEFT: @@ -252,6 +242,36 @@ private void doTestJoinBuffer( } } + /** */ + private static void sortResults(List res, JoinRelType joinType) { + res.sort(new Comparator<>() { + @Override public int compare(Object[] row0, Object[] row1) { + assert row0.length == row1.length; + + int v1; + int v2; + + if (joinType == SEMI || joinType == ANTI) { + assert row0.length == 1; + assert row0[0] != null && row1[0] != null; + + v1 = (int)row0[0]; + v2 = (int)row1[0]; + } + else { + assert row0.length == 2; + assert (row0[0] == row0[1] && row0[0] != null) || row0[0] != null || row0[1] != null; + assert (row1[0] == row1[1] && row1[0] != null) || row1[0] != null || row1[1] != null; + + v1 = (int)(row0[0] == null ? row0[1] : row0[0]); + v2 = (int)(row1[0] == null ? row1[1] : row1[0]); + } + + return Integer.compare(v1, v2); + } + }); + } + /** */ @FunctionalInterface protected interface JoinFactory { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java index 43d5d022c246c..d209fa66d77e2 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java @@ -277,21 +277,24 @@ public void testInnerJoin() { .returns(4, 4, 4) .check(); + // MERGE JOIN doesn't support non-equi conditions. + if (joinType != JoinType.MERGE) { + assertQuery("select t1.c2, t1.c3, t2.c2+3 as t2c2, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 Date: Wed, 30 Jul 2025 19:27:41 +0300 Subject: [PATCH 18/28] test fix --- .../query/calcite/planner/CorrelatedSubqueryPlannerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedSubqueryPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedSubqueryPlannerTest.java index 2b3e17264c200..97e6a4debbb70 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedSubqueryPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/CorrelatedSubqueryPlannerTest.java @@ -336,8 +336,8 @@ public void testCorrelatedDistribution() throws Exception { // Correlate on top of another join. sql = "SELECT a FROM ta1 WHERE ta1.a IN (SELECT a FROM th1) AND EXISTS (SELECT 1 FROM ta2 WHERE ta2.b = ta1.a)"; assertPlan(sql, schema, hasChildThat(isInstanceOf(IgniteCorrelatedNestedLoopJoin.class) - .and(input(0, isInstanceOf(AbstractIgniteJoin.class))) - .and(input(1, isInstanceOf(IgniteColocatedAggregateBase.class) + .and(input(0, nodeOrAnyChild(isInstanceOf(AbstractIgniteJoin.class)))) + .and(input(1, nodeOrAnyChild(isInstanceOf(IgniteColocatedAggregateBase.class)) .and(hasChildThat(isInstanceOf(IgniteExchange.class)).negate()) )))); From 9b2036bb0ec0799ace89456cae2b2626ee55e228 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Fri, 16 Jan 2026 16:08:04 +0300 Subject: [PATCH 19/28] + master --- .../query/calcite/exec/rel/HashJoinNode.java | 6 +++--- .../query/calcite/rel/IgniteJoinInfo.java | 17 +++++++++++------ .../calcite/rule/HashJoinConverterRule.java | 2 +- .../calcite/exec/rel/HashJoinExecutionTest.java | 8 ++++++-- .../exec/rel/JoinBuffersExecutionTest.java | 5 ++++- 5 files changed, 25 insertions(+), 13 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 07abb343c651a..d645ebdf8f75b 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -81,7 +81,7 @@ protected HashJoinNode( super(ctx, rowType); // For IS NOT DISTINCT we have to keep rows with null values. - if (info.hasMatchingNulls()) + if (!info.allowNulls().isEmpty()) keepRowsWithNull = true; leftKeys = info.leftKeys.toIntArray(); @@ -93,7 +93,7 @@ protected HashJoinNode( this.nonEqCond = nonEqCond; - rightHashStore = new RuntimeHashIndex<>(ctx, info.rightKeys, keepRowsWithNull, info.hasMatchingNulls(), + rightHashStore = new RuntimeHashIndex<>(ctx, info.rightKeys, keepRowsWithNull, !info.allowNulls().isEmpty(), INITIAL_CAPACITY, TouchedArrayList::new); remappedLeftSearcher = rightHashStore.remappedSearcher(leftKeys); @@ -119,7 +119,7 @@ public static HashJoinNode create( @Nullable BiPredicate nonEqCond ) { assert !info.pairs().isEmpty() && (info.isEqui() || type == JoinRelType.INNER || type == JoinRelType.SEMI); - assert !info.hasMatchingNulls() || info.matchingNullsCnt() == info.leftKeys.size(); + assert info.allowNulls().isEmpty() || info.allowNulls().cardinality() == info.leftKeys.size(); assert nonEqCond == null || type == JoinRelType.INNER || type == JoinRelType.SEMI; IgniteTypeFactory typeFactory = ctx.getTypeFactory(); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java index ea3db26098233..95e2ecbd6b432 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoinInfo.java @@ -26,6 +26,7 @@ import org.apache.calcite.rex.RexNode; import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.ImmutableIntList; +import org.apache.ignite.internal.util.typedef.F; /** Extended {@link JoinInfo}. */ public class IgniteJoinInfo extends JoinInfo { @@ -33,7 +34,7 @@ public class IgniteJoinInfo extends JoinInfo { private final ImmutableBitSet allowNulls; /** */ - protected IgniteJoinInfo( + public IgniteJoinInfo( ImmutableIntList leftKeys, ImmutableIntList rightKeys, ImmutableBitSet allowNulls, @@ -54,17 +55,21 @@ public static IgniteJoinInfo of(Join join) { RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(), join.getCondition(), leftKeys, rightKeys, skipNulls, nonEquis); - ImmutableBitSet.Builder allowNulls = ImmutableBitSet.builder(); + ImmutableBitSet.Builder allowNulls = null; - for (int i = 0; i < skipNulls.size(); ++i) { - if (!skipNulls.get(i)) - allowNulls.set(i); + if (!F.isEmpty(skipNulls)) { + allowNulls = ImmutableBitSet.builder(); + + for (int i = 0; i < skipNulls.size(); ++i) { + if (!skipNulls.get(i)) + allowNulls.set(i); + } } return new IgniteJoinInfo( ImmutableIntList.copyOf(leftKeys), ImmutableIntList.copyOf(rightKeys), - allowNulls == null ? ImmutableBitSet.of() : ImmutableBitSet.of(allowNulls.build()), + allowNulls == null ? ImmutableBitSet.of() : allowNulls.build(), ImmutableList.copyOf(nonEquis) ); } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java index 48a78d1e3d7eb..92fb049e0ded5 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java @@ -56,7 +56,7 @@ private HashJoinConverterRule() { return false; // IS NOT DISTINCT is currently not supported simultaneously with equi conditions. - if (joinInfo.hasMatchingNulls() && joinInfo.matchingNullsCnt() != joinInfo.pairs().size()) + if (!joinInfo.allowNulls().isEmpty() && joinInfo.allowNulls().cardinality() != joinInfo.pairs().size()) return false; // Current limitation: unmatched products on left or right part requires special handling of non-equi condition diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java index cd8a4ca5f4c5c..27776792a55f1 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinExecutionTest.java @@ -23,8 +23,10 @@ import java.util.function.BiPredicate; import java.util.stream.IntStream; import java.util.stream.Stream; +import com.google.common.collect.ImmutableList; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; @@ -398,8 +400,10 @@ private static HashJoinNode createJoinNode( ? leftType : TypeUtils.combinedRowType(tf, leftType, rightType); - return HashJoinNode.create(ctx, outType, leftType, rightType, joinType, - IgniteJoinInfo.of(ImmutableIntList.of(2), ImmutableIntList.of(0)), postCondition); + IgniteJoinInfo joinInfo = new IgniteJoinInfo(ImmutableIntList.of(2), ImmutableIntList.of(0), + ImmutableBitSet.of(), ImmutableList.of()); + + return HashJoinNode.create(ctx, outType, leftType, rightType, joinType, joinInfo, postCondition); } /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java index b7cf320a8ee26..397fb702f4462 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinBuffersExecutionTest.java @@ -25,8 +25,10 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.stream.IntStream; +import com.google.common.collect.ImmutableList; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.ImmutableIntList; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoinInfo; @@ -77,7 +79,8 @@ public void testNLJoinBuffers() throws Exception { /** */ @Test public void testHashJoinBuffers() throws Exception { - IgniteJoinInfo joinInfo = IgniteJoinInfo.of(ImmutableIntList.of(0), ImmutableIntList.of(0)); + IgniteJoinInfo joinInfo = new IgniteJoinInfo(ImmutableIntList.of(0), ImmutableIntList.of(0), + ImmutableBitSet.of(), ImmutableList.of()); JoinFactory joinFactory = (ctx, outType, leftType, rightType, joinType) -> HashJoinNode.create(ctx, outType, leftType, rightType, joinType, joinInfo, null); From 414f886006be4d4cf9d2c498132570ad053ee4eb Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Fri, 16 Jan 2026 16:30:45 +0300 Subject: [PATCH 20/28] allow mix with is not distinct --- .../query/calcite/exec/rel/HashJoinNode.java | 1 - .../calcite/rule/HashJoinConverterRule.java | 4 - .../integration/JoinIntegrationTest.java | 105 +++++++++--------- .../calcite/planner/HashJoinPlannerTest.java | 2 +- 4 files changed, 52 insertions(+), 60 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index d645ebdf8f75b..dc6603086263b 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -119,7 +119,6 @@ public static HashJoinNode create( @Nullable BiPredicate nonEqCond ) { assert !info.pairs().isEmpty() && (info.isEqui() || type == JoinRelType.INNER || type == JoinRelType.SEMI); - assert info.allowNulls().isEmpty() || info.allowNulls().cardinality() == info.leftKeys.size(); assert nonEqCond == null || type == JoinRelType.INNER || type == JoinRelType.SEMI; IgniteTypeFactory typeFactory = ctx.getTypeFactory(); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java index 92fb049e0ded5..de1016d3c511a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/HashJoinConverterRule.java @@ -55,10 +55,6 @@ private HashJoinConverterRule() { if (joinInfo.pairs().isEmpty()) return false; - // IS NOT DISTINCT is currently not supported simultaneously with equi conditions. - if (!joinInfo.allowNulls().isEmpty() && joinInfo.allowNulls().cardinality() != joinInfo.pairs().size()) - return false; - // Current limitation: unmatched products on left or right part requires special handling of non-equi condition // on execution level. return joinInfo.isEqui() || NON_EQ_CONDITIONS_SUPPORT.contains(join.getJoinType()); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java index 035fe5b624554..8e81db6d4f1de 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java @@ -332,9 +332,19 @@ public void testInnerJoin() { .check(); } + assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") + .returns(1, 1, 1, 1) + .returns(2, 2, 2, 2) + .returns(3, 3, 3, 3) + .returns(3, 3, 3, 3) + .returns(4, 4, 4, 4) + .returns(null, 2, 2, null) + .check(); + // HASH JOIN doesn't support: completely non-equi conditions, additional non-equi conditions (post filters) - // except INNER and SEMI joins, equi and IS NOT DISTINCT conditions simultaneously. - // MERGE JOIN doesn't support: non-equi conditions, equi and IS NOT DISTINCT conditions simultaneously. + // except INNER and SEMI joins. + // MERGE JOIN doesn't support: non-equi conditions. if (joinType == JoinType.HASH || joinType == JoinType.MERGE) return; @@ -360,17 +370,6 @@ public void testInnerJoin() { .returns(1, 3) .returns(1, 3) .check(); - - // TODO : for merge join, revise after https://issues.apache.org/jira/browse/IGNITE-26048 - assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 join t2 on t1.c2 is not distinct from t2.c3 and " + - "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") - .returns(1, 1, 1, 1) - .returns(2, 2, 2, 2) - .returns(3, 3, 3, 3) - .returns(3, 3, 3, 3) - .returns(4, 4, 4, 4) - .returns(null, 2, 2, null) - .check(); } /** @@ -612,9 +611,20 @@ public void testLeftJoin() { .returns(null, 2, 2, null) .check(); + assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 left join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") + .returns(1, 1, 1, 1) + .returns(2, 2, 2, 2) + .returns(3, 3, 3, 3) + .returns(3, 3, 3, 3) + .returns(3, null, null, null) + .returns(4, 4, 4, 4) + .returns(null, 2, 2, null) + .check(); + // HASH JOIN doesn't support: completely non-equi conditions, additional non-equi conditions (post filters) - // except INNER and SEMI joins, equi and IS NOT DISTINCT conditions simultaneously. - // MERGE JOIN doesn't support: non-equi conditions, equi and IS NOT DISTINCT conditions simultaneously. + // except INNER and SEMI joins. + // MERGE JOIN doesn't support: non-equi conditions. if (joinType == JoinType.MERGE || joinType == JoinType.HASH) return; @@ -670,18 +680,6 @@ public void testLeftJoin() { .returns(3, null) .returns(4, null) .check(); - - // TODO : for merge join, revise after https://issues.apache.org/jira/browse/IGNITE-26048 - assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 left join t2 on t1.c2 is not distinct from t2.c3 and " + - "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") - .returns(1, 1, 1, 1) - .returns(2, 2, 2, 2) - .returns(3, 3, 3, 3) - .returns(3, 3, 3, 3) - .returns(3, null, null, null) - .returns(4, 4, 4, 4) - .returns(null, 2, 2, null) - .check(); } /** @@ -960,9 +958,19 @@ public void testRightJoin() { .returns(null, 2, 2, null) .check(); + assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 right join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") + .returns(1, 1, 1, 1) + .returns(2, 2, 2, 2) + .returns(3, 3, 3, 3) + .returns(3, 3, 3, 3) + .returns(4, 4, 4, 4) + .returns(null, 2, 2, null) + .check(); + // HASH JOIN doesn't support: completely non-equi conditions, additional non-equi conditions (post filters) - // except INNER and SEMI joins, equi and IS NOT DISTINCT conditions simultaneously. - // MERGE JOIN doesn't support: non-equi conditions, equi and IS NOT DISTINCT conditions simultaneously. + // except INNER and SEMI joins. + // MERGE JOIN doesn't support: non-equi conditions. if (joinType == JoinType.MERGE || joinType == JoinType.HASH) return; @@ -1017,17 +1025,6 @@ public void testRightJoin() { .returns(null, 4) .returns(null, null) .check(); - - // TODO : for merge join, revise after https://issues.apache.org/jira/browse/IGNITE-26048 - assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 right join t2 on t1.c2 is not distinct from t2.c3 and " + - "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") - .returns(1, 1, 1, 1) - .returns(2, 2, 2, 2) - .returns(3, 3, 3, 3) - .returns(3, 3, 3, 3) - .returns(4, 4, 4, 4) - .returns(null, 2, 2, null) - .check(); } /** @@ -1078,8 +1075,20 @@ public void testFullJoin() { .returns(null, 2, 2, null) .check(); -// // Merge join doesn't support absence of equi pairs and not equi conditions. -// // Hash join supports non-equi conditions for INNER and SEMI joins. + assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 full join t2 on t1.c2 is not distinct from t2.c3 and " + + "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") + .returns(1, 1, 1, 1) + .returns(2, 2, 2, 2) + .returns(3, 3, 3, 3) + .returns(3, 3, 3, 3) + .returns(3, null, null, null) + .returns(4, 4, 4, 4) + .returns(null, 2, 2, null) + .check(); + + // HASH JOIN doesn't support: completely non-equi conditions, additional non-equi conditions (post filters) + // except INNER and SEMI joins. + // MERGE JOIN doesn't support: non-equi conditions. if (joinType == JoinType.MERGE || joinType == JoinType.HASH) return; @@ -1150,18 +1159,6 @@ public void testFullJoin() { .returns(null, 4) .returns(null, null) .check(); - - // TODO : for merge join, revise after https://issues.apache.org/jira/browse/IGNITE-26048 - assertQuery("select t1.c2, t1.c3, t2.c1, t2.c3 from t1 full join t2 on t1.c2 is not distinct from t2.c3 and " + - "t1.c3 = t2.c1 order by t1.c2, t1.c3, t2.c1, t2.c3") - .returns(1, 1, 1, 1) - .returns(2, 2, 2, 2) - .returns(3, 3, 3, 3) - .returns(3, 3, 3, 3) - .returns(3, null, null, null) - .returns(4, 4, 4, 4) - .returns(null, 2, 2, null) - .check(); } /** diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java index e59c874edbb85..1b68cee57c3cf 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/HashJoinPlannerTest.java @@ -133,7 +133,7 @@ public void testHashJoinApplied() throws Exception { F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1", true, false), F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 is not distinct from t2.id", true, false), - F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 = t2.id", false, false), + F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 = t2.id", true, false), F.asList("select t1.c1 from t1 %s join t2 ON t1.id is not distinct from t2.c1 and t1.c1 > t2.id", true, true), F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = ?", false, false), F.asList("select t1.c1 from t1 %s join t2 on t1.c1 = OCTET_LENGTH('TEST')", false, false), From 304c43a20b1150ca5686042a560dbcfcd506d41c Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 17 Jan 2026 17:43:38 +0300 Subject: [PATCH 21/28] review fixes --- .../query/calcite/exec/MappingRowHandler.java | 15 ++-- .../query/calcite/exec/RuntimeHashIndex.java | 85 ++++++++++--------- .../query/calcite/exec/exp/agg/GroupKey.java | 10 ++- .../AbstractRightMaterializedJoinNode.java | 9 ++ .../query/calcite/exec/rel/HashJoinNode.java | 16 ++-- .../calcite/exec/rel/IndexSpoolNode.java | 4 +- .../calcite/exec/rel/NestedLoopJoinNode.java | 9 -- .../query/calcite/rel/IgniteHashJoin.java | 6 +- .../ignite/testsuites/ExecutionTestSuite.java | 2 - 9 files changed, 85 insertions(+), 71 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java index ff79d8f887f67..06947d6d5cfa0 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MappingRowHandler.java @@ -19,7 +19,6 @@ import java.lang.reflect.Type; import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; /** * Read only handler to process subset of columns. @@ -29,28 +28,28 @@ public class MappingRowHandler implements RowHandler { private final RowHandler delegate; /** */ - private final ImmutableIntList mapping; + private final int[] mapping; - /** Creates a mapping handler based on a bit set mapping. Keeps columns unique, but may change required colums order. */ + /** */ public MappingRowHandler(RowHandler delegate, ImmutableBitSet requiredColumns) { this.delegate = delegate; - mapping = ImmutableIntList.of(requiredColumns.toArray()); + mapping = requiredColumns.toArray(); } - /** Creates a mapping handler based on a array mapping as is. */ - public MappingRowHandler(RowHandler delegate, ImmutableIntList requiredColumns) { + /** */ + public MappingRowHandler(RowHandler delegate, int[] requiredColumns) { this.delegate = delegate; mapping = requiredColumns; } /** {@inheritDoc} */ @Override public Object get(int field, Row row) { - return delegate.get(mapping.get(field), row); + return delegate.get(mapping[field], row); } /** {@inheritDoc} */ @Override public int columnCount(Row row) { - return mapping.size(); + return mapping.length; } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java index 69e292534c9b0..a37311afd6cc4 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java @@ -24,7 +24,9 @@ import java.util.Map; import java.util.function.Supplier; import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey; import org.jetbrains.annotations.Nullable; @@ -32,17 +34,6 @@ * Runtime hash index based on on-heap hash map. */ public class RuntimeHashIndex implements RuntimeIndex { - /** Allowed key for null values but matching no any other key. */ - private static final GroupKey NON_MATCHING_NULLS_KEY = new GroupKey<>(null, null) { - @Override public boolean equals(Object o) { - throw new UnsupportedOperationException("Rows with null values must not be compared at all."); - } - - @Override public int hashCode() { - return 0; - } - }; - /** */ protected final ExecutionContext ectx; @@ -55,29 +46,24 @@ public class RuntimeHashIndex implements RuntimeIndex { /** */ private final Supplier> collectionFactory; - /** Allow NULL values. */ - private final boolean allowNulls; - /** */ - private final boolean nullsMatch; + private final ImmutableBitSet nullsMatch; /** Creates hash index with the default collection supplier. */ - public RuntimeHashIndex(ExecutionContext ectx, ImmutableBitSet keys, boolean allowNulls) { - this(ectx, ImmutableIntList.of(keys.toArray()), allowNulls, true, -1, null); + public RuntimeHashIndex(ExecutionContext ectx, ImmutableBitSet keys, ImmutableBitSet nullsMatch) { + this(ectx, keys.toArray(), nullsMatch, -1, null); } /** */ public RuntimeHashIndex( ExecutionContext ectx, - ImmutableIntList keys, - boolean allowNulls, - boolean nullsMatch, + int[] keys, + ImmutableBitSet nullsMatch, int initCapacity, @Nullable Supplier> collectionFactory ) { this( ectx, - allowNulls, nullsMatch, new MappingRowHandler<>(ectx.rowHandler(), keys), initCapacity >= 0 ? new HashMap<>(initCapacity) : new HashMap<>(), @@ -88,15 +74,13 @@ public RuntimeHashIndex( /** Fields setting constructor. */ private RuntimeHashIndex( ExecutionContext ectx, - boolean allowNulls, - boolean nullsMatch, + ImmutableBitSet nullsMatch, RowHandler keysRowHnd, Map, Collection> rows, @Nullable Supplier> collectionFactory ) { this.ectx = ectx; - this.allowNulls = allowNulls; - this.nullsMatch = allowNulls && nullsMatch; + this.nullsMatch = nullsMatch; this.keysRowHnd = keysRowHnd; this.rows = rows; @@ -137,16 +121,7 @@ public IndexScan scan(Supplier searchRow) { * IS NOT DISTINCT FROM condition). */ private @Nullable GroupKey key(Row r) { - assert !nullsMatch || allowNulls; - - if (!allowNulls || !nullsMatch) { - for (int i = 0; i < keysRowHnd.columnCount(r); i++) { - if (keysRowHnd.get(i, r) == null) - return allowNulls ? NON_MATCHING_NULLS_KEY : null; - } - } - - return new GroupKey<>(r, keysRowHnd); + return new NullsCheckingGroupKey<>(r, keysRowHnd); } /** */ @@ -154,6 +129,36 @@ public RuntimeHashIndex remappedSearcher(int[] remappedKeys) { return new RemappedSearcher<>(this, remappedKeys); } + /** */ + private class NullsCheckingGroupKey extends GroupKey { + /** */ + private NullsCheckingGroupKey(Row row, RowHandler hnd) { + super(row, hnd); + } + + /** {@inheritDoc} */ + @Override protected boolean columnValuesEquals(int colIdx, Object v1, Object v2) { + if (v1 == null && v2 == null) { + if (nullsMatch.cardinality() == 0) + return false; + + return nullsMatch.get(colIdx); + } + + return super.columnValuesEquals(colIdx, v1, v2); + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + throw new UnsupportedOperationException("Serialization of row keys is not supported."); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + throw new UnsupportedOperationException("Deserialization of row keys is not supported."); + } + } + /** */ private static class RemappedSearcher extends RuntimeHashIndex { /** */ @@ -161,10 +166,10 @@ private static class RemappedSearcher extends RuntimeHashIndex { /** */ private RemappedSearcher(RuntimeHashIndex o, int[] remappedKeys) { - super(o.ectx, o.allowNulls, o.nullsMatch, new MappingRowHandler<>(o.ectx.rowHandler(), ImmutableIntList.of(remappedKeys)), - o.rows, o.collectionFactory); + super(o.ectx, o.nullsMatch, new MappingRowHandler<>(o.ectx.rowHandler(), remappedKeys), o.rows, + o.collectionFactory); - this.origin = o; + origin = o; } /** {@inheritDoc} */ @@ -191,7 +196,7 @@ private IndexScan(Supplier searchRow) { public @Nullable Collection get() { GroupKey key = key(searchRow.get()); - if (key == null || key == NON_MATCHING_NULLS_KEY) + if (key == null) return null; return rows.get(key); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java index 42fb13507b97a..c771857b35c96 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java @@ -95,13 +95,21 @@ public RowHandler rowHandler() { return false; for (int i = 0; i < colCnt; i++) { - if (!Objects.equals(hnd.get(i, row), other.hnd.get(i, other.row))) + Object o1 = hnd.get(i, row); + Object o2 = other.hnd.get(i, other.row); + + if (!columnValuesEquals(i, o1, o2) || !Objects.equals(o1, o2)) return false; } return true; } + /** */ + protected boolean columnValuesEquals(int colIdx, Object val1, Object val2) { + return true; + } + /** {@inheritDoc} */ @Override public int hashCode() { int hashCode = 0; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java index 2695bbc4aadc0..1b0d712deff39 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -158,6 +158,15 @@ private void endRight() throws Exception { join(); } + /** */ + protected void tryToRequestInputs() throws Exception { + if (waitingLeft == 0 && leftInBuf.size() <= HALF_BUF_SIZE) + leftSource().request(waitingLeft = IN_BUFFER_SIZE - leftInBuf.size()); + + if (waitingRight == 0 && requested > 0) + rightSource().request(waitingRight = IN_BUFFER_SIZE); + } + /** */ protected Node leftSource() { return sources().get(0); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index dc6603086263b..60f91d6ddabaa 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -24,6 +24,7 @@ import java.util.function.BiPredicate; import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; import org.apache.ignite.internal.processors.query.calcite.exec.RuntimeHashIndex; @@ -81,7 +82,7 @@ protected HashJoinNode( super(ctx, rowType); // For IS NOT DISTINCT we have to keep rows with null values. - if (!info.allowNulls().isEmpty()) + if (!keepRowsWithNull && info.allowNulls().cardinality() > 0) keepRowsWithNull = true; leftKeys = info.leftKeys.toIntArray(); @@ -93,7 +94,7 @@ protected HashJoinNode( this.nonEqCond = nonEqCond; - rightHashStore = new RuntimeHashIndex<>(ctx, info.rightKeys, keepRowsWithNull, !info.allowNulls().isEmpty(), + rightHashStore = new RuntimeHashIndex<>(ctx, rightKeys, keepRowsWithNull ? info.allowNulls() : ImmutableBitSet.of(), INITIAL_CAPACITY, TouchedArrayList::new); remappedLeftSearcher = rightHashStore.remappedSearcher(leftKeys); @@ -199,11 +200,7 @@ protected void requestMoreOrEnd() throws Exception { return; } - if (waitingRight == 0 && requested > 0) - rightSource().request(waitingRight = IN_BUFFER_SIZE); - - if (waitingLeft == 0 && leftInBuf.size() <= HALF_BUF_SIZE) - leftSource().request(waitingLeft = IN_BUFFER_SIZE - leftInBuf.size()); + tryToRequestInputs(); } /** */ @@ -234,7 +231,7 @@ private InnerHashJoin(ExecutionContext ctx, try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { // Proceed with next left row, if previous was fully processed. - if (!rightIt.hasNext()) { + if (left == null) { left = leftInBuf.remove(); rightIt = lookup(left).iterator(); @@ -673,6 +670,7 @@ private static final class AntiHashJoin extends HashJoinNode { * @param rowType Out row type. * @param info Join info. * @param outRowHnd Output row handler. + * @param nonEqCond Non-equi conditions. */ private AntiHashJoin( ExecutionContext ctx, @@ -681,7 +679,7 @@ private AntiHashJoin( RowHandler outRowHnd, @Nullable BiPredicate nonEqCond ) { - super(ctx, rowType, info, outRowHnd, false, null); + super(ctx, rowType, info, outRowHnd, false, nonEqCond); } /** {@inheritDoc} */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java index 998792f0f36c1..73998d1b72d11 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/IndexSpoolNode.java @@ -191,7 +191,9 @@ public static IndexSpoolNode createHashSpool( Supplier searchRow, boolean allowNulls ) { - RuntimeHashIndex idx = new RuntimeHashIndex<>(ctx, keys, allowNulls); + ImmutableBitSet allowNulls0 = allowNulls ? ImmutableBitSet.range(keys.cardinality()) : ImmutableBitSet.of(); + + RuntimeHashIndex idx = new RuntimeHashIndex<>(ctx, keys, allowNulls0); ScanNode scan = new ScanNode<>( ctx, diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index f8560a1a9afc8..a660f8d2840c0 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -629,13 +629,4 @@ protected boolean checkJoinFinished() throws Exception { return false; } - - /** */ - protected void tryToRequestInputs() throws Exception { - if (waitingLeft == 0 && leftInBuf.size() <= HALF_BUF_SIZE) - leftSource().request(waitingLeft = IN_BUFFER_SIZE - leftInBuf.size()); - - if (waitingRight == 0 && requested > 0) - rightSource().request(waitingRight = IN_BUFFER_SIZE); - } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java index b609cc795d478..0b0c58c09122f 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteHashJoin.java @@ -38,6 +38,9 @@ /** */ public class IgniteHashJoin extends AbstractIgniteJoin { + /** */ + private static final double DISTINCT_RIGHT_ROWS_RATIO = 0.9; + /** */ public IgniteHashJoin( RelOptCluster cluster, @@ -81,7 +84,8 @@ public IgniteHashJoin(RelInput input) { double rightSize = rightRowCnt * IgniteCost.AVERAGE_FIELD_SIZE * getRight().getRowType().getFieldCount(); - double distRightRows = Util.first(mq.getDistinctRowCount(right, ImmutableBitSet.of(joinInfo.rightKeys), null), 0.9 * rightRowCnt); + double distRightRows = Util.first(mq.getDistinctRowCount(right, ImmutableBitSet.of(joinInfo.rightKeys), null), + DISTINCT_RIGHT_ROWS_RATIO * rightRowCnt); rightSize += distRightRows * rightKeysSize * IgniteCost.AVERAGE_FIELD_SIZE; diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java index fd653758effa3..bd6f6eb2258c3 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java @@ -23,7 +23,6 @@ import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateSingleGroupExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashIndexSpoolExecutionTest; -import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashJoinExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.JoinBuffersExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.LimitExecutionTest; @@ -47,7 +46,6 @@ ContinuousExecutionTest.class, MergeJoinExecutionTest.class, NestedLoopJoinExecutionTest.class, - HashJoinExecutionTest.class, JoinBuffersExecutionTest.class, TableSpoolExecutionTest.class, SortedIndexSpoolExecutionTest.class, From 630a213a2d612f396fb5f7c23133cab4facdb803 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 17 Jan 2026 18:00:25 +0300 Subject: [PATCH 22/28] refactor to checkJoinFinished() / tryToRequestInputs(); --- .../query/calcite/exec/rel/HashJoinNode.java | 49 ++++++++++++++----- 1 file changed, 38 insertions(+), 11 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 60f91d6ddabaa..e96980033413e 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -188,19 +188,28 @@ protected Iterator untouched() { } /** */ - protected void requestMoreOrEnd() throws Exception { - if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && leftInBuf.isEmpty() && left == null - && !rightIt.hasNext()) { + protected boolean leftFinished() { + return waitingLeft == NOT_WAITING && left == null && leftInBuf.isEmpty(); + } + + /** */ + protected boolean rightFinished() { + return waitingRight == NOT_WAITING && !rightIt.hasNext(); + } + + /** */ + protected boolean checkJoinFinished() throws Exception { + if (requested > 0 && leftFinished() && rightFinished()) { requested = 0; rightHashStore.close(); downstream().end(); - return; + return true; } - tryToRequestInputs(); + return false; } /** */ @@ -264,7 +273,10 @@ private InnerHashJoin(ExecutionContext ctx, } } - requestMoreOrEnd(); + if (checkJoinFinished()) + return; + + tryToRequestInputs(); } } @@ -344,7 +356,10 @@ private LeftHashJoin( } } - requestMoreOrEnd(); + if (checkJoinFinished()) + return; + + tryToRequestInputs(); } } @@ -450,7 +465,10 @@ private RightHashJoin( } } - requestMoreOrEnd(); + if (checkJoinFinished()) + return; + + tryToRequestInputs(); } /** {@inheritDoc} */ @@ -577,7 +595,10 @@ private FullOuterHashJoin( } } - requestMoreOrEnd(); + if (checkJoinFinished()) + return; + + tryToRequestInputs(); } /** {@inheritDoc} */ @@ -657,7 +678,10 @@ private SemiHashJoin( } } - requestMoreOrEnd(); + if (checkJoinFinished()) + return; + + tryToRequestInputs(); } } @@ -709,7 +733,10 @@ private AntiHashJoin( } } - requestMoreOrEnd(); + if (checkJoinFinished()) + return; + + tryToRequestInputs(); } } From e42f8caaa5b0a65559e01e27d921934d8d3dde88 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 17 Jan 2026 22:04:25 +0300 Subject: [PATCH 23/28] + rows number processing limitation --- .../query/calcite/exec/rel/HashJoinNode.java | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index e96980033413e..24b2775d32a3a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -236,6 +236,7 @@ private InnerHashJoin(ExecutionContext ctx, @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -251,6 +252,13 @@ private InnerHashJoin(ExecutionContext ctx, while (requested > 0 && rightIt.hasNext()) { checkState(); + if (processed++ > IN_BUFFER_SIZE) { + // Allow others to do their job. + context().execute(this::join, this::onError); + + return; + } + RowT right = rightIt.next(); if (nonEqCond != null && !nonEqCond.test(left, right)) @@ -312,6 +320,7 @@ private LeftHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -337,6 +346,13 @@ private LeftHashJoin( while (requested > 0 && rightIt.hasNext()) { checkState(); + if (processed++ > IN_BUFFER_SIZE) { + // Allow others to do their job. + context().execute(this::join, this::onError); + + return; + } + RowT right = rightIt.next(); --requested; @@ -398,6 +414,7 @@ private RightHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -415,6 +432,12 @@ private RightHashJoin( while (requested > 0 && rightIt.hasNext()) { checkState(); + if (processed++ > IN_BUFFER_SIZE) { + context().execute(this::join, this::onError); + + return; + } + RowT right = rightIt.next(); --requested; @@ -437,6 +460,7 @@ private RightHashJoin( // Emit unmatched right rows. if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; + int processed = 0; try { if (!rightIt.hasNext() && !drainMaterialization) { @@ -451,6 +475,12 @@ private RightHashJoin( while (requested > 0 && rightIt.hasNext()) { checkState(); + if (processed++ > IN_BUFFER_SIZE) { + context().execute(this::join, this::onError); + + return; + } + RowT right = rightIt.next(); RowT row = outRowHnd.concat(emptyLeft, right); @@ -521,6 +551,7 @@ private FullOuterHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -543,6 +574,12 @@ private FullOuterHashJoin( if (rightIt.hasNext()) { // Emits matched rows. while (requested > 0 && rightIt.hasNext()) { + if (processed++ > IN_BUFFER_SIZE) { + context().execute(this::join, this::onError); + + return; + } + checkState(); RowT right = rightIt.next(); @@ -567,6 +604,7 @@ private FullOuterHashJoin( // Emit unmatched right rows. if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; + int processed = 0; try { if (!rightIt.hasNext() && !drainMaterialization) { @@ -579,6 +617,12 @@ private FullOuterHashJoin( RowT emptyLeft = leftRowFactory.create(); while (requested > 0 && rightIt.hasNext()) { + if (processed++ > IN_BUFFER_SIZE) { + context().execute(this::join, this::onError); + + return; + } + checkState(); RowT right = rightIt.next(); @@ -634,6 +678,7 @@ private SemiHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -658,6 +703,12 @@ private SemiHashJoin( break; } + + if (processed++ > IN_BUFFER_SIZE) { + context().execute(this::join, this::onError); + + return; + } } } @@ -710,11 +761,18 @@ private AntiHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; + int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { checkState(); + if (processed++ > IN_BUFFER_SIZE) { + context().execute(this::join, this::onError); + + return; + } + left = leftInBuf.remove(); Collection rightRows = lookup(left); From f1da1c187664dc78a61d54acb49794eb9d7576bf Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Sat, 17 Jan 2026 22:16:23 +0300 Subject: [PATCH 24/28] refactor join reschedule --- .../AbstractRightMaterializedJoinNode.java | 16 ++++++ .../query/calcite/exec/rel/HashJoinNode.java | 50 +++---------------- 2 files changed, 24 insertions(+), 42 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java index 1b0d712deff39..4fd5723a5ed58 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -50,6 +50,9 @@ public abstract class AbstractRightMaterializedJoinNode extends MemoryTrack /** */ protected @Nullable Row left; + /** */ + protected int processed; + /** */ protected AbstractRightMaterializedJoinNode(ExecutionContext ctx, RelDataType rowType) { super(ctx, rowType); @@ -183,4 +186,17 @@ private void doJoin() throws Exception { join(); } + + /** */ + protected boolean rescheduleJoin() { + if (processed++ > IN_BUFFER_SIZE) { + processed = 0; + + context().execute(this::doJoin, this::onError); + + return true; + } + + return false; + } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java index 24b2775d32a3a..63c0a0f837bbe 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/HashJoinNode.java @@ -236,7 +236,6 @@ private InnerHashJoin(ExecutionContext ctx, @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; - int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -252,12 +251,8 @@ private InnerHashJoin(ExecutionContext ctx, while (requested > 0 && rightIt.hasNext()) { checkState(); - if (processed++ > IN_BUFFER_SIZE) { - // Allow others to do their job. - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } RowT right = rightIt.next(); @@ -320,7 +315,6 @@ private LeftHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; - int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -346,12 +340,8 @@ private LeftHashJoin( while (requested > 0 && rightIt.hasNext()) { checkState(); - if (processed++ > IN_BUFFER_SIZE) { - // Allow others to do their job. - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } RowT right = rightIt.next(); @@ -414,7 +404,6 @@ private RightHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; - int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -432,11 +421,8 @@ private RightHashJoin( while (requested > 0 && rightIt.hasNext()) { checkState(); - if (processed++ > IN_BUFFER_SIZE) { - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } RowT right = rightIt.next(); @@ -460,7 +446,6 @@ private RightHashJoin( // Emit unmatched right rows. if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; - int processed = 0; try { if (!rightIt.hasNext() && !drainMaterialization) { @@ -475,11 +460,8 @@ private RightHashJoin( while (requested > 0 && rightIt.hasNext()) { checkState(); - if (processed++ > IN_BUFFER_SIZE) { - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } RowT right = rightIt.next(); @@ -551,7 +533,6 @@ private FullOuterHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; - int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -574,11 +555,8 @@ private FullOuterHashJoin( if (rightIt.hasNext()) { // Emits matched rows. while (requested > 0 && rightIt.hasNext()) { - if (processed++ > IN_BUFFER_SIZE) { - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } checkState(); @@ -604,7 +582,6 @@ private FullOuterHashJoin( // Emit unmatched right rows. if (left == null && leftInBuf.isEmpty() && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && requested > 0) { inLoop = true; - int processed = 0; try { if (!rightIt.hasNext() && !drainMaterialization) { @@ -617,11 +594,8 @@ private FullOuterHashJoin( RowT emptyLeft = leftRowFactory.create(); while (requested > 0 && rightIt.hasNext()) { - if (processed++ > IN_BUFFER_SIZE) { - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } checkState(); @@ -678,7 +652,6 @@ private SemiHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; - int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { @@ -704,11 +677,8 @@ private SemiHashJoin( break; } - if (processed++ > IN_BUFFER_SIZE) { - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } } } @@ -761,17 +731,13 @@ private AntiHashJoin( @Override protected void join() throws Exception { if (waitingRight == NOT_WAITING) { inLoop = true; - int processed = 0; try { while (requested > 0 && (left != null || !leftInBuf.isEmpty())) { checkState(); - if (processed++ > IN_BUFFER_SIZE) { - context().execute(this::join, this::onError); - + if (rescheduleJoin()) return; - } left = leftInBuf.remove(); From d8ebf60221d03c23ff0846ea75a70f2a0803c00a Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 19 Jan 2026 22:04:21 +0300 Subject: [PATCH 25/28] review fixes --- .../calcite/exec/rel/NestedLoopJoinNode.java | 24 +++---------------- .../ignite/testsuites/ExecutionTestSuite.java | 2 ++ 2 files changed, 5 insertions(+), 21 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index a660f8d2840c0..e68da5e22f347 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -37,6 +37,9 @@ public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJ /** */ protected final List rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE); + /** */ + protected int rightIdx; + /** * @param ctx Execution context. * @param rowType Row type. @@ -118,9 +121,6 @@ public static NestedLoopJoinNode create( /** */ private static class InnerJoin extends NestedLoopJoinNode { - /** */ - private int rightIdx; - /** */ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); @@ -179,9 +179,6 @@ private static class LeftJoin extends NestedLoopJoinNode { /** Shows whether current left row was matched. */ private boolean matched; - /** */ - private int rightIdx; - /** */ public LeftJoin( ExecutionContext ctx, @@ -267,9 +264,6 @@ private static class RightJoin extends NestedLoopJoinNode { /** */ private int lastPushedInd; - /** */ - private int rightIdx; - /** */ public RightJoin( ExecutionContext ctx, @@ -385,12 +379,6 @@ private static class FullOuterJoin extends NestedLoopJoinNode { /** */ private int lastPushedInd; - /** */ - private Row left; - - /** */ - private int rightIdx; - /** */ public FullOuterJoin( ExecutionContext ctx, @@ -510,9 +498,6 @@ public FullOuterJoin( /** */ private static class SemiJoin extends NestedLoopJoinNode { - /** */ - private int rightIdx; - /** */ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); @@ -562,9 +547,6 @@ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate extends NestedLoopJoinNode { - /** */ - private int rightIdx; - /** */ public AntiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java index bd6f6eb2258c3..7b0cb00b361b9 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateSingleGroupExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashIndexSpoolExecutionTest; +import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashJoinExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.JoinBuffersExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.LimitExecutionTest; @@ -45,6 +46,7 @@ ExecutionTest.class, ContinuousExecutionTest.class, MergeJoinExecutionTest.class, + HashJoinExecutionTest.class, NestedLoopJoinExecutionTest.class, JoinBuffersExecutionTest.class, TableSpoolExecutionTest.class, From ea11f2a05481ddb87865a833da04401ba51b45ef Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 19 Jan 2026 22:04:21 +0300 Subject: [PATCH 26/28] review fixes --- .../AbstractRightMaterializedJoinNode.java | 16 ++++++------- .../calcite/exec/rel/NestedLoopJoinNode.java | 24 +++---------------- .../ignite/testsuites/ExecutionTestSuite.java | 2 ++ 3 files changed, 13 insertions(+), 29 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java index 4fd5723a5ed58..10c406b3f457b 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -74,7 +74,7 @@ protected AbstractRightMaterializedJoinNode(ExecutionContext ctx, RelDataTy requested = rowsCnt; if (!inLoop) - context().execute(this::doJoin, this::onError); + context().execute(this::join0, this::onError); } /** {@inheritDoc} */ @@ -134,7 +134,7 @@ private void pushLeft(Row row) throws Exception { leftInBuf.add(row); - join(); + join0(); } /** */ @@ -146,7 +146,7 @@ private void endLeft() throws Exception { waitingLeft = NOT_WAITING; - join(); + join0(); } /** */ @@ -158,7 +158,7 @@ private void endRight() throws Exception { waitingRight = NOT_WAITING; - join(); + join0(); } /** */ @@ -181,18 +181,18 @@ protected Node rightSource() { } /** */ - private void doJoin() throws Exception { + private void join0() throws Exception { checkState(); + processed = 0; + join(); } /** */ protected boolean rescheduleJoin() { if (processed++ > IN_BUFFER_SIZE) { - processed = 0; - - context().execute(this::doJoin, this::onError); + context().execute(this::join0, this::onError); return true; } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index a660f8d2840c0..e68da5e22f347 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -37,6 +37,9 @@ public abstract class NestedLoopJoinNode extends AbstractRightMaterializedJ /** */ protected final List rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE); + /** */ + protected int rightIdx; + /** * @param ctx Execution context. * @param rowType Row type. @@ -118,9 +121,6 @@ public static NestedLoopJoinNode create( /** */ private static class InnerJoin extends NestedLoopJoinNode { - /** */ - private int rightIdx; - /** */ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); @@ -179,9 +179,6 @@ private static class LeftJoin extends NestedLoopJoinNode { /** Shows whether current left row was matched. */ private boolean matched; - /** */ - private int rightIdx; - /** */ public LeftJoin( ExecutionContext ctx, @@ -267,9 +264,6 @@ private static class RightJoin extends NestedLoopJoinNode { /** */ private int lastPushedInd; - /** */ - private int rightIdx; - /** */ public RightJoin( ExecutionContext ctx, @@ -385,12 +379,6 @@ private static class FullOuterJoin extends NestedLoopJoinNode { /** */ private int lastPushedInd; - /** */ - private Row left; - - /** */ - private int rightIdx; - /** */ public FullOuterJoin( ExecutionContext ctx, @@ -510,9 +498,6 @@ public FullOuterJoin( /** */ private static class SemiJoin extends NestedLoopJoinNode { - /** */ - private int rightIdx; - /** */ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); @@ -562,9 +547,6 @@ public SemiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate extends NestedLoopJoinNode { - /** */ - private int rightIdx; - /** */ public AntiJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate cond) { super(ctx, rowType, cond); diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java index bd6f6eb2258c3..7b0cb00b361b9 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/ExecutionTestSuite.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateSingleGroupExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashIndexSpoolExecutionTest; +import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashJoinExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.JoinBuffersExecutionTest; import org.apache.ignite.internal.processors.query.calcite.exec.rel.LimitExecutionTest; @@ -45,6 +46,7 @@ ExecutionTest.class, ContinuousExecutionTest.class, MergeJoinExecutionTest.class, + HashJoinExecutionTest.class, NestedLoopJoinExecutionTest.class, JoinBuffersExecutionTest.class, TableSpoolExecutionTest.class, From 15e1e85e7c673c65e84dcd4df15ee93ca84ec030 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 19 Jan 2026 22:35:23 +0300 Subject: [PATCH 27/28] review fixes --- .../exec/rel/AbstractRightMaterializedJoinNode.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java index 10c406b3f457b..7dce770bcaeed 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractRightMaterializedJoinNode.java @@ -69,8 +69,6 @@ protected AbstractRightMaterializedJoinNode(ExecutionContext ctx, RelDataTy assert !F.isEmpty(sources()) && sources().size() == 2; assert rowsCnt > 0 && requested == 0; - checkState(); - requested = rowsCnt; if (!inLoop) @@ -128,8 +126,6 @@ private void pushLeft(Row row) throws Exception { assert downstream() != null; assert waitingLeft > 0; - checkState(); - --waitingLeft; leftInBuf.add(row); @@ -142,8 +138,6 @@ private void endLeft() throws Exception { assert downstream() != null; assert waitingLeft > 0; - checkState(); - waitingLeft = NOT_WAITING; join0(); @@ -154,8 +148,6 @@ private void endRight() throws Exception { assert downstream() != null; assert waitingRight > 0; - checkState(); - waitingRight = NOT_WAITING; join0(); From dabf87138a13d855caa43b915a581aeb1efa5b2c Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Mon, 19 Jan 2026 22:44:37 +0300 Subject: [PATCH 28/28] review fixes --- .../calcite/exec/rel/NestedLoopJoinNode.java | 26 ++----------------- 1 file changed, 2 insertions(+), 24 deletions(-) diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java index e68da5e22f347..f2f42e3b68dab 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java @@ -56,6 +56,8 @@ private NestedLoopJoinNode(ExecutionContext ctx, RelDataType rowType, BiPre @Override protected void rewindInternal() { super.rewindInternal(); + rightIdx = 0; + rightMaterialized.clear(); } @@ -126,13 +128,6 @@ public InnerJoin(ExecutionContext ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate ctx, RelDataType rowType, BiPredicate