You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by gv...@apache.org on 2020/09/03 12:28:03 UTC

[ignite] branch ignite-12248 updated: IGNITE-12620: Calcite integration. Index Nested Loop Join/Hash Join. This closes #8210

This is an automated email from the ASF dual-hosted git repository.

gvvinblade pushed a commit to branch ignite-12248
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-12248 by this push:
     new 2075b9e  IGNITE-12620: Calcite integration. Index Nested Loop Join/Hash Join. This closes #8210
2075b9e is described below

commit 2075b9e2231827ac844a627acf3324c336bd2c07
Author: Igor Seliverstov <gv...@gmail.com>
AuthorDate: Thu Sep 3 15:26:58 2020 +0300

    IGNITE-12620: Calcite integration. Index Nested Loop Join/Hash Join. This closes #8210
---
 modules/calcite/pom.xml                            |   7 +
 .../plan/volcano/VolcanoUtils.java}                |  14 +-
 .../query/calcite/CalciteQueryProcessor.java       |  51 +-
 .../query/calcite/exec/ArrayRowHandler.java        |   7 +
 ...eNode.java => ExecutionCancelledException.java} |  14 +-
 .../query/calcite/exec/ExecutionContext.java       |  35 +-
 .../query/calcite/exec/ExecutionServiceImpl.java   |   9 +-
 .../processors/query/calcite/exec/IndexScan.java   |  15 +-
 .../query/calcite/exec/LogicalRelImplementor.java  | 100 +--
 .../processors/query/calcite/exec/RowHandler.java  |   3 +
 .../query/calcite/exec/exp/ExpressionFactory.java  |   2 +-
 .../calcite/exec/exp/ExpressionFactoryImpl.java    | 157 ++--
 .../calcite/exec/exp/agg/AccumulatorsFactory.java  |  12 +-
 .../query/calcite/exec/rel/AbstractJoinNode.java   | 216 ------
 .../query/calcite/exec/rel/AbstractNode.java       |  95 ++-
 .../query/calcite/exec/rel/AggregateNode.java      | 125 +--
 .../query/calcite/exec/rel/AntiJoinNode.java       |  77 --
 .../exec/rel/CorrelatedNestedLoopJoinNode.java     | 464 ++++++++++++
 .../query/calcite/exec/rel/FilterNode.java         | 109 +--
 .../query/calcite/exec/rel/FullOuterJoinNode.java  | 141 ----
 .../processors/query/calcite/exec/rel/Inbox.java   | 213 +++---
 .../query/calcite/exec/rel/InnerJoinNode.java      |  75 --
 .../query/calcite/exec/rel/LeftJoinNode.java       |  99 ---
 .../query/calcite/exec/rel/ModifyNode.java         | 133 ++--
 .../query/calcite/exec/rel/NestedLoopJoinNode.java | 842 +++++++++++++++++++++
 .../processors/query/calcite/exec/rel/Node.java    |  18 +
 .../processors/query/calcite/exec/rel/Outbox.java  | 185 ++---
 .../query/calcite/exec/rel/ProjectNode.java        |  57 +-
 .../query/calcite/exec/rel/RightJoinNode.java      | 118 ---
 .../query/calcite/exec/rel/RootNode.java           | 115 +--
 .../query/calcite/exec/rel/ScanNode.java           |  88 +--
 .../query/calcite/exec/rel/SemiJoinNode.java       |  79 --
 .../query/calcite/exec/rel/SingleNode.java         |   8 +-
 .../query/calcite/exec/rel/SortNode.java           |  95 +--
 .../query/calcite/exec/rel/UnionAllNode.java       |  64 +-
 .../calcite/metadata/IgniteMdCumulativeCost.java   |  81 ++
 .../calcite/metadata/IgniteMdDistribution.java     |   1 +
 .../calcite/metadata/IgniteMdNodesMapping.java     |   5 +-
 .../metadata/IgniteMdNonCumulativeCost.java        |  46 ++
 .../metadata/IgniteMdPercentageOriginalRows.java   | 161 ++++
 .../query/calcite/metadata/IgniteMdPredicates.java |  59 ++
 .../query/calcite/metadata/IgniteMdRowCount.java   | 107 +++
 .../query/calcite/metadata/IgniteMetadata.java     |  49 +-
 .../query/calcite/metadata/RelMetadataQueryEx.java |  44 +-
 .../processors/query/calcite/prepare/Cloner.java   |  21 +-
 .../query/calcite/prepare/FragmentDescription.java |  15 +-
 .../query/calcite/prepare/FragmentSplitter.java    |  14 +-
 .../query/calcite/prepare/IgnitePlanner.java       |   7 +-
 .../query/calcite/prepare/PlannerPhase.java        |  12 +-
 .../query/calcite/prepare/PlanningContext.java     |  45 +-
 .../processors/query/calcite/prepare/Splitter.java |  14 +-
 .../calcite/rel/AbstractIgniteNestedLoopJoin.java  | 414 ++++++++++
 .../query/calcite/rel/IgniteAggregate.java         | 275 ++++---
 .../rel/IgniteCorrelatedNestedLoopJoin.java        | 162 ++++
 .../query/calcite/rel/IgniteExchange.java          |  17 +-
 .../processors/query/calcite/rel/IgniteFilter.java |  11 +
 .../{IgniteTableScan.java => IgniteIndexScan.java} |  86 ++-
 .../processors/query/calcite/rel/IgniteJoin.java   | 365 ---------
 .../query/calcite/rel/IgniteNestedLoopJoin.java    |  87 +++
 .../query/calcite/rel/IgniteProject.java           | 169 +++--
 .../processors/query/calcite/rel/IgniteRel.java    |   8 +
 .../query/calcite/rel/IgniteRelVisitor.java        |   9 +-
 .../query/calcite/rel/IgniteTrimExchange.java      |   9 +
 .../query/calcite/rel/IgniteUnionAll.java          |  88 ++-
 .../calcite/rule/AbstractIgniteConverterRule.java  |   7 +-
 .../CorrelatedNestedLoopJoinConverterRule.java     | 130 ++++
 ...RegisterIndexRule.java => ExposeIndexRule.java} |  59 +-
 ...rRule.java => NestedLoopJoinConverterRule.java} |  12 +-
 .../query/calcite/rule/PushFilterIntoScanRule.java |  60 +-
 .../query/calcite/rule/ValuesConverterRule.java    |   5 +-
 .../logical/LogicalFilterProjectTransposeRule.java |   2 +-
 .../calcite/rule/logical/LogicalOrToUnionRule.java |  27 +-
 .../query/calcite/schema/IgniteIndex.java          |   5 +-
 .../query/calcite/schema/IgniteTable.java          |   4 +-
 .../query/calcite/schema/IgniteTableImpl.java      |  10 +-
 .../query/calcite/schema/TableDescriptorImpl.java  |  17 +-
 .../processors/query/calcite/trait/AllNodes.java   |   4 +-
 .../query/calcite/trait/Destination.java           |   4 +-
 .../query/calcite/trait/DistributionFunction.java  |  64 +-
 .../query/calcite/trait/Partitioned.java           |  25 +-
 .../processors/query/calcite/trait/RandomNode.java |   4 +-
 .../trait/{Destination.java => RelFactory.java}    |  22 +-
 .../query/calcite/trait/RewindabilityTrait.java    | 102 +++
 .../query/calcite/trait/RewindabilityTraitDef.java |  52 ++
 .../processors/query/calcite/trait/TraitUtils.java | 100 ++-
 .../query/calcite/trait/TraitsAwareIgniteRel.java  | 153 ++++
 .../calcite/trait/TraitsPropagationContext.java    | 100 +++
 .../{Destination.java => TraitsPropagator.java}    |  22 +-
 .../processors/query/calcite/util/Commons.java     |  46 +-
 .../calcite/util/ConvertingClosableIterator.java   |   3 +-
 .../query/calcite/util/IgniteMethod.java           |   7 +
 .../processors/query/calcite/util/RexUtils.java    |  84 ++
 .../processors/query/calcite/util/TypeUtils.java   | 121 +++
 .../CalciteBasicSecondaryIndexIntegrationTest.java | 131 ++--
 .../processors/query/calcite/PlannerTest.java      |  96 ++-
 .../processors/query/calcite/QueryChecker.java     |  24 +-
 .../query/calcite/exec/rel/ExecutionTest.java      |  96 +--
 .../query/calcite/rules/OrToUnionRuleTest.java     |  33 +-
 .../org/apache/ignite/internal/IgniteKernal.java   |   5 +-
 99 files changed, 5216 insertions(+), 2643 deletions(-)

diff --git a/modules/calcite/pom.xml b/modules/calcite/pom.xml
index cf8a8bb..469a1d8 100644
--- a/modules/calcite/pom.xml
+++ b/modules/calcite/pom.xml
@@ -30,6 +30,7 @@
         <janino.version>3.0.11</janino.version>
         <avatica.version>1.16.0</avatica.version>
         <jsonpath.version>2.4.0</jsonpath.version>
+        <reflections.version>0.9.10</reflections.version>
     </properties>
 
     <parent>
@@ -127,6 +128,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>${reflections.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-core</artifactId>
             <version>${project.version}</version>
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java b/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
similarity index 74%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
copy to modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
index 3b96914..185d453 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
+++ b/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
@@ -15,16 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+package org.apache.calcite.plan.volcano;
 
-import org.apache.ignite.internal.util.typedef.F;
+import org.apache.calcite.plan.RelOptCost;
 
-/**
- * A node with a single input
- */
-public interface SingleNode<Row> extends Node<Row> {
+/** */
+public class VolcanoUtils {
     /** */
-    default void register(Node<Row> src) {
-        register(F.asList(src));
+    public static RelOptCost bestCost(RelSubset relSubset) {
+        return relSubset.bestCost;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
index 5ec299e..f59b50f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
@@ -61,35 +61,38 @@ import org.apache.ignite.internal.processors.query.calcite.util.LifecycleAware;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.calcite.rex.RexUtil.EXECUTOR;
+
 /** */
 public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
     /** */
     public static final FrameworkConfig FRAMEWORK_CONFIG = Frameworks.newConfigBuilder()
-            .sqlToRelConverterConfig(SqlToRelConverter.configBuilder()
-                .withTrimUnusedFields(true)
-                .withDecorrelationEnabled(true)
-                .build())
-            .parserConfig(SqlParser.configBuilder()
-                // Lexical configuration defines how identifiers are quoted, whether they are converted to upper or lower
-                // case when they are read, and whether identifiers are matched case-sensitively.
-                .setLex(Lex.ORACLE)
+        .executor(EXECUTOR)
+        .sqlToRelConverterConfig(SqlToRelConverter.configBuilder()
+            .withTrimUnusedFields(true)
+            .withDecorrelationEnabled(true)
+            .build())
+        .parserConfig(SqlParser.configBuilder()
+            // Lexical configuration defines how identifiers are quoted, whether they are converted to upper or lower
+            // case when they are read, and whether identifiers are matched case-sensitively.
+            .setLex(Lex.ORACLE)
 //                .setParserFactory(SqlDdlParserImpl.FACTORY) // Enables DDL support
-                .setConformance(SqlConformanceEnum.DEFAULT)
-                .build())
-            .sqlValidatorConfig(SqlValidator.Config.DEFAULT
-                .withIdentifierExpansion(true)
-                .withSqlConformance(SqlConformanceEnum.DEFAULT))
-            // Dialects support.
-            .operatorTable(SqlLibraryOperatorTableFactory.INSTANCE
-                .getOperatorTable(
-                    SqlLibrary.STANDARD,
-                    SqlLibrary.MYSQL))
-            // Context provides a way to store data within the planner session that can be accessed in planner rules.
-            .context(Contexts.empty())
-            // Custom cost factory to use during optimization
-            .costFactory(RelOptCostImpl.FACTORY)
-            .typeSystem(IgniteTypeSystem.INSTANCE)
-            .build();
+            .setConformance(SqlConformanceEnum.DEFAULT)
+            .build())
+        .sqlValidatorConfig(SqlValidator.Config.DEFAULT
+            .withIdentifierExpansion(true)
+            .withSqlConformance(SqlConformanceEnum.DEFAULT))
+        // Dialects support.
+        .operatorTable(SqlLibraryOperatorTableFactory.INSTANCE
+            .getOperatorTable(
+                SqlLibrary.STANDARD,
+                SqlLibrary.MYSQL))
+        // Context provides a way to store data within the planner session that can be accessed in planner rules.
+        .context(Contexts.empty())
+        // Custom cost factory to use during optimization
+        .costFactory(RelOptCostImpl.FACTORY)
+        .typeSystem(IgniteTypeSystem.INSTANCE)
+        .build();
 
     /** */
     private final QueryPlanCache qryPlanCache;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
index 1f53ad2..be2eeb6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
@@ -56,10 +56,17 @@ public class ArrayRowHandler implements RowHandler<Object[]> {
         int rowLen = types.length;
 
         return new RowFactory<Object[]>() {
+            /** {@inheritDoc} */
+            @Override public RowHandler<Object[]> handler() {
+                return ArrayRowHandler.this;
+            }
+
+            /** {@inheritDoc} */
             @Override public Object[] create() {
                 return new Object[rowLen];
             }
 
+            /** {@inheritDoc} */
             @Override public Object[] create(Object... fields) {
                 assert fields.length == rowLen;
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
similarity index 78%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
index 3b96914..fb4a8d8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
@@ -15,16 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
-import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.IgniteCheckedException;
 
-/**
- * A node with a single input
- */
-public interface SingleNode<Row> extends Node<Row> {
-    /** */
-    default void register(Node<Row> src) {
-        register(F.asList(src));
-    }
+/** */
+public class ExecutionCancelledException extends IgniteCheckedException {
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
index 4e7cdd5..c9a355b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
@@ -31,6 +31,10 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping
 import org.apache.ignite.internal.processors.query.calcite.prepare.FragmentDescription;
 import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.checkRange;
 
 /**
  * Runtime context allowing access to the tables in a database.
@@ -57,6 +61,9 @@ public class ExecutionContext<Row> implements DataContext {
     /** */
     private final ExpressionFactory<Row> expressionFactory;
 
+    /** */
+    private Object[] correlations = new Object[16];
+
     /**
      * @param ctx Parent context.
      * @param qryId Query ID.
@@ -106,8 +113,8 @@ public class ExecutionContext<Row> implements DataContext {
     /**
      * @return Interested partitions.
      */
-    public int[] partitions() {
-        return fragmentDesc.partitions();
+    public int[] localPartitions() {
+        return fragmentDesc.localPartitions();
     }
 
     /** */
@@ -188,6 +195,30 @@ public class ExecutionContext<Row> implements DataContext {
     }
 
     /**
+     * Gets correlated value.
+     *
+     * @param id Correlation ID.
+     * @return Correlated value.
+     */
+    public @NotNull Object getCorrelated(int id) {
+        checkRange(correlations, id);
+
+        return correlations[id];
+    }
+
+    /**
+     * Sets correlated value.
+     *
+     * @param id Correlation ID.
+     * @param value Correlated value.
+     */
+    public void setCorrelated(@NotNull Object value, int id) {
+        correlations = Commons.ensureCapacity(correlations, id + 1);
+
+        correlations[id] = value;
+    }
+
+    /**
      * Executes a query task.
      *
      * @param task Query task.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index aea46c7..99b0ea5 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -99,6 +99,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTraitDef;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
@@ -463,7 +464,8 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
         RelTraitDef<?>[] traitDefs = {
             ConventionTraitDef.INSTANCE,
             RelCollationTraitDef.INSTANCE,
-            DistributionTraitDef.INSTANCE
+            DistributionTraitDef.INSTANCE,
+            RewindabilityTraitDef.INSTANCE
         };
 
         return PlanningContext.builder()
@@ -493,7 +495,8 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
         RelTraitDef<?>[] traitDefs = {
             ConventionTraitDef.INSTANCE,
             RelCollationTraitDef.INSTANCE,
-            DistributionTraitDef.INSTANCE
+            DistributionTraitDef.INSTANCE,
+            RewindabilityTraitDef.INSTANCE
         };
 
         return PlanningContext.builder()
@@ -1025,7 +1028,7 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
                 running.remove(ctx.queryId());
 
                 // 2) close local fragment
-                root.proceedClose();
+                root.onClose();
 
                 // 3) close remote fragments
                 for (UUID nodeId : remotes) {
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java
index e15a8b6..0a20171 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java
@@ -22,6 +22,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.function.Predicate;
+import java.util.function.Supplier;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.ignite.IgniteCheckedException;
@@ -86,10 +87,10 @@ public class IndexScan<Row> implements Iterable<Row>, AutoCloseable {
     private final Predicate<Row> filters;
 
     /** Lower index scan bound. */
-    private final Row lowerBound;
+    private final Supplier<Row> lowerBound;
 
     /** Upper index scan bound. */
-    private final Row upperBound;
+    private final Supplier<Row> upperBound;
 
     /** */
     private final int[] partsArr;
@@ -111,8 +112,8 @@ public class IndexScan<Row> implements Iterable<Row>, AutoCloseable {
         ExecutionContext<Row> ectx,
         IgniteIndex igniteIdx,
         Predicate<Row> filters,
-        Row lowerBound,
-        Row upperBound
+        Supplier<Row> lowerBound,
+        Supplier<Row> upperBound
     ) {
         this.ectx = ectx;
         desc = igniteIdx.table().descriptor();
@@ -128,7 +129,7 @@ public class IndexScan<Row> implements Iterable<Row>, AutoCloseable {
         this.filters = filters;
         this.lowerBound = lowerBound;
         this.upperBound = upperBound;
-        partsArr = ectx.partitions();
+        partsArr = ectx.localPartitions();
         mvccSnapshot = ectx.mvccSnapshot();
     }
 
@@ -136,8 +137,8 @@ public class IndexScan<Row> implements Iterable<Row>, AutoCloseable {
     @Override public synchronized Iterator<Row> iterator() {
         reserve();
         try {
-            H2Row lower = lowerBound == null ? null : new H2PlainRow(values(coCtx, ectx, lowerBound));
-            H2Row upper = upperBound == null ? null : new H2PlainRow(values(coCtx, ectx, upperBound));
+            H2Row lower = lowerBound == null ? null : new H2PlainRow(values(coCtx, ectx, lowerBound.get()));
+            H2Row upper = upperBound == null ? null : new H2PlainRow(values(coCtx, ectx, upperBound.get()));
 
             return new IteratorImpl(idx.find(lower, upper, filterClosure()));
         }
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 8b9ae84..77465e0 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
@@ -26,6 +26,7 @@ import java.util.function.ToIntFunction;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
@@ -35,29 +36,26 @@ import org.apache.ignite.internal.processors.failure.FailureProcessor;
 import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
 import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
 import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractJoinNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.AggregateNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.AntiJoinNode;
+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.FullOuterJoinNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.InnerJoinNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.LeftJoinNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.ModifyNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.NestedLoopJoinNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.Node;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.ProjectNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.RightJoinNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.ScanNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.rel.SemiJoinNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.SortNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.UnionAllNode;
 import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
 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.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMapAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReduceAggregate;
@@ -66,7 +64,6 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTrimExchange;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteUnionAll;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
@@ -79,6 +76,8 @@ import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribut
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.util.typedef.F;
 
+import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.combinedRowType;
+
 /**
  * Implements a query plan.
  */
@@ -125,7 +124,7 @@ public class LogicalRelImplementor<Row> implements IgniteRelVisitor<Node<Row>> {
     @Override public Node<Row> visit(IgniteSender rel) {
         IgniteDistribution distribution = rel.distribution();
 
-        Destination dest = distribution.function().destination(partSvc, ctx.targetMapping(), distribution.getKeys());
+        Destination<Row> dest = distribution.function().destination(ctx, partSvc, ctx.targetMapping(), distribution.getKeys());
 
         // Outbox fragment ID is used as exchange ID as well.
         Outbox<Row> outbox =
@@ -178,54 +177,33 @@ public class LogicalRelImplementor<Row> implements IgniteRelVisitor<Node<Row>> {
     }
 
     /** {@inheritDoc} */
-    @Override public Node<Row> visit(IgniteJoin rel) {
-        RelDataType rowType = Commons.combinedRowType(ctx.getTypeFactory(), rel.getLeft().getRowType(), rel.getRight().getRowType());
+    @Override public Node<Row> visit(IgniteNestedLoopJoin rel) {
+        RelDataType leftType = rel.getLeft().getRowType();
+        RelDataType rightType = rel.getRight().getRowType();
+        JoinRelType joinType = rel.getJoinType();
 
+        RelDataType rowType = combinedRowType(ctx.getTypeFactory(), leftType, rightType);
         Predicate<Row> cond = expressionFactory.predicate(rel.getCondition(), rowType);
 
-        AbstractJoinNode<Row> node;
-        switch (rel.getJoinType()) {
-            case INNER:
-                node = new InnerJoinNode<>(ctx, cond);
-
-                break;
-
-            case LEFT: {
-                RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rel.getRight().getRowType());
-                node = new LeftJoinNode<>(ctx, cond, rowFactory);
-
-                break;
-            }
+        Node<Row> node = NestedLoopJoinNode.create(ctx, leftType, rightType, joinType, cond);
 
-            case RIGHT: {
-                RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rel.getLeft().getRowType());
-                node = new RightJoinNode<>(ctx, cond, rowFactory);
-
-                break;
-            }
-
-            case FULL: {
-                RowFactory<Row> leftRowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rel.getLeft().getRowType());
-                RowFactory<Row> rightRowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rel.getRight().getRowType());
-
-                node = new FullOuterJoinNode<>(ctx, cond, leftRowFactory, rightRowFactory);
+        Node<Row> leftInput = visit(rel.getLeft());
+        Node<Row> rightInput = visit(rel.getRight());
 
-                break;
-            }
+        node.register(F.asList(leftInput, rightInput));
 
-            case SEMI:
-                node = new SemiJoinNode<>(ctx, cond);
+        return node;
+    }
 
-                break;
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteCorrelatedNestedLoopJoin rel) {
+        RelDataType rowType = combinedRowType(ctx.getTypeFactory(), rel.getLeft().getRowType(), rel.getRight().getRowType());
 
-            case ANTI:
-                node = new AntiJoinNode<>(ctx, cond);
+        Predicate<Row> cond = expressionFactory.predicate(rel.getCondition(), rowType);
 
-                break;
+        assert rel.getJoinType() == JoinRelType.INNER; // TODO LEFT, SEMI, ANTI
 
-            default:
-                throw new IllegalStateException("Join type \"" + rel.getJoinType() + "\" is not supported yet");
-        }
+        Node<Row> node = new CorrelatedNestedLoopJoinNode<>(ctx, cond, rel.getVariablesSet());
 
         Node<Row> leftInput = visit(rel.getLeft());
         Node<Row> rightInput = visit(rel.getRight());
@@ -236,23 +214,21 @@ public class LogicalRelImplementor<Row> implements IgniteRelVisitor<Node<Row>> {
     }
 
     /** {@inheritDoc} */
-    @Override public Node<Row> visit(IgniteTableScan scan) {
-        Predicate<Row> filters = scan.condition() == null ? null :
-            expressionFactory.predicate(scan.condition(), scan.getRowType());
+    @Override public Node<Row> visit(IgniteIndexScan rel) {
+        RexNode condition = rel.condition();
+        Predicate<Row> filters = condition == null ? null : expressionFactory.predicate(condition, rel.getRowType());
 
-        List<RexNode> lowerCond = scan.lowerIndexCondition();
-        Row lowerBound = lowerCond == null ? null :
-            expressionFactory.asRow(lowerCond, scan.getRowType());
+        List<RexNode> lowerCond = rel.lowerIndexCondition();
+        Supplier<Row> lower = lowerCond == null ? null : expressionFactory.rowSource(lowerCond);
 
-        List<RexNode> upperCond = scan.upperIndexCondition();
-        Row upperBound = upperCond == null ? null :
-            expressionFactory.asRow(upperCond, scan.getRowType());
+        List<RexNode> upperCond = rel.upperIndexCondition();
+        Supplier<Row> upper = upperCond == null ? null : expressionFactory.rowSource(upperCond);
 
-        IgniteTable tbl = scan.igniteTable();
+        IgniteTable tbl = rel.igniteTable();
 
-        IgniteIndex idx = tbl.getIndex(scan.indexName());
+        IgniteIndex idx = tbl.getIndex(rel.indexName());
 
-        Iterable<Row> rowsIter = idx.scan(ctx, filters, lowerBound, upperBound);
+        Iterable<Row> rowsIter = idx.scan(ctx, filters, lower, upper);
 
         return new ScanNode<>(ctx, rowsIter);
     }
@@ -391,10 +367,10 @@ public class LogicalRelImplementor<Row> implements IgniteRelVisitor<Node<Row>> {
     private Predicate<Row> partitionFilter(IgniteDistribution distr) {
         assert distr.getType() == RelDistribution.Type.HASH_DISTRIBUTED;
 
-        ImmutableBitSet filter = ImmutableBitSet.of(ctx.partitions());
+        ImmutableBitSet filter = ImmutableBitSet.of(ctx.localPartitions());
         DistributionFunction function = distr.function();
         ImmutableIntList keys = distr.getKeys();
-        ToIntFunction<Object> partFunction = function.partitionFunction(partSvc, ctx.partitionsCount(), keys);
+        ToIntFunction<Row> partFunction = function.partitionFunction(ctx, partSvc, keys);
 
         return o -> filter.get(partFunction.applyAsInt(o));
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
index f6e4e9b..6dac05c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
@@ -62,6 +62,9 @@ public interface RowHandler<Row> {
     @SuppressWarnings("PublicInnerClass")
     interface RowFactory<Row> {
         /** */
+        RowHandler<Row> handler();
+
+        /** */
         Row create();
 
         /** */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
index 69ee0a2..e731601 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
@@ -79,7 +79,7 @@ public interface ExpressionFactory<Row> {
      * @param values Values.
      * @return Row.
      */
-    Row asRow(List<RexNode> values, RelDataType rowType);
+    Supplier<Row> rowSource(List<RexNode> values);
 
     /**
      * Creates {@link Scalar}, a code-generated expressions evaluator.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
index 34b5284..27670d2 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
@@ -21,6 +21,7 @@ import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
@@ -33,6 +34,8 @@ import com.google.common.collect.Ordering;
 import com.google.common.primitives.Primitives;
 import org.apache.calcite.adapter.enumerable.EnumUtils;
 import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator.InputGetter;
+import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.tree.BlockBuilder;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
@@ -45,10 +48,12 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCorrelVariable;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
@@ -147,6 +152,11 @@ public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
     }
 
     /** {@inheritDoc} */
+    @Override public Supplier<Row> rowSource(List<RexNode> values) {
+        return new ValuesImpl(scalar(values, null), ctx.rowHandler().factory(typeFactory, RexUtil.types(values)));
+    }
+
+    /** {@inheritDoc} */
     @Override public Iterable<Row> values(List<RexLiteral> values, RelDataType rowType) {
         RowHandler<Row> handler = ctx.rowHandler();
         RowFactory<Row> factory = handler.factory(typeFactory, rowType);
@@ -175,21 +185,23 @@ public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
     }
 
     /** {@inheritDoc} */
-    @Override public Row asRow(List<RexNode> values, RelDataType rowType) {
-        RowFactory<Row> factory = ctx.rowHandler().factory(typeFactory, rowType);
-        ProjectImpl project = new ProjectImpl(scalar(values, rowType), factory);
-
-        return project.apply(factory.create());
-    }
-
-    /** {@inheritDoc} */
     @Override public Scalar scalar(List<RexNode> nodes, RelDataType type) {
         return SCALAR_CACHE.computeIfAbsent(digest(nodes, type), k -> compile(nodes, type));
     }
 
     /** */
     private Scalar compile(Iterable<RexNode> nodes, RelDataType type) {
-        RelDataType rowType = type == null ? emptyType : type;
+        if (type == null)
+            type = emptyType;
+
+        RexProgramBuilder programBuilder = new RexProgramBuilder(type, rexBuilder);
+
+        for (RexNode node : nodes)
+            programBuilder.addProject(node, null);
+
+        RexProgram program = programBuilder.getProgram();
+
+        BlockBuilder builder = new BlockBuilder();
 
         ParameterExpression ctx_ =
             Expressions.parameter(ExecutionContext.class, "ctx");
@@ -200,37 +212,23 @@ public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
         ParameterExpression out_ =
             Expressions.parameter(Object.class, "out");
 
-        RexToLixTranslator.InputGetter inputGetter = new FieldGetter(ctx_, in_, rowType);
-
-//        RexToLixTranslator.InputGetter inputGetter =
-//            new RexToLixTranslator.InputGetterImpl(
-//                ImmutableList.of(
-//                    Pair.of(in_,
-//                        PhysTypeImpl.of(typeFactory, type,
-//                            JavaRowFormat.ARRAY, false))));
-
-        RexProgramBuilder programBuilder = new RexProgramBuilder(rowType, rexBuilder);
-
-        for (RexNode node : nodes)
-            programBuilder.addProject(node, null);
-
-        RexProgram program = programBuilder.getProgram();
-
-        BlockBuilder builder = new BlockBuilder();
-
-        Expression handler_ = builder.append("hnd",
+        Expression hnd_ = builder.append("hnd",
             Expressions.call(ctx_,
                 IgniteMethod.CONTEXT_ROW_HANDLER.method()));
 
-        List<Expression> list = RexToLixTranslator.translateProjects(program,
-            typeFactory, conformance, builder, null, ctx_, inputGetter, null);
+        InputGetter inputGetter = new FieldGetter(hnd_, in_, type);
+
+        Function1<String, InputGetter> correlates = new CorrelatesBuilder(builder, ctx_, hnd_).build(nodes);
+
+        List<Expression> projects = RexToLixTranslator.translateProjects(program, typeFactory, conformance,
+            builder, null, ctx_, inputGetter, correlates);
 
-        for (int i = 0; i < list.size(); i++) {
+        for (int i = 0; i < projects.size(); i++) {
             builder.add(
                 Expressions.statement(
-                    Expressions.call(handler_,
+                    Expressions.call(hnd_,
                         IgniteMethod.ROW_HANDLER_SET.method(),
-                            Expressions.constant(i), out_, list.get(i))));
+                            Expressions.constant(i), out_, projects.get(i))));
         }
 
         MethodDeclaration decl = Expressions.methodDecl(
@@ -316,33 +314,53 @@ public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
     }
 
     /** */
-    private class FieldGetter implements RexToLixTranslator.InputGetter {
+    private class ValuesImpl implements Supplier<Row> {
         /** */
-        private final Expression ctx;
+        private final Scalar scalar;
 
         /** */
-        private final Expression row;
+        private final RowFactory<Row> factory;
+
+        /** */
+        private ValuesImpl(Scalar scalar, RowFactory<Row> factory) {
+            this.scalar = scalar;
+            this.factory = factory;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Row get() {
+            Row res = factory.create();
+            scalar.execute(ctx, null, res);
+
+            return res;
+        }
+    }
+
+    /** */
+    private class FieldGetter implements InputGetter {
+        /** */
+        private final Expression hnd_;
+
+        /** */
+        private final Expression row_;
 
         /** */
         private final RelDataType rowType;
 
         /** */
-        private FieldGetter(Expression ctx, Expression row, RelDataType rowType) {
-            this.ctx = ctx;
-            this.row = row;
+        private FieldGetter(Expression hnd_, Expression row_, RelDataType rowType) {
+            this.hnd_ = hnd_;
+            this.row_ = row_;
             this.rowType = rowType;
         }
 
         /** {@inheritDoc} */
         @Override public Expression field(BlockBuilder list, int index, Type desiredType) {
-            Expression row = list.append("row", this.row);
-            Expression hnd = list.append("hnd",
-                Expressions.call(ctx,
-                    IgniteMethod.CONTEXT_ROW_HANDLER.method()));
+            Expression row_ = list.append("row", this.row_);
 
-            Expression field = Expressions.call(hnd,
+            Expression field = Expressions.call(hnd_,
                 IgniteMethod.ROW_HANDLER_GET.method(),
-                    Expressions.constant(index), row);
+                    Expressions.constant(index), row_);
 
             Type fieldType = typeFactory.getJavaClass(rowType.getFieldList().get(index).getType());
 
@@ -358,4 +376,53 @@ public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
             return EnumUtils.convert(field, fieldType, desiredType);
         }
     }
+
+    /** */
+    private class CorrelatesBuilder extends RexShuttle {
+        /** */
+        private final BlockBuilder builder;
+
+        /** */
+        private final Expression ctx_;
+
+        /** */
+        private final Expression hnd_;
+
+        /** */
+        private Map<String, FieldGetter> correlates;
+
+        /** */
+        public CorrelatesBuilder(BlockBuilder builder, Expression ctx_, Expression hnd_) {
+            this.builder = builder;
+            this.hnd_ = hnd_;
+            this.ctx_ = ctx_;
+        }
+
+        /** */
+        public Function1<String, InputGetter> build(Iterable<RexNode> nodes) {
+            try {
+                for (RexNode node : nodes)
+                    node.accept(this);
+
+                return correlates == null ? null : correlates::get;
+            }
+            finally {
+                correlates = null;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public RexNode visitCorrelVariable(RexCorrelVariable variable) {
+            Expression corr_ = builder.append("corr",
+                Expressions.call(ctx_, IgniteMethod.CONTEXT_GET_CORRELATED_VALUE.method(),
+                    Expressions.constant(variable.id.getId())));
+
+            if (correlates == null)
+                correlates = new HashMap<>();
+
+            correlates.put(variable.getName(), new FieldGetter(hnd_, corr_, variable.getType()));
+
+            return variable;
+        }
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
index 521be1a..09344e6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
@@ -50,11 +50,14 @@ import org.apache.ignite.IgniteException;
 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.rel.AggregateNode.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.util.typedef.F;
 import org.jetbrains.annotations.NotNull;
 
+import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.createRowType;
+
 /** */
 public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrapper<Row>>> {
     /** */
@@ -86,7 +89,7 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
 
     /** */
     private static Function<Object, Object> cast0(Pair<RelDataType,RelDataType> types) {
-        IgniteTypeFactory typeFactory = new IgniteTypeFactory();
+        IgniteTypeFactory typeFactory = PlanningContext.empty().typeFactory();
 
         RelDataType from = types.left;
         RelDataType to = types.right;
@@ -97,8 +100,13 @@ public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrappe
         if (fromType == toType)
             return Function.identity();
 
-        RelDataType rowType = typeFactory.createStructType(F.asList(types.left), F.asList("$EXPR"));
+        return compileCast(typeFactory, from, to);
+    }
 
+    /** */
+    private static Function<Object, Object> compileCast(IgniteTypeFactory typeFactory, RelDataType from,
+        RelDataType to) {
+        RelDataType rowType = createRowType(typeFactory, from);
         ParameterExpression in_ = Expressions.parameter(Object.class, "in");
 
         RexToLixTranslator.InputGetter getter =
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractJoinNode.java
deleted file mode 100644
index a64e48b..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractJoinNode.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * 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.ArrayList;
-import java.util.Deque;
-import java.util.List;
-import java.util.function.Predicate;
-
-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;
-
-/** */
-public abstract class AbstractJoinNode<Row> extends AbstractNode<Row> {
-    /** Special value to highlights that all row were received and we are not waiting any more. */
-    protected static final int NOT_WAITING = -1;
-
-    /** */
-    protected final Predicate<Row> cond;
-
-    /** */
-    protected final RowHandler<Row> handler;
-
-    /** */
-    protected int requested;
-
-    /** */
-    protected int waitingLeft;
-
-    /** */
-    protected int waitingRight;
-
-    /** */
-    protected final List<Row> rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE);
-
-    /** */
-    protected final Deque<Row> leftInBuf = new ArrayDeque<>(IN_BUFFER_SIZE);
-
-    /** */
-    private boolean inLoop;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    protected AbstractJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond) {
-        super(ctx);
-
-        this.cond = cond;
-        handler = ctx.rowHandler();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert !F.isEmpty(sources) && sources.size() == 2;
-        assert rowsCnt > 0 && requested == 0;
-
-        requested = rowsCnt;
-
-        if (!inLoop)
-            context().execute(this::body);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Downstream<Row> requestDownstream(int idx) {
-        if (idx == 0)
-            return new Downstream<Row>() {
-                /** {@inheritDoc} */
-                @Override public void push(Row row) {
-                    pushLeft(row);
-                }
-
-                /** {@inheritDoc} */
-                @Override public void end() {
-                    endLeft();
-                }
-
-                /** {@inheritDoc} */
-                @Override public void onError(Throwable e) {
-                    AbstractJoinNode.this.onError(e);
-                }
-            };
-        else if (idx == 1)
-            return new Downstream<Row>() {
-                /** {@inheritDoc} */
-                @Override public void push(Row row) {
-                    pushRight(row);
-                }
-
-                /** {@inheritDoc} */
-                @Override public void end() {
-                    endRight();
-                }
-
-                /** {@inheritDoc} */
-                @Override public void onError(Throwable e) {
-                    AbstractJoinNode.this.onError(e);
-                }
-            };
-
-        throw new IndexOutOfBoundsException();
-    }
-
-    /** */
-    private void pushLeft(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
-        assert waitingLeft > 0;
-
-        waitingLeft--;
-
-        leftInBuf.add(row);
-
-        body();
-    }
-
-    /** */
-    private void pushRight(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
-        assert waitingRight > 0;
-
-        waitingRight--;
-
-        rightMaterialized.add(row);
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-    }
-
-    /** */
-    private void endLeft() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
-        assert waitingLeft > 0;
-
-        waitingLeft = NOT_WAITING;
-
-        body();
-    }
-
-    /** */
-    private void endRight() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
-        assert waitingRight > 0;
-
-        waitingRight = NOT_WAITING;
-
-        body();
-    }
-
-    /** */
-    private void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
-
-        downstream.onError(e);
-    }
-
-    /** */
-    private void body() {
-        inLoop = true;
-        try {
-            doJoin();
-        }
-        catch (Exception e) {
-            downstream.onError(e);
-        }
-        finally {
-            inLoop = false;
-        }
-    }
-
-    /** */
-    protected abstract void doJoin();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
index bd1ad5e..c47321f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
@@ -21,7 +21,10 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
 
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -50,16 +53,16 @@ public abstract class AbstractNode<Row> implements Node<Row> {
      * creates on first message received from a remote source. This case the context
      * sets in scope of {@link Inbox#init(ExecutionContext, Collection, Comparator)} method call.
      */
-    protected ExecutionContext<Row> ctx;
+    private volatile ExecutionContext<Row> ctx;
 
     /** */
-    protected Downstream<Row> downstream;
+    private Downstream<Row> downstream;
 
     /** */
-    protected List<Node<Row>> sources;
+    private boolean closed;
 
     /** */
-    protected boolean closed;
+    private List<Node<Row>> sources;
 
     /**
      * @param ctx Execution context.
@@ -68,11 +71,20 @@ public abstract class AbstractNode<Row> implements Node<Row> {
         this.ctx = ctx;
     }
 
-    /** {@inheritDoc} */
+    /**
+     * {@link Inbox} node may not have proper context at creation time in case it
+     * creates on first message received from a remote source. This case the context
+     * sets in scope of {@link Inbox#init(ExecutionContext, Collection, Comparator)} method call.
+     */ /** {@inheritDoc} */
     @Override public ExecutionContext<Row> context() {
         return ctx;
     }
 
+    /** */
+    protected void context(ExecutionContext<Row> ctx) {
+        this.ctx = ctx;
+    }
+
     /** {@inheritDoc} */
     @Override public void register(List<Node<Row>> sources) {
         this.sources = sources;
@@ -82,21 +94,62 @@ public abstract class AbstractNode<Row> implements Node<Row> {
     }
 
     /** {@inheritDoc} */
-    @Override public void onRegister(Downstream<Row> downstream) {
-        this.downstream = downstream;
+    @Override public List<Node<Row>> sources() {
+        return sources;
     }
 
     /** {@inheritDoc} */
     @Override public void close() {
-        checkThread();
+        if (isClosed())
+            return;
+
+        onClose();
+
+        if (!F.isEmpty(sources()))
+            sources().forEach(U::closeQuiet);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rewind() {
+        onRewind();
+
+        if (!F.isEmpty(sources()))
+            sources().forEach(Node::rewind);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onRegister(Downstream<Row> downstream) {
+        this.downstream = downstream;
+    }
+
+    /** */
+    protected abstract void onRewind();
+
+    /**
+     * Processes given exception.
+     *
+     * @param e Exception.
+     */
+    public void onError(Throwable e) {
+        assert downstream() != null;
+
+        if (e instanceof ExecutionCancelledException) {
+            U.warn(context().planningContext().logger(), "Execution is cancelled.", e);
 
-        if (closed)
             return;
+        }
+
+        try {
+            downstream().onError(e);
+        }
+        finally {
+            U.closeQuiet(this);
+        }
+    }
 
+    /** */
+    protected void onClose() {
         closed = true;
-
-        if (!F.isEmpty(sources))
-            sources.forEach(U::closeQuiet);
     }
 
     /**
@@ -107,16 +160,24 @@ public abstract class AbstractNode<Row> implements Node<Row> {
     }
 
     /** */
-    protected abstract Downstream<Row> requestDownstream(int idx);
-
-    /** */
-    protected void checkThread() {
+    protected void checkState() throws IgniteCheckedException {
+        if (isClosed())
+            throw new ExecutionCancelledException();
+        if (Thread.interrupted())
+            throw new IgniteInterruptedCheckedException("Thread was interrupted.");
         if (!U.assertionsEnabled())
             return;
-
         if (thread == null)
             thread = Thread.currentThread();
         else
             assert thread == Thread.currentThread();
     }
+
+    /** */
+    protected abstract Downstream<Row> requestDownstream(int idx);
+
+    /** */
+    @Override public Downstream<Row> downstream() {
+        return downstream;
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AggregateNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AggregateNode.java
index f496b56..8d300b6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AggregateNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AggregateNode.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.function.Supplier;
 
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
@@ -37,6 +38,9 @@ import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.util.typedef.F;
 
+import static java.util.stream.Collectors.toCollection;
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.negate;
+
 /**
  *
  */
@@ -95,75 +99,68 @@ public class AggregateNode<Row> extends AbstractNode<Row> implements SingleNode<
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert !F.isEmpty(sources()) && sources().size() == 1;
         assert rowsCnt > 0 && requested == 0;
+        assert waiting <= 0;
+
+        try {
+            checkState();
 
-        requested = rowsCnt;
+            requested = rowsCnt;
 
-        if (waiting == -1 && !inLoop)
-            context().execute(this::flushFromBuffer);
-        else if (waiting == 0)
-            F.first(sources).request(waiting = IN_BUFFER_SIZE);
-        else
-            throw new AssertionError();
+            if (waiting == 0)
+                source().request(waiting = IN_BUFFER_SIZE);
+            else if (!inLoop)
+                context().execute(this::doFlush);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void push(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting--;
-
         try {
+            checkState();
+
+            waiting--;
+
             for (Grouping grouping : groupings)
                 grouping.add(row);
 
             if (waiting == 0)
-                F.first(sources).request(waiting = IN_BUFFER_SIZE);
+                source().request(waiting = IN_BUFFER_SIZE);
         }
         catch (Exception e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting = -1;
-
         try {
-            flushFromBuffer();
+            checkState();
+
+            waiting = -1;
+
+            flush();
         }
         catch (Exception e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
-
-        downstream.onError(e);
+    @Override protected void onRewind() {
+        requested = 0;
+        waiting = 0;
+        groupings.forEach(grouping -> grouping.groups.clear());
     }
 
     /** {@inheritDoc} */
@@ -175,30 +172,43 @@ public class AggregateNode<Row> extends AbstractNode<Row> implements SingleNode<
     }
 
     /** */
-    public void flushFromBuffer() {
+    private void doFlush() {
+        try {
+            checkState();
+
+            flush();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void flush() throws IgniteCheckedException {
         assert waiting == -1;
 
+        int processed = 0;
+        ArrayDeque<Grouping> groupingsQueue = groupingsQueue();
+
         inLoop = true;
         try {
-            int processed = 0;
-
-            ArrayDeque<Grouping> groupingsQueue = groupingsQueue();
-
             while (requested > 0 && !groupingsQueue.isEmpty()) {
                 Grouping grouping = groupingsQueue.peek();
 
                 int toSnd = Math.min(requested, IN_BUFFER_SIZE - processed);
 
                 for (Row row : grouping.getRows(toSnd)) {
+                    checkState();
+
                     requested--;
-                    downstream.push(row);
+                    downstream().push(row);
 
                     processed++;
                 }
 
                 if (processed >= IN_BUFFER_SIZE && requested > 0) {
                     // allow others to do their job
-                    context().execute(this::flushFromBuffer);
+                    context().execute(this::doFlush);
 
                     return;
                 }
@@ -206,27 +216,22 @@ public class AggregateNode<Row> extends AbstractNode<Row> implements SingleNode<
                 if (grouping.isEmpty())
                     groupingsQueue.remove();
             }
-
-            if (requested > 0) {
-                downstream.end();
-                requested = 0;
-            }
         }
         finally {
             inLoop = false;
         }
+
+        if (requested > 0) {
+            requested = 0;
+            downstream().end();
+        }
     }
 
     /** */
     private ArrayDeque<Grouping> groupingsQueue() {
-        ArrayDeque<Grouping> res = new ArrayDeque<>(groupings.size());
-
-        for (Grouping grouping : groupings) {
-            if (!grouping.isEmpty())
-                res.add(grouping);
-        }
-
-        return res;
+        return groupings.stream()
+            .filter(negate(Grouping::isEmpty))
+            .collect(toCollection(ArrayDeque::new));
     }
 
     /** */
@@ -261,7 +266,7 @@ public class AggregateNode<Row> extends AbstractNode<Row> implements SingleNode<
             this.grpId = grpId;
             this.grpFields = grpFields;
 
-            handler = ctx.rowHandler();
+            handler = context().rowHandler();
         }
 
         /** */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AntiJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AntiJoinNode.java
deleted file mode 100644
index 17f4e97..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AntiJoinNode.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.function.Predicate;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-
-/** */
-public class AntiJoinNode<Row> extends AbstractJoinNode<Row> {
-    /** */
-    private Row left;
-
-    /** */
-    private int rightIdx;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    public AntiJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond) {
-        super(ctx, cond);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void doJoin() {
-        if (waitingRight == NOT_WAITING) {
-            while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
-                if (left == null)
-                    left = leftInBuf.remove();
-
-                boolean matched = false;
-
-                while (!matched && rightIdx < rightMaterialized.size()) {
-                    Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
-
-                    if (cond.test(row))
-                        matched = true;
-                }
-
-                if (!matched) {
-                    requested--;
-                    downstream.push(left);
-                }
-
-                left = null;
-                rightIdx = 0;
-            }
-        }
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-
-        if (waitingLeft == 0 && leftInBuf.isEmpty())
-            sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
-
-        if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) {
-            downstream.end();
-            requested = 0;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java
new file mode 100644
index 0000000..c88a9a8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java
@@ -0,0 +1,464 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.ignite.IgniteCheckedException;
+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;
+
+/**
+ *
+ */
+public class CorrelatedNestedLoopJoinNode<Row> extends AbstractNode<Row> {
+    /** */
+    private final Predicate<Row> cond;
+
+    /** */
+    private final List<CorrelationId> correlationIds;
+
+    /** */
+    private final RowHandler<Row> handler;
+
+    /** */
+    private final int leftInBufferSize;
+
+    /** */
+    private final int rightInBufferSize;
+
+    /** */
+    private int requested;
+
+    /** */
+    private int waitingLeft;
+
+    /** */
+    private int waitingRight;
+
+    /** */
+    private List<Row> leftInBuf;
+
+    /** */
+    private List<Row> rightInBuf;
+
+    /** */
+    private int leftIdx;
+
+    /** */
+    private int rightIdx;
+
+    /** */
+    private State state = State.INITIAL;
+
+    /** */
+    private enum State {
+        INITIAL, FILLING_LEFT, FILLING_RIGHT, IDLE, IN_LOOP, END
+    }
+
+    /**
+     * @param ctx Execution context.
+     * @param cond Join expression.
+     */
+    public CorrelatedNestedLoopJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond, Set<CorrelationId> correlationIds) {
+        super(ctx);
+
+        assert !F.isEmpty(correlationIds);
+
+        this.cond = cond;
+        this.correlationIds = new ArrayList<>(correlationIds);
+
+        leftInBufferSize = correlationIds.size();
+        rightInBufferSize = IN_BUFFER_SIZE;
+
+        handler = ctx.rowHandler();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) {
+        assert !F.isEmpty(sources()) && sources().size() == 2;
+        assert rowsCnt > 0 && requested == 0;
+
+        try {
+            checkState();
+
+            requested = rowsCnt;
+
+            onRequest();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        leftInBuf = null;
+        rightInBuf = null;
+
+        leftIdx = 0;
+        rightIdx = 0;
+
+        requested = 0;
+        waitingLeft = 0;
+        waitingRight = 0;
+
+        state = State.INITIAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        if (idx == 0)
+            return new Downstream<Row>() {
+                /** {@inheritDoc} */
+                @Override public void push(Row row) {
+                    pushLeft(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() {
+                    endLeft();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    CorrelatedNestedLoopJoinNode.this.onError(e);
+                }
+            };
+        else if (idx == 1)
+            return new Downstream<Row>() {
+                /** {@inheritDoc} */
+                @Override public void push(Row row) {
+                    pushRight(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() {
+                    endRight();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    CorrelatedNestedLoopJoinNode.this.onError(e);
+                }
+            };
+
+        throw new IndexOutOfBoundsException();
+    }
+
+    /** */
+    private void pushLeft(Row row) {
+        assert downstream() != null;
+        assert waitingLeft > 0;
+
+        try {
+            checkState();
+
+            waitingLeft--;
+
+            if (leftInBuf == null)
+                leftInBuf = new ArrayList<>(leftInBufferSize);
+
+            leftInBuf.add(row);
+
+            onPushLeft();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void pushRight(Row row) {
+        assert downstream() != null;
+        assert waitingRight > 0;
+
+        try {
+            checkState();
+
+            waitingRight--;
+
+            if (rightInBuf == null)
+                rightInBuf = new ArrayList<>(rightInBufferSize);
+
+            rightInBuf.add(row);
+
+            onPushRight();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void endLeft() {
+        assert downstream() != null;
+        assert waitingLeft > 0;
+
+        try {
+            checkState();
+
+            waitingLeft = -1;
+
+            if (leftInBuf == null)
+                leftInBuf = Collections.emptyList();
+
+            onEndLeft();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void endRight() {
+        assert downstream() != null;
+        assert waitingRight > 0;
+
+        try {
+            checkState();
+
+            waitingRight = -1;
+
+            if (rightInBuf == null)
+                rightInBuf = Collections.emptyList();
+
+            onEndRight();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void onRequest() {
+        switch (state) {
+            case IN_LOOP:
+                break;
+            case INITIAL:
+                assert waitingLeft == 0;
+                assert waitingRight == 0;
+                assert F.isEmpty(leftInBuf);
+                assert F.isEmpty(rightInBuf);
+
+                context().execute(() -> {
+                    try {
+                        checkState();
+
+                        state = State.FILLING_LEFT;
+                        leftSource().request(waitingLeft = leftInBufferSize);
+                    }
+                    catch (Exception e) {
+                        onError(e);
+                    }
+                });
+
+                break;
+            case IDLE:
+                assert rightInBuf != null;
+                assert leftInBuf != null;
+                assert waitingRight == -1 || waitingRight == 0 && rightInBuf.size() == rightInBufferSize;
+                assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+                context().execute(() -> {
+                    try {
+                        checkState();
+
+                        join();
+                    }
+                    catch (Exception e) {
+                        onError(e);
+                    }
+                });
+
+                break;
+            default:
+                throw new AssertionError("Unexpected state:" + state);
+        }
+    }
+
+    /** */
+    private void onPushLeft() {
+        assert state == State.FILLING_LEFT : "Unexpected state:" + state;
+        assert waitingRight == 0 || waitingRight == -1;
+        assert F.isEmpty(rightInBuf);
+
+        if (leftInBuf.size() == leftInBufferSize) {
+            assert waitingLeft == 0;
+
+            prepareCorrelations();
+
+            if (waitingRight == -1)
+                rightSource().rewind();
+
+            state = State.FILLING_RIGHT;
+            rightSource().request(waitingRight = rightInBufferSize);
+        }
+    }
+
+    /** */
+    private void onPushRight() throws IgniteCheckedException {
+        assert state == State.FILLING_RIGHT : "Unexpected state:" + state;
+        assert !F.isEmpty(leftInBuf);
+        assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+        if (rightInBuf.size() == rightInBufferSize) {
+            assert waitingRight == 0;
+
+            state = State.IDLE;
+
+            join();
+        }
+    }
+
+    /** */
+    private void onEndLeft() {
+        assert state == State.FILLING_LEFT : "Unexpected state:" + state;
+        assert waitingLeft == -1;
+        assert waitingRight == 0 || waitingRight == -1;
+        assert F.isEmpty(rightInBuf);
+
+        if (F.isEmpty(leftInBuf)) {
+            waitingRight = -1;
+            state = State.END;
+
+            requested = 0;
+            downstream().end();
+        }
+        else {
+            prepareCorrelations();
+
+            if (waitingRight == -1)
+                rightSource().rewind();
+
+            state = State.FILLING_RIGHT;
+            rightSource().request(waitingRight = rightInBufferSize);
+        }
+    }
+
+    /** */
+    private void onEndRight() throws IgniteCheckedException {
+        assert state == State.FILLING_RIGHT : "Unexpected state:" + state;
+        assert waitingRight == -1;
+        assert !F.isEmpty(leftInBuf);
+        assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+        state = State.IDLE;
+
+        join();
+    }
+
+    /** */
+    private void join() throws IgniteCheckedException {
+        assert state == State.IDLE;
+
+        state = State.IN_LOOP;
+        try {
+            while (requested > 0 && rightIdx < rightInBuf.size()) {
+                if (leftIdx == leftInBuf.size())
+                    leftIdx = 0;
+
+                while (requested > 0 && leftIdx < leftInBuf.size()) {
+                    checkState();
+
+                    Row row = handler.concat(leftInBuf.get(leftIdx), rightInBuf.get(rightIdx));
+
+                    leftIdx++;
+
+                    if (cond.test(row)) {
+                        requested--;
+                        downstream().push(row);
+                    }
+                }
+
+                if (leftIdx == leftInBuf.size())
+                    rightInBuf.set(rightIdx++, null);
+            }
+        }
+        finally {
+            state = State.IDLE;
+        }
+
+        if (rightIdx == rightInBuf.size()) {
+            leftIdx = 0;
+            rightIdx = 0;
+
+            if (waitingRight == 0) {
+                rightInBuf = null;
+
+                state = State.FILLING_RIGHT;
+
+                rightSource().request(waitingRight = rightInBufferSize);
+
+                return;
+            }
+
+            if (waitingLeft == 0) {
+                rightInBuf = null;
+                leftInBuf = null;
+
+                state = State.FILLING_LEFT;
+
+                leftSource().request(waitingLeft = leftInBufferSize);
+
+                return;
+            }
+
+            assert waitingLeft == -1 && waitingRight == -1;
+
+            if (requested > 0) {
+                leftInBuf = null;
+                rightInBuf = null;
+
+                state = State.END;
+
+                requested = 0;
+                downstream().end();
+
+                return;
+            }
+
+            // let's free the rows for GC
+            leftInBuf = Collections.emptyList();
+            rightInBuf = Collections.emptyList();
+        }
+    }
+
+    /** */
+    private Node<Row> leftSource() {
+        return sources().get(0);
+    }
+
+    /** */
+    private Node<Row> rightSource() {
+        return sources().get(1);
+    }
+
+    private void prepareCorrelations() {
+        for (int i = 0; i < correlationIds.size(); i++) {
+            Row row = i < leftInBuf.size() ? leftInBuf.get(i) : F.first(leftInBuf);
+            context().setCorrelated(row, correlationIds.get(i).getId());
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
index 19c87d3..ba8f5fe 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
@@ -21,6 +21,7 @@ import java.util.ArrayDeque;
 import java.util.Deque;
 import java.util.function.Predicate;
 
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.util.typedef.F;
 
@@ -55,73 +56,60 @@ public class FilterNode<Row> extends AbstractNode<Row> implements SingleNode<Row
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert !F.isEmpty(sources()) && sources().size() == 1;
         assert rowsCnt > 0 && requested == 0;
 
-        requested = rowsCnt;
+        try {
+            checkState();
+
+            requested = rowsCnt;
 
-        if (!inLoop)
-            context().execute(this::flushFromBuffer);
+            if (!inLoop)
+                context().execute(this::doFilter);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void push(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting--;
-
         try {
+            checkState();
+
+            waiting--;
+
             if (pred.test(row))
                 inBuf.add(row);
 
-            flushFromBuffer();
+            filter();
         }
         catch (Exception e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting = -1;
-
         try {
-            flushFromBuffer();
+            checkState();
+
+            waiting = -1;
+
+            filter();
         }
         catch (Exception e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
-
-        downstream.onError(e);
-    }
-
-    /** {@inheritDoc} */
     @Override protected Downstream<Row> requestDownstream(int idx) {
         if (idx != 0)
             throw new IndexOutOfBoundsException();
@@ -129,27 +117,48 @@ public class FilterNode<Row> extends AbstractNode<Row> implements SingleNode<Row
         return this;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        requested = 0;
+        waiting = 0;
+        inBuf.clear();
+    }
+
     /** */
-    public void flushFromBuffer() {
-        inLoop = true;
+    private void doFilter() {
         try {
-            while (requested > 0 && !inBuf.isEmpty()) {
-                requested--;
-                downstream.push(inBuf.remove());
-            }
+            checkState();
 
-            if (inBuf.isEmpty() && waiting == 0)
-                F.first(sources).request(waiting = IN_BUFFER_SIZE);
+            filter();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
 
-            if (waiting == -1 && requested > 0) {
-                assert inBuf.isEmpty();
+    /** */
+    private void filter() throws IgniteCheckedException {
+        inLoop = true;
+        try {
+            while (requested > 0 && !inBuf.isEmpty()) {
+                checkState();
 
-                downstream.end();
-                requested = 0;
+                requested--;
+                downstream().push(inBuf.remove());
             }
         }
         finally {
             inLoop = false;
         }
+
+        if (inBuf.isEmpty() && waiting == 0)
+            source().request(waiting = IN_BUFFER_SIZE);
+
+        if (waiting == -1 && requested > 0) {
+            assert inBuf.isEmpty();
+
+            requested = 0;
+            downstream().end();
+        }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FullOuterJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FullOuterJoinNode.java
deleted file mode 100644
index 2e999d3..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FullOuterJoinNode.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * 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.BitSet;
-import java.util.function.Predicate;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
-
-/** */
-public class FullOuterJoinNode<Row> extends AbstractJoinNode<Row> {
-    /** Left row factory. */
-    private final RowHandler.RowFactory<Row> leftRowFactory;
-
-    /** Right row factory. */
-    private final RowHandler.RowFactory<Row> rightRowFactory;
-
-    /** Whether current left row was matched or not. */
-    private boolean leftMatched;
-
-    /** */
-    private BitSet rightNotMatchedIndexes;
-
-    /** */
-    private int lastPushedInd;
-
-    /** */
-    private Row left;
-
-    /** */
-    private int rightIdx;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    public FullOuterJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond, RowHandler.RowFactory<Row> leftRowFactory,
-        RowHandler.RowFactory<Row> rightRowFactory) {
-        super(ctx, cond);
-
-        this.leftRowFactory = leftRowFactory;
-        this.rightRowFactory = rightRowFactory;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void doJoin() {
-        if (waitingRight == NOT_WAITING) {
-            if (rightNotMatchedIndexes == null) {
-                rightNotMatchedIndexes = new BitSet(rightMaterialized.size());
-
-                rightNotMatchedIndexes.set(0, rightMaterialized.size());
-            }
-
-            while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
-                if (left == null) {
-                    left = leftInBuf.remove();
-
-                    leftMatched = false;
-                }
-
-                while (requested > 0 && rightIdx < rightMaterialized.size()) {
-                    Row right = rightMaterialized.get(rightIdx++);
-                    Row joined = handler.concat(left, right);
-
-                    if (!cond.test(joined))
-                        continue;
-
-                    requested--;
-                    leftMatched = true;
-                    rightNotMatchedIndexes.clear(rightIdx - 1);
-                    downstream.push(joined);
-                }
-
-                if (rightIdx == rightMaterialized.size()) {
-                    boolean wasPushed = false;
-
-                    if (!leftMatched && requested > 0) {
-                        requested--;
-                        wasPushed = true;
-
-                        downstream.push(handler.concat(left, rightRowFactory.create()));
-                    }
-
-                    if (leftMatched || wasPushed) {
-                        left = null;
-                        rightIdx = 0;
-                    }
-                }
-            }
-        }
-
-        if (waitingLeft == NOT_WAITING && requested > 0 && !rightNotMatchedIndexes.isEmpty()) {
-            assert lastPushedInd >= 0;
-
-            for (lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd);;
-                lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd + 1)
-            ) {
-                if (lastPushedInd < 0)
-                    break;
-
-                Row row = handler.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd));
-
-                rightNotMatchedIndexes.clear(lastPushedInd);
-
-                requested--;
-                downstream.push(row);
-
-                if (lastPushedInd == Integer.MAX_VALUE || requested <= 0)
-                    break;
-            }
-        }
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-
-        if (waitingLeft == 0 && leftInBuf.isEmpty())
-            sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
-
-        if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null
-            && leftInBuf.isEmpty() && rightNotMatchedIndexes.isEmpty()) {
-            downstream.end();
-            requested = 0;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
index 5e7e281..58c084e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
@@ -111,7 +111,7 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
         // It's important to set proper context here because
         // because the one, that is created on a first message
         // received doesn't have all context variables in place.
-        this.ctx = ctx;
+        context(ctx);
         this.comp = comp;
 
         // memory barier
@@ -120,28 +120,27 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
         assert srcNodeIds != null;
         assert rowsCnt > 0 && requested == 0;
 
-        requested = rowsCnt;
+        try {
+            checkState();
 
-        if (!inLoop)
-            context().execute(this::pushInternal);
+            requested = rowsCnt;
+
+            if (!inLoop)
+                context().execute(this::doPush);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void close() {
-        if (isClosed())
-            return;
+    @Override public void onClose() {
+        super.onClose();
 
         registry.unregister(this);
-
-        super.close();
     }
 
     /** {@inheritDoc} */
@@ -154,6 +153,11 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
         throw new UnsupportedOperationException();
     }
 
+    /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        throw new UnsupportedOperationException();
+    }
+
     /**
      * Pushes a batch into a buffer.
      *
@@ -163,66 +167,52 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
      * @param rows Rows.
      */
     public void onBatchReceived(UUID src, int batchId, boolean last, List<Row> rows) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        Buffer buf = getOrCreateBuffer(src);
-
-        boolean waitingBefore = buf.check() == State.WAITING;
-
-        buf.offer(batchId, last, rows);
-
-        if (requested > 0 && waitingBefore && buf.check() != State.WAITING)
-            pushInternal();
-    }
+        try {
+            checkState();
 
-    /**
-     * @param e Error.
-     */
-    private void onError(Throwable e) {
-        checkThread();
+            Buffer buf = getOrCreateBuffer(src);
 
-        assert downstream != null;
+            boolean waitingBefore = buf.check() == State.WAITING;
 
-        downstream.onError(e);
+            buf.offer(batchId, last, rows);
 
-        close();
+            if (requested > 0 && waitingBefore && buf.check() != State.WAITING)
+                push();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** */
-    private void pushInternal() {
-        if (isClosed())
-            return;
-
-        assert downstream != null;
-
-        inLoop = true;
-
+    private void doPush() {
         try {
-            if (buffers == null) {
-                for (UUID node : srcNodeIds)
-                    checkNode(node);
-
-                buffers = srcNodeIds.stream()
-                    .map(this::getOrCreateBuffer)
-                    .collect(Collectors.toList());
+            checkState();
 
-                assert buffers.size() == perNodeBuffers.size();
-            }
-
-            if (comp != null)
-                pushOrdered();
-            else
-                pushUnordered();
+            push();
         }
-        catch (Throwable e) {
+        catch (Exception e) {
             onError(e);
         }
-        finally {
-            inLoop = false;
+    }
+
+    /** */
+    private void push() throws IgniteCheckedException {
+        if (buffers == null) {
+            for (UUID node : srcNodeIds)
+                checkNode(node);
+
+            buffers = srcNodeIds.stream()
+                .map(this::getOrCreateBuffer)
+                .collect(Collectors.toList());
+
+            assert buffers.size() == perNodeBuffers.size();
         }
+
+        if (comp != null)
+            pushOrdered();
+        else
+            pushUnordered();
     }
 
     /** */
@@ -250,35 +240,40 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
             }
         }
 
-        while (requested > 0 && !heap.isEmpty()) {
-            if (isClosed())
-                return;
+        inLoop = true;
+        try {
+            while (requested > 0 && !heap.isEmpty()) {
+                checkState();
 
-            Buffer buf = heap.poll().right;
+                Buffer buf = heap.poll().right;
 
-            requested--;
-            downstream.push(buf.remove());
+                requested--;
+                downstream().push(buf.remove());
 
-            switch (buf.check()) {
-                case END:
-                    buffers.remove(buf);
+                switch (buf.check()) {
+                    case END:
+                        buffers.remove(buf);
 
-                    break;
-                case READY:
-                    heap.offer(Pair.of(buf.peek(), buf));
+                        break;
+                    case READY:
+                        heap.offer(Pair.of(buf.peek(), buf));
 
-                    break;
-                case WAITING:
+                        break;
+                    case WAITING:
 
-                    return;
+                        return;
+                }
             }
         }
+        finally {
+            inLoop = false;
+        }
 
         if (requested > 0 && heap.isEmpty()) {
             assert buffers.isEmpty();
 
-            downstream.end();
             requested = 0;
+            downstream().end();
         }
     }
 
@@ -286,37 +281,42 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
     private void pushUnordered() throws IgniteCheckedException {
         int idx = 0, noProgress = 0;
 
-        while (requested > 0 && !buffers.isEmpty()) {
-            if (isClosed())
-                return;
+        inLoop = true;
+        try {
+            while (requested > 0 && !buffers.isEmpty()) {
+                checkState();
+
+                Buffer buf = buffers.get(idx);
 
-            Buffer buf = buffers.get(idx);
+                switch (buf.check()) {
+                    case END:
+                        buffers.remove(idx--);
 
-            switch (buf.check()) {
-                case END:
-                    buffers.remove(idx--);
+                        break;
+                    case READY:
+                        noProgress = 0;
+                        requested--;
+                        downstream().push(buf.remove());
 
-                    break;
-                case READY:
-                    noProgress = 0;
-                    requested--;
-                    downstream.push(buf.remove());
+                        break;
+                    case WAITING:
+                        if (++noProgress >= buffers.size())
+                            return;
 
-                    break;
-                case WAITING:
-                    if (++noProgress >= buffers.size())
-                        return;
+                        break;
+                }
 
-                    break;
+                if (++idx == buffers.size())
+                    idx = 0;
             }
-
-            if (++idx == buffers.size())
-                idx = 0;
+        }
+        finally {
+            inLoop = false;
         }
 
         if (requested > 0 && buffers.isEmpty()) {
-            downstream.end();
             requested = 0;
+            downstream().end();
         }
     }
 
@@ -337,18 +337,23 @@ public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Singl
 
     /** */
     public void onNodeLeft(UUID nodeId) {
-        if (ctx.originatingNodeId().equals(nodeId) && srcNodeIds == null)
-            ctx.execute(this::close);
+        if (context().originatingNodeId().equals(nodeId) && srcNodeIds == null)
+            context().execute(this::close);
         else if (srcNodeIds != null && srcNodeIds.contains(nodeId))
-            ctx.execute(() -> onNodeLeft0(nodeId));
+            context().execute(() -> onNodeLeft0(nodeId));
     }
 
     /** */
     private void onNodeLeft0(UUID nodeId) {
-        checkThread();
+        try {
+            checkState();
 
-        if (getOrCreateBuffer(nodeId).check() != State.END)
-            onError(new ClusterTopologyCheckedException("Node left [nodeId=" + nodeId + ']'));
+            if (getOrCreateBuffer(nodeId).check() != State.END)
+                onError(new ClusterTopologyCheckedException("Node left [nodeId=" + nodeId + ']'));
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/InnerJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/InnerJoinNode.java
deleted file mode 100644
index 13623c2..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/InnerJoinNode.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.function.Predicate;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-
-/** */
-public class InnerJoinNode<Row> extends AbstractJoinNode<Row> {
-    /** */
-    private Row left;
-
-    /** */
-    private int rightIdx;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    public InnerJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond) {
-        super(ctx, cond);
-    }
-
-    /** */
-    @Override protected void doJoin() {
-        if (waitingRight == NOT_WAITING) {
-            while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
-                if (left == null)
-                    left = leftInBuf.remove();
-
-                while (requested > 0 && rightIdx < rightMaterialized.size()) {
-                    Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
-
-                    if (!cond.test(row))
-                        continue;
-
-                    requested--;
-                    downstream.push(row);
-                }
-
-                if (rightIdx == rightMaterialized.size()) {
-                    left = null;
-                    rightIdx = 0;
-                }
-            }
-        }
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-
-        if (waitingLeft == 0 && leftInBuf.isEmpty())
-            sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
-
-        if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) {
-            downstream.end();
-            requested = 0;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/LeftJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/LeftJoinNode.java
deleted file mode 100644
index e249a42..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/LeftJoinNode.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.function.Predicate;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
-
-/** */
-public class LeftJoinNode<Row> extends AbstractJoinNode<Row> {
-    /** Right row factory. */
-    private final RowHandler.RowFactory<Row> rightRowFactory;
-
-    /** Whether current left row was matched or not. */
-    private boolean matched;
-
-    /** */
-    private Row left;
-
-    /** */
-    private int rightIdx;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    public LeftJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond, RowHandler.RowFactory<Row> rightRowFactory) {
-        super(ctx, cond);
-
-        this.rightRowFactory = rightRowFactory;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void doJoin() {
-        if (waitingRight == NOT_WAITING) {
-            while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
-                if (left == null) {
-                    left = leftInBuf.remove();
-
-                    matched = false;
-                }
-
-                while (requested > 0 && rightIdx < rightMaterialized.size()) {
-                    Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
-
-                    if (!cond.test(row))
-                        continue;
-
-                    requested--;
-                    matched = true;
-                    downstream.push(row);
-                }
-
-                if (rightIdx == rightMaterialized.size()) {
-                    boolean wasPushed = false;
-
-                    if (!matched && requested > 0) {
-                        requested--;
-                        wasPushed = true;
-
-                        downstream.push(handler.concat(left, rightRowFactory.create()));
-                    }
-
-                    if (matched || wasPushed) {
-                        left = null;
-                        rightIdx = 0;
-                    }
-                }
-            }
-        }
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-
-        if (waitingLeft == 0 && leftInBuf.isEmpty())
-            sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
-
-        if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null && leftInBuf.isEmpty()) {
-            downstream.end();
-            requested = 0;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java
index 773a173..5336481 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
@@ -89,39 +90,40 @@ public class ModifyNode<Row> extends AbstractNode<Row> implements SingleNode<Row
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert !F.isEmpty(sources()) && sources().size() == 1;
         assert rowsCnt > 0 && requested == 0;
 
-        requested = rowsCnt;
+        try {
+            checkState();
 
-        if (!inLoop)
-            tryEnd();
+            requested = rowsCnt;
+
+            if (!inLoop)
+                tryEnd();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void push(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
         assert state == State.UPDATING;
 
-        waiting--;
-
         try {
+            checkState();
+
+            waiting--;
+
             switch (op) {
                 case DELETE:
                 case UPDATE:
                 case INSERT:
-                    addToBatch(row);
+                    tuples.add(desc.toTuple(context(), row, op, cols));
+
+                    flushTuples(false);
 
                     break;
                 default:
@@ -129,36 +131,34 @@ public class ModifyNode<Row> extends AbstractNode<Row> implements SingleNode<Row
             }
 
             if (waiting == 0)
-                F.first(sources).request(waiting = MODIFY_BATCH_SIZE);
+                source().request(waiting = MODIFY_BATCH_SIZE);
         }
         catch (Exception e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting = -1;
-        state = State.UPDATED;
+        try {
+            checkState();
+
+            waiting = -1;
+            state = State.UPDATED;
 
-        tryEnd();
+            tryEnd();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
-
-        downstream.onError(e);
+    @Override protected void onRewind() {
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
@@ -170,46 +170,36 @@ public class ModifyNode<Row> extends AbstractNode<Row> implements SingleNode<Row
     }
 
     /** */
-    private void addToBatch(Row row) throws IgniteCheckedException {
-        tuples.add(desc.toTuple(context(), row, op, cols));
-
-        flush(false);
-    }
-
-    /** */
-    private void tryEnd() {
-        assert downstream != null;
+    private void tryEnd() throws IgniteCheckedException {
+        assert downstream() != null;
 
-        inLoop = true;
-        try {
-            if (state == State.UPDATING && waiting == 0)
-                F.first(sources).request(waiting = MODIFY_BATCH_SIZE);
+        if (state == State.UPDATING && waiting == 0)
+            source().request(waiting = MODIFY_BATCH_SIZE);
 
-            if (state == State.UPDATED && requested > 0) {
-                flush(true);
+        if (state == State.UPDATED && requested > 0) {
+            flushTuples(true);
 
-                state = State.END;
+            state = State.END;
 
+            inLoop = true;
+            try {
                 requested--;
-                downstream.push(ctx.rowHandler().factory(long.class).create(updatedRows));
+                downstream().push(context().rowHandler().factory(long.class).create(updatedRows));
             }
-
-            if (state == State.END && requested > 0) {
-                downstream.end();
-                requested = 0;
+            finally {
+                inLoop = false;
             }
         }
-        catch (Exception e) {
-            downstream.onError(e);
-        }
-        finally {
-            inLoop = false;
+
+        if (state == State.END && requested > 0) {
+            requested = 0;
+            downstream().end();
         }
     }
 
     /** */
     @SuppressWarnings("unchecked")
-    private void flush(boolean force) throws IgniteCheckedException {
+    private void flushTuples(boolean force) throws IgniteCheckedException {
         if (F.isEmpty(tuples) || !force && tuples.size() < MODIFY_BATCH_SIZE)
             return;
 
@@ -223,14 +213,13 @@ public class ModifyNode<Row> extends AbstractNode<Row> implements SingleNode<Row
         long updated = res.values().stream().mapToLong(EntryProcessorResult::get).sum();
 
         if (op == TableModify.Operation.INSERT && updated != res.size()) {
-            List<Object> duplicates = new ArrayList<>(res.size());
+            List<Object> duplicates = res.entrySet().stream()
+                .filter(e -> e.getValue().get() == 0)
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
 
-            for (Map.Entry<?, EntryProcessorResult<Long>> e : res.entrySet()) {
-                if (e.getValue().get() == 0)
-                    duplicates.add(e.getKey());
-            }
-
-            throw duplicateKeysException(duplicates);
+            throw new IgniteSQLException("Failed to INSERT some keys because they are already in cache. " +
+                "[keys=" + duplicates + ']', DUPLICATE_KEY);
         }
 
         updatedRows += updated;
@@ -264,12 +253,6 @@ public class ModifyNode<Row> extends AbstractNode<Row> implements SingleNode<Row
     }
 
     /** */
-    private IgniteSQLException duplicateKeysException(List<Object> keys) {
-        return new IgniteSQLException("Failed to INSERT some keys because they are already in cache [keys=" +
-            keys + ']', DUPLICATE_KEY);
-    }
-
-    /** */
     private enum State {
         /** */
         UPDATING,
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
new file mode 100644
index 0000000..c65dc68
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/NestedLoopJoinNode.java
@@ -0,0 +1,842 @@
+/*
+ * 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.ArrayList;
+import java.util.BitSet;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.IgniteCheckedException;
+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<Row> extends AbstractNode<Row> {
+    /** Special value to highlights that all row were received and we are not waiting any more. */
+    protected static final int NOT_WAITING = -1;
+
+    /** */
+    protected final Predicate<Row> cond;
+
+    /** */
+    protected final RowHandler<Row> handler;
+
+    /** */
+    protected int requested;
+
+    /** */
+    protected int waitingLeft;
+
+    /** */
+    protected int waitingRight;
+
+    /** */
+    protected final List<Row> rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE);
+
+    /** */
+    protected final Deque<Row> leftInBuf = new ArrayDeque<>(IN_BUFFER_SIZE);
+
+    /** */
+    protected boolean inLoop;
+
+    /**
+     * @param ctx Execution context.
+     * @param cond Join expression.
+     */
+    private NestedLoopJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond) {
+        super(ctx);
+
+        this.cond = cond;
+        handler = ctx.rowHandler();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) {
+        assert !F.isEmpty(sources()) && sources().size() == 2;
+        assert rowsCnt > 0 && requested == 0;
+
+        try {
+            checkState();
+
+            requested = rowsCnt;
+
+            if (!inLoop)
+                context().execute(this::doJoin);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void doJoin() {
+        try {
+            checkState();
+
+            join();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        requested = 0;
+        waitingLeft = 0;
+        waitingRight = 0;
+
+        rightMaterialized.clear();
+        leftInBuf.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        if (idx == 0)
+            return new Downstream<Row>() {
+                /** {@inheritDoc} */
+                @Override public void push(Row row) {
+                    pushLeft(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() {
+                    endLeft();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    NestedLoopJoinNode.this.onError(e);
+                }
+            };
+        else if (idx == 1)
+            return new Downstream<Row>() {
+                /** {@inheritDoc} */
+                @Override public void push(Row row) {
+                    pushRight(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() {
+                    endRight();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    NestedLoopJoinNode.this.onError(e);
+                }
+            };
+
+        throw new IndexOutOfBoundsException();
+    }
+
+    /** */
+    private void pushLeft(Row row) {
+        assert downstream() != null;
+        assert waitingLeft > 0;
+
+        try {
+            checkState();
+
+            waitingLeft--;
+
+            leftInBuf.add(row);
+
+            join();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void pushRight(Row row) {
+        assert downstream() != null;
+        assert waitingRight > 0;
+
+        try {
+            checkState();
+
+            waitingRight--;
+
+            rightMaterialized.add(row);
+
+            if (waitingRight == 0)
+                rightSource().request(waitingRight = IN_BUFFER_SIZE);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void endLeft() {
+        assert downstream() != null;
+        assert waitingLeft > 0;
+
+        try {
+            checkState();
+
+            waitingLeft = NOT_WAITING;
+
+            join();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void endRight() {
+        assert downstream() != null;
+        assert waitingRight > 0;
+
+        try {
+            checkState();
+
+            waitingRight = NOT_WAITING;
+
+            join();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    protected Node<Row> leftSource() {
+        return sources().get(0);
+    }
+
+    /** */
+    protected Node<Row> rightSource() {
+        return sources().get(1);
+    }
+
+    /** */
+    protected abstract void join() throws IgniteCheckedException;
+
+    /** */
+    @NotNull public static <Row> NestedLoopJoinNode<Row> create(ExecutionContext<Row> ctx, RelDataType leftRowType,
+        RelDataType rightRowType, JoinRelType joinType, Predicate<Row> cond) {
+        switch (joinType) {
+            case INNER:
+                return new InnerJoin<>(ctx, cond);
+
+            case LEFT: {
+                RowHandler.RowFactory<Row> rightRowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rightRowType);
+
+                return new LeftJoin<>(ctx, cond, rightRowFactory);
+            }
+
+            case RIGHT: {
+                RowHandler.RowFactory<Row> leftRowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), leftRowType);
+
+                return new RightJoin<>(ctx, cond, leftRowFactory);
+            }
+
+            case FULL: {
+                RowHandler.RowFactory<Row> leftRowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), leftRowType);
+                RowHandler.RowFactory<Row> rightRowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rightRowType);
+
+                return new FullOuterJoin<>(ctx, cond, leftRowFactory, rightRowFactory);
+            }
+
+            case SEMI:
+                return new SemiJoin<>(ctx, cond);
+
+            case ANTI:
+                return new AntiJoin<>(ctx, cond);
+
+            default:
+                throw new IllegalStateException("Join type \"" + joinType + "\" is not supported yet");
+        }
+    }
+
+    /** */
+    private static class InnerJoin<Row> extends NestedLoopJoinNode<Row> {
+        /** */
+        private Row left;
+
+        /** */
+        private int rightIdx;
+
+        /**
+         * @param ctx Execution context.
+         * @param cond Join expression.
+         */
+        public InnerJoin(ExecutionContext<Row> ctx, Predicate<Row> cond) {
+            super(ctx, cond);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void onRewind() {
+            left = null;
+            rightIdx = 0;
+
+            super.onRewind();
+        }
+
+        /** */
+        @Override protected void join() throws IgniteCheckedException {
+            if (waitingRight == NOT_WAITING) {
+                inLoop = true;
+                try {
+                    while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
+                        if (left == null)
+                            left = leftInBuf.remove();
+
+                        while (requested > 0 && rightIdx < rightMaterialized.size()) {
+                            checkState();
+
+                            Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
+
+                            if (!cond.test(row))
+                                continue;
+
+                            requested--;
+                            downstream().push(row);
+                        }
+
+                        if (rightIdx == rightMaterialized.size()) {
+                            left = null;
+                            rightIdx = 0;
+                        }
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            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 && left == null && leftInBuf.isEmpty()) {
+                requested = 0;
+                downstream().end();
+            }
+        }
+    }
+
+    /** */
+    private static class LeftJoin<Row> extends NestedLoopJoinNode<Row> {
+        /** Right row factory. */
+        private final RowHandler.RowFactory<Row> rightRowFactory;
+
+        /** Whether current left row was matched or not. */
+        private boolean matched;
+
+        /** */
+        private Row left;
+
+        /** */
+        private int rightIdx;
+
+        /**
+         * @param ctx Execution context.
+         * @param cond Join expression.
+         */
+        public LeftJoin(ExecutionContext<Row> ctx, Predicate<Row> cond, RowHandler.RowFactory<Row> rightRowFactory) {
+            super(ctx, cond);
+
+            this.rightRowFactory = rightRowFactory;
+        }
+
+        /** */
+        @Override protected void onRewind() {
+            matched = false;
+            left = null;
+            rightIdx = 0;
+
+            super.onRewind();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void join() throws IgniteCheckedException {
+            if (waitingRight == NOT_WAITING) {
+                inLoop = true;
+                try {
+                    while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
+                        if (left == null) {
+                            left = leftInBuf.remove();
+
+                            matched = false;
+                        }
+
+                        while (requested > 0 && rightIdx < rightMaterialized.size()) {
+                            checkState();
+
+                            Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
+
+                            if (!cond.test(row))
+                                continue;
+
+                            requested--;
+                            matched = true;
+                            downstream().push(row);
+                        }
+
+                        if (rightIdx == rightMaterialized.size()) {
+                            boolean wasPushed = false;
+
+                            if (!matched && requested > 0) {
+                                requested--;
+                                wasPushed = true;
+
+                                downstream().push(handler.concat(left, rightRowFactory.create()));
+                            }
+
+                            if (matched || wasPushed) {
+                                left = null;
+                                rightIdx = 0;
+                            }
+                        }
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            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 && left == null && leftInBuf.isEmpty()) {
+                requested = 0;
+                downstream().end();
+            }
+        }
+    }
+
+    /** */
+    private static class RightJoin<Row> extends NestedLoopJoinNode<Row> {
+        /** Right row factory. */
+        private final RowHandler.RowFactory<Row> leftRowFactory;
+
+        /** */
+        private BitSet rightNotMatchedIndexes;
+
+        /** */
+        private int lastPushedInd;
+
+        /** */
+        private Row left;
+
+        /** */
+        private int rightIdx;
+
+        /**
+         * @param ctx Execution context.
+         * @param cond Join expression.
+         */
+        public RightJoin(ExecutionContext<Row> ctx, Predicate<Row> cond, RowHandler.RowFactory<Row> leftRowFactory) {
+            super(ctx, cond);
+
+            this.leftRowFactory = leftRowFactory;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void onRewind() {
+            left = null;
+            rightNotMatchedIndexes.clear();
+            lastPushedInd = 0;
+            rightIdx = 0;
+
+            super.onRewind();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void join() throws IgniteCheckedException {
+            if (waitingRight == NOT_WAITING) {
+                if (rightNotMatchedIndexes == null) {
+                    rightNotMatchedIndexes = new BitSet(rightMaterialized.size());
+
+                    rightNotMatchedIndexes.set(0, rightMaterialized.size());
+                }
+
+                inLoop = true;
+                try {
+                    while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
+                        if (left == null)
+                            left = leftInBuf.remove();
+
+                        while (requested > 0 && rightIdx < rightMaterialized.size()) {
+                            checkState();
+
+                            Row right = rightMaterialized.get(rightIdx++);
+                            Row joined = handler.concat(left, right);
+
+                            if (!cond.test(joined))
+                                continue;
+
+                            requested--;
+                            rightNotMatchedIndexes.clear(rightIdx - 1);
+                            downstream().push(joined);
+                        }
+
+                        if (rightIdx == rightMaterialized.size()) {
+                            left = null;
+                            rightIdx = 0;
+                        }
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            if (waitingLeft == NOT_WAITING && requested > 0 && !rightNotMatchedIndexes.isEmpty()) {
+                assert lastPushedInd >= 0;
+
+                inLoop = true;
+                try {
+                    for (lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd);;
+                        lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd + 1)
+                    ) {
+                        checkState();
+
+                        if (lastPushedInd < 0)
+                            break;
+
+                        Row row = handler.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd));
+
+                        rightNotMatchedIndexes.clear(lastPushedInd);
+
+                        requested--;
+                        downstream().push(row);
+
+                        if (lastPushedInd == Integer.MAX_VALUE || requested <= 0)
+                            break;
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            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 && left == null
+                && leftInBuf.isEmpty() && rightNotMatchedIndexes.isEmpty()) {
+                requested = 0;
+                downstream().end();
+            }
+        }
+    }
+
+    /** */
+    private static class FullOuterJoin<Row> extends NestedLoopJoinNode<Row> {
+        /** Left row factory. */
+        private final RowHandler.RowFactory<Row> leftRowFactory;
+
+        /** Right row factory. */
+        private final RowHandler.RowFactory<Row> rightRowFactory;
+
+        /** Whether current left row was matched or not. */
+        private boolean leftMatched;
+
+        /** */
+        private BitSet rightNotMatchedIndexes;
+
+        /** */
+        private int lastPushedInd;
+
+        /** */
+        private Row left;
+
+        /** */
+        private int rightIdx;
+
+        /**
+         * @param ctx Execution context.
+         * @param cond Join expression.
+         */
+        public FullOuterJoin(ExecutionContext<Row> ctx, Predicate<Row> cond, RowHandler.RowFactory<Row> leftRowFactory,
+            RowHandler.RowFactory<Row> rightRowFactory) {
+            super(ctx, cond);
+
+            this.leftRowFactory = leftRowFactory;
+            this.rightRowFactory = rightRowFactory;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void onRewind() {
+            left = null;
+            leftMatched = false;
+            rightNotMatchedIndexes.clear();
+            lastPushedInd = 0;
+            rightIdx = 0;
+
+            super.onRewind();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void join() throws IgniteCheckedException {
+            if (waitingRight == NOT_WAITING) {
+                if (rightNotMatchedIndexes == null) {
+                    rightNotMatchedIndexes = new BitSet(rightMaterialized.size());
+
+                    rightNotMatchedIndexes.set(0, rightMaterialized.size());
+                }
+
+                inLoop = true;
+                try {
+                    while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
+                        if (left == null) {
+                            left = leftInBuf.remove();
+
+                            leftMatched = false;
+                        }
+
+                        while (requested > 0 && rightIdx < rightMaterialized.size()) {
+                            checkState();
+
+                            Row right = rightMaterialized.get(rightIdx++);
+                            Row joined = handler.concat(left, right);
+
+                            if (!cond.test(joined))
+                                continue;
+
+                            requested--;
+                            leftMatched = true;
+                            rightNotMatchedIndexes.clear(rightIdx - 1);
+                            downstream().push(joined);
+                        }
+
+                        if (rightIdx == rightMaterialized.size()) {
+                            boolean wasPushed = false;
+
+                            if (!leftMatched && requested > 0) {
+                                requested--;
+                                wasPushed = true;
+
+                                downstream().push(handler.concat(left, rightRowFactory.create()));
+                            }
+
+                            if (leftMatched || wasPushed) {
+                                left = null;
+                                rightIdx = 0;
+                            }
+                        }
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            if (waitingLeft == NOT_WAITING && requested > 0 && !rightNotMatchedIndexes.isEmpty()) {
+                assert lastPushedInd >= 0;
+
+                inLoop = true;
+                try {
+                    for (lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd);;
+                        lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd + 1)
+                    ) {
+                        checkState();
+
+                        if (lastPushedInd < 0)
+                            break;
+
+                        Row row = handler.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd));
+
+                        rightNotMatchedIndexes.clear(lastPushedInd);
+
+                        requested--;
+                        downstream().push(row);
+
+                        if (lastPushedInd == Integer.MAX_VALUE || requested <= 0)
+                            break;
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            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 && left == null
+                && leftInBuf.isEmpty() && rightNotMatchedIndexes.isEmpty()) {
+                requested = 0;
+                downstream().end();
+            }
+        }
+    }
+
+    /** */
+    private static class SemiJoin<Row> extends NestedLoopJoinNode<Row> {
+        /** */
+        private Row left;
+
+        /** */
+        private int rightIdx;
+
+        /**
+         * @param ctx Execution context.
+         * @param cond Join expression.
+         */
+        public SemiJoin(ExecutionContext<Row> ctx, Predicate<Row> cond) {
+            super(ctx, cond);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void onRewind() {
+            left = null;
+            rightIdx = 0;
+
+            super.onRewind();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void join() throws IgniteCheckedException {
+            if (waitingRight == NOT_WAITING) {
+                while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
+                    if (left == null)
+                        left = leftInBuf.remove();
+
+                    boolean matched = false;
+
+                    while (!matched && requested > 0 && rightIdx < rightMaterialized.size()) {
+                        checkState();
+
+                        Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
+
+                        if (!cond.test(row))
+                            continue;
+
+                        requested--;
+                        downstream().push(left);
+
+                        matched = true;
+                    }
+
+                    if (matched || rightIdx == rightMaterialized.size()) {
+                        left = null;
+                        rightIdx = 0;
+                    }
+                }
+            }
+
+            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 && left == null
+                && leftInBuf.isEmpty()) {
+                downstream().end();
+                requested = 0;
+            }
+        }
+    }
+
+    /** */
+    private static class AntiJoin<Row> extends NestedLoopJoinNode<Row> {
+        /** */
+        private Row left;
+
+        /** */
+        private int rightIdx;
+
+        /**
+         * @param ctx Execution context.
+         * @param cond Join expression.
+         */
+        public AntiJoin(ExecutionContext<Row> ctx, Predicate<Row> cond) {
+            super(ctx, cond);
+        }
+
+        /** */
+        @Override protected void onRewind() {
+            left = null;
+            rightIdx = 0;
+
+            super.onRewind();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void join() throws IgniteCheckedException {
+            if (waitingRight == NOT_WAITING) {
+                inLoop = true;
+                try {
+                    while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
+                        if (left == null)
+                            left = leftInBuf.remove();
+
+                        boolean matched = false;
+
+                        while (!matched && rightIdx < rightMaterialized.size()) {
+                            checkState();
+
+                            Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
+
+                            if (cond.test(row))
+                                matched = true;
+                        }
+
+                        if (!matched) {
+                            requested--;
+                            downstream().push(left);
+                        }
+
+                        left = null;
+                        rightIdx = 0;
+                    }
+                }
+                finally {
+                    inLoop = false;
+                }
+            }
+
+            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 && left == null && leftInBuf.isEmpty()) {
+                requested = 0;
+                downstream().end();
+            }
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java
index 6c99f4d..fb1c1c1 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java
@@ -36,6 +36,11 @@ public interface Node<Row> extends AutoCloseable {
     ExecutionContext<Row> context();
 
     /**
+     * @return Node downstream.
+     */
+    Downstream<Row> downstream();
+
+    /**
      * Registers node sources.
      *
      * @param sources Sources collection.
@@ -43,6 +48,13 @@ public interface Node<Row> extends AutoCloseable {
     void register(List<Node<Row>> sources);
 
     /**
+     * Returns registered node sources.
+     *
+     * @return Node sources.
+     */
+    List<Node<Row>> sources();
+
+    /**
      * Registers downstream.
      *
      * @param downstream Downstream.
@@ -53,4 +65,10 @@ public interface Node<Row> extends AutoCloseable {
      * Requests next bunch of rows.
      */
     void request(int rowsCnt);
+
+
+    /**
+     * Rewinds upstream.
+     */
+    void rewind();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
index 199f528..25a4673 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.stream.Collectors;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
@@ -52,7 +53,7 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
     private final long targetFragmentId;
 
     /** */
-    private final Destination dest;
+    private final Destination<Row> dest;
 
     /** */
     private final Deque<Row> inBuf = new ArrayDeque<>(IN_BUFFER_SIZE);
@@ -77,7 +78,7 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
         MailboxRegistry registry,
         long exchangeId,
         long targetFragmentId,
-        Destination dest
+        Destination<Row> dest
     ) {
         super(ctx);
         this.exchange = exchange;
@@ -99,48 +100,62 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
      * @param batchId Batch ID.
      */
     public void onAcknowledge(UUID nodeId, int batchId) {
-        Buffer buffer = nodeBuffers.get(nodeId);
+        assert nodeBuffers.containsKey(nodeId);
 
-        assert buffer != null;
+        try {
+            checkState();
 
-        buffer.onAcknowledge(batchId);
+            nodeBuffers.get(nodeId).acknowledge(batchId);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** */
     public void init() {
-        checkThread();
+        try {
+            checkState();
 
-        flushFromBuffer();
+            flush();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void push(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
+    @Override public void request(int rowCnt) {
+        throw new UnsupportedOperationException();
+    }
 
+    /** {@inheritDoc} */
+    @Override public void push(Row row) {
         assert waiting > 0;
 
-        waiting--;
+        try {
+            checkState();
+
+            waiting--;
 
-        inBuf.add(row);
+            inBuf.add(row);
 
-        flushFromBuffer();
+            flush();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
         assert waiting > 0;
 
-        waiting = -1;
-
         try {
+            checkState();
+
+            waiting = -1;
+
             for (UUID node : dest.targets())
                 getOrCreateBuffer(node).end();
         }
@@ -161,31 +176,29 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
             U.error(context().planningContext().logger(),
                 "Error occurred during send error message: " + X.getFullStackTrace(e));
         }
-
-        close();
+        finally {
+            U.closeQuiet(this);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void close() {
-        if (isClosed())
-            return;
+    @Override public void onClose() {
+        super.onClose();
 
         registry.unregister(this);
 
         // Send cancel message for the Inbox to close Inboxes created by batch message race.
         for (UUID node : dest.targets())
             getOrCreateBuffer(node).close();
-
-        super.close();
     }
 
     /** {@inheritDoc} */
-    @Override public void request(int rowCnt) {
+    @Override public void onRegister(Downstream<Row> downstream) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public void onRegister(Downstream<Row> downstream) {
+    @Override protected void onRewind() {
         throw new UnsupportedOperationException();
     }
 
@@ -204,7 +217,7 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
 
     /** */
     private void sendError(Throwable err) throws IgniteCheckedException {
-        exchange.sendError(ctx.originatingNodeId(), queryId(), fragmentId(), err);
+        exchange.sendError(context().originatingNodeId(), queryId(), fragmentId(), err);
     }
 
     /** */
@@ -228,45 +241,35 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
     }
 
     /** */
-    private void flushFromBuffer() {
-        try {
-            while (!inBuf.isEmpty()) {
-                Row row = inBuf.remove();
-
-                List<UUID> nodes = dest.targets(row);
+    private void flush() throws IgniteCheckedException {
+        while (!inBuf.isEmpty()) {
+            checkState();
 
-                assert !F.isEmpty(nodes);
+            Collection<Buffer> buffers = dest.targets(inBuf.peek()).stream()
+                .map(this::getOrCreateBuffer)
+                .collect(Collectors.toList());
 
-                Collection<Buffer> buffers = new ArrayList<>(nodes.size());
+            assert !F.isEmpty(buffers);
 
-                for (UUID node : nodes) {
-                    Buffer dest = getOrCreateBuffer(node);
+            if (!buffers.stream().allMatch(Buffer::ready))
+                return;
 
-                    if (dest.ready())
-                        buffers.add(dest);
-                    else {
-                        inBuf.addFirst(row);
+            Row row = inBuf.remove();
 
-                        return;
-                    }
-                }
+            for (Buffer dest : buffers)
+                dest.add(row);
+        }
 
-                for (Buffer dest : buffers)
-                    dest.add(row);
-            }
+        assert inBuf.isEmpty();
 
-            if (waiting == 0)
-                F.first(sources).request(waiting = IN_BUFFER_SIZE);
-        }
-        catch (Exception e) {
-            onError(e);
-        }
+        if (waiting == 0)
+            source().request(waiting = IN_BUFFER_SIZE);
     }
 
     /** */
     public void onNodeLeft(UUID nodeId) {
-        if (nodeId.equals(ctx.originatingNodeId()))
-            ctx.execute(this::close);
+        if (nodeId.equals(context().originatingNodeId()))
+            context().execute(this::close);
     }
 
     /** */
@@ -287,7 +290,19 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
         private Buffer(UUID nodeId) {
             this.nodeId = nodeId;
 
-            curr = new ArrayList<>(IO_BATCH_SIZE); // extra space for end marker;
+            curr = new ArrayList<>(IO_BATCH_SIZE);
+        }
+
+        /**
+         * Checks whether there is a place for a new row.
+         *
+         * @return {@code True} is it possible to add a row to a batch.
+         */
+        private boolean ready() {
+            if (hwm == Integer.MAX_VALUE)
+                return false;
+
+            return curr.size() < IO_BATCH_SIZE || hwm - lwm < IO_BATCH_CNT;
         }
 
         /**
@@ -301,7 +316,7 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
             if (curr.size() == IO_BATCH_SIZE) {
                 sendBatch(nodeId, ++hwm, false, curr);
 
-                curr = new ArrayList<>(IO_BATCH_SIZE); // extra space for end marker;
+                curr = new ArrayList<>(IO_BATCH_SIZE);
             }
 
             curr.add(row);
@@ -323,39 +338,12 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
             sendBatch(nodeId, batchId, true, tmp);
         }
 
-        /** */
-        public void close() {
-            int currBatchId = hwm;
-
-            if (hwm == Integer.MAX_VALUE)
-                return;
-
-            hwm = Integer.MAX_VALUE;
-
-            curr = null;
-
-            if (currBatchId >= 0)
-                sendInboxClose(nodeId);
-        }
-
-        /**
-         * Checks whether there is a place for a new row.
-         *
-         * @return {@code True} is it possible to add a row to a batch.
-         */
-        private boolean ready() {
-            if (hwm == Integer.MAX_VALUE)
-                return false;
-
-            return curr.size() < IO_BATCH_SIZE || hwm - lwm < IO_BATCH_CNT;
-        }
-
         /**
          * Callback method.
          *
          * @param id batch ID.
          */
-        private void onAcknowledge(int id) {
+        private void acknowledge(int id) throws IgniteCheckedException {
             if (lwm > id)
                 return;
 
@@ -364,7 +352,22 @@ public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, Sing
             lwm = id;
 
             if (!readyBefore && ready())
-                flushFromBuffer();
+                flush();
+        }
+
+        /** */
+        public void close() {
+            int currBatchId = hwm;
+
+            if (hwm == Integer.MAX_VALUE)
+                return;
+
+            hwm = Integer.MAX_VALUE;
+
+            curr = null;
+
+            if (currBatchId >= 0)
+                sendInboxClose(nodeId);
         }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java
index afe7fc0..5f3205e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java
@@ -40,54 +40,51 @@ public class ProjectNode<Row> extends AbstractNode<Row> implements SingleNode<Ro
     }
 
     /** {@inheritDoc} */
-    @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
+    @Override protected void onRewind() {
+        // No-op.
+    }
 
-        assert !F.isEmpty(sources) && sources.size() == 1;
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) {
+        assert !F.isEmpty(sources()) && sources().size() == 1;
         assert rowsCnt > 0;
 
-        F.first(sources).request(rowsCnt);
+        try {
+            checkState();
+
+            source().request(rowsCnt);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void push(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
 
         try {
-            downstream.push(prj.apply(row));
+            checkState();
+
+            downstream().push(prj.apply(row));
         }
         catch (Throwable e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
-
-        downstream.end();
-    }
+        assert downstream() != null;
 
-    /** {@inheritDoc} */
-    @Override public void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
+        try {
+            checkState();
 
-        downstream.onError(e);
+            downstream().end();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RightJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RightJoinNode.java
deleted file mode 100644
index 8b175f7..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RightJoinNode.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.BitSet;
-import java.util.function.Predicate;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
-
-/** */
-public class RightJoinNode<Row> extends AbstractJoinNode<Row> {
-    /** Right row factory. */
-    private final RowHandler.RowFactory<Row> leftRowFactory;
-
-    /** */
-    private BitSet rightNotMatchedIndexes;
-
-    /** */
-    private int lastPushedInd;
-
-    /** */
-    private Row left;
-
-    /** */
-    private int rightIdx;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    public RightJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond, RowHandler.RowFactory<Row> leftRowFactory) {
-        super(ctx, cond);
-
-        this.leftRowFactory = leftRowFactory;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void doJoin() {
-        if (waitingRight == NOT_WAITING) {
-            if (rightNotMatchedIndexes == null) {
-                rightNotMatchedIndexes = new BitSet(rightMaterialized.size());
-
-                rightNotMatchedIndexes.set(0, rightMaterialized.size());
-            }
-
-            while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
-                if (left == null)
-                    left = leftInBuf.remove();
-
-                while (requested > 0 && rightIdx < rightMaterialized.size()) {
-                    Row right = rightMaterialized.get(rightIdx++);
-                    Row joined = handler.concat(left, right);
-
-                    if (!cond.test(joined))
-                        continue;
-
-                    requested--;
-                    rightNotMatchedIndexes.clear(rightIdx - 1);
-                    downstream.push(joined);
-                }
-
-                if (rightIdx == rightMaterialized.size()) {
-                    left = null;
-                    rightIdx = 0;
-                }
-            }
-        }
-
-        if (waitingLeft == NOT_WAITING && requested > 0 && !rightNotMatchedIndexes.isEmpty()) {
-            assert lastPushedInd >= 0;
-
-            for (lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd);;
-                lastPushedInd = rightNotMatchedIndexes.nextSetBit(lastPushedInd + 1)
-            ) {
-                if (lastPushedInd < 0)
-                    break;
-
-                Row row = handler.concat(leftRowFactory.create(), rightMaterialized.get(lastPushedInd));
-
-                rightNotMatchedIndexes.clear(lastPushedInd);
-
-                requested--;
-                downstream.push(row);
-
-                if (lastPushedInd == Integer.MAX_VALUE || requested <= 0)
-                    break;
-            }
-        }
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-
-        if (waitingLeft == 0 && leftInBuf.isEmpty())
-            sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
-
-        if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null
-            && leftInBuf.isEmpty() && rightNotMatchedIndexes.isEmpty()) {
-            downstream.end();
-            requested = 0;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
index 68da4fd..825fe42 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  * Client iterator.
@@ -64,13 +65,7 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
      * @param ctx Execution context.
      */
     public RootNode(ExecutionContext<Row> ctx) {
-        super(ctx);
-
-        buff = new ArrayDeque<>(IN_BUFFER_SIZE);
-        lock = new ReentrantLock();
-        cond = lock.newCondition();
-
-        onClose = this::proceedClose;
+        this(ctx, null);
     }
 
     /**
@@ -91,20 +86,6 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
         return context().queryId();
     }
 
-    /** */
-    public void proceedClose() {
-        context().execute(() -> {
-            checkThread();
-
-            if (isClosed())
-                return;
-
-            buff.clear();
-
-            super.close();
-        });
-    }
-
     /** {@inheritDoc} */
     @Override public void close() {
         lock.lock();
@@ -122,54 +103,83 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
             lock.unlock();
         }
 
-        onClose.run();
+        if (onClose == null)
+            onClose();
+        else
+            onClose.run();
     }
 
     /** {@inheritDoc} */
-    @Override public void push(Row row) {
-        checkThread();
+    @Override protected boolean isClosed() {
+        return state == State.CANCELLED || state == State.CLOSED;
+    }
 
-        int req = 0;
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        context().execute(() -> {
+            buff.clear();
+
+            U.closeQuiet(super::close);
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void push(Row row) {
+        assert waiting > 0;
 
-        lock.lock();
         try {
-            assert waiting > 0 : "waiting=" + waiting;
+            checkState();
 
-            waiting--;
+            int req = 0;
 
-            if (state != State.RUNNING)
-                return;
+            lock.lock();
+            try {
+                if (state != State.RUNNING)
+                    return;
 
-            buff.offer(row);
+                waiting--;
 
-            if (waiting == 0)
-                waiting = req = IN_BUFFER_SIZE - buff.size();
+                buff.offer(row);
 
-            cond.signalAll();
+                if (waiting == 0)
+                    waiting = req = IN_BUFFER_SIZE - buff.size();
+
+                cond.signalAll();
+            }
+            finally {
+                lock.unlock();
+            }
+
+            if (req > 0)
+                source().request(req);
         }
-        finally {
-            lock.unlock();
+        catch (Exception e) {
+            onError(e);
         }
-
-        if (req > 0)
-            F.first(sources).request(req);
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        lock.lock();
         try {
-            assert waiting > 0 : "waiting=" + waiting;
+            checkState();
 
-            waiting = -1;
+            lock.lock();
+            try {
+                assert waiting > 0 : "waiting=" + waiting;
 
-            if (state != State.RUNNING)
-                return;
+                waiting = -1;
 
-            cond.signalAll();
+                if (state != State.RUNNING)
+                    return;
+
+                cond.signalAll();
+            }
+            finally {
+                lock.unlock();
+            }
         }
-        finally {
-            lock.unlock();
+        catch (Exception e) {
+            onError(e);
         }
     }
 
@@ -178,7 +188,7 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
         if (!ex.compareAndSet(null, e))
             ex.get().addSuppressed(e);
 
-        close();
+        U.closeQuiet(this);
     }
 
     /** {@inheritDoc} */
@@ -216,13 +226,18 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
     }
 
     /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
         throw new UnsupportedOperationException();
     }
 
     /** */
     private Row take() {
-        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert !F.isEmpty(sources()) && sources().size() == 1;
 
         lock.lock();
         try {
@@ -236,7 +251,7 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
                     break;
                 else if (waiting == 0) {
                     int req = waiting = IN_BUFFER_SIZE;
-                    context().execute(() -> F.first(sources).request(req));
+                    context().execute(() -> source().request(req));
                 }
 
                 cond.await();
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java
index fae9530..3e82b63 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
@@ -52,31 +52,46 @@ public class ScanNode<Row> extends AbstractNode<Row> implements SingleNode<Row>
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
+        assert rowsCnt > 0 && requested == 0;
 
-        if (isClosed())
-            return;
+        try {
+            checkState();
 
-        assert rowsCnt > 0 && requested == 0;
+            requested = rowsCnt;
 
-        requested = rowsCnt;
+            if (!inLoop)
+                context().execute(this::doPush);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
 
-        if (!inLoop)
-            context().execute(this::pushInternal);
+    /** */
+    private void doPush() {
+        try {
+            checkState();
+
+            push();
+        }
+        catch (Exception e) {
+            e.printStackTrace();
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void close() {
-        if (isClosed())
-            return;
+    @Override public void onClose() {
+        super.onClose();
 
         Commons.closeQuiet(it);
-
         it = null;
-
         Commons.closeQuiet(src);
+    }
 
-        super.close();
+    /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        Commons.closeQuiet(it);
+        it = null;
     }
 
     /** {@inheritDoc} */
@@ -90,62 +105,37 @@ public class ScanNode<Row> extends AbstractNode<Row> implements SingleNode<Row>
     }
 
     /** */
-    private void pushInternal() {
-        if (isClosed())
-            return;
-
+    private void push() throws IgniteCheckedException {
         inLoop = true;
         try {
             if (it == null)
                 it = src.iterator();
 
             int processed = 0;
-
-            Thread thread = Thread.currentThread();
-
             while (requested > 0 && it.hasNext()) {
-                if (isClosed())
-                    return;
-
-                if (thread.isInterrupted())
-                    throw new IgniteInterruptedCheckedException("Thread was interrupted.");
+                checkState();
 
                 requested--;
-                downstream.push(it.next());
+                downstream().push(it.next());
 
                 if (++processed == IN_BUFFER_SIZE && requested > 0) {
                     // allow others to do their job
-                    context().execute(this::pushInternal);
+                    context().execute(this::doPush);
 
                     return;
                 }
             }
-
-            if (requested > 0 && !it.hasNext()) {
-                downstream.end();
-                requested = 0;
-
-                Commons.closeQuiet(it);
-
-                it = null;
-            }
-        }
-        catch (Throwable e) {
-            onError(e);
         }
         finally {
             inLoop = false;
         }
-    }
 
-    /** */
-    private void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
+        if (requested > 0 && !it.hasNext()) {
+            Commons.closeQuiet(it);
+            it = null;
 
-        downstream.onError(e);
-
-        close();
+            requested = 0;
+            downstream().end();
+        }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SemiJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SemiJoinNode.java
deleted file mode 100644
index 38f65f5..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SemiJoinNode.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.function.Predicate;
-
-import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-
-/** */
-public class SemiJoinNode<Row> extends AbstractJoinNode<Row> {
-    /** */
-    private Row left;
-
-    /** */
-    private int rightIdx;
-
-    /**
-     * @param ctx Execution context.
-     * @param cond Join expression.
-     */
-    public SemiJoinNode(ExecutionContext<Row> ctx, Predicate<Row> cond) {
-        super(ctx, cond);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void doJoin() {
-        if (waitingRight == NOT_WAITING) {
-            while (requested > 0 && (left != null || !leftInBuf.isEmpty())) {
-                if (left == null)
-                    left = leftInBuf.remove();
-
-                boolean matched = false;
-
-                while (!matched && requested > 0 && rightIdx < rightMaterialized.size()) {
-                    Row row = handler.concat(left, rightMaterialized.get(rightIdx++));
-
-                    if (!cond.test(row))
-                        continue;
-
-                    requested--;
-                    matched = true;
-                    downstream.push(left);
-                }
-
-                if (matched || rightIdx == rightMaterialized.size()) {
-                    left = null;
-                    rightIdx = 0;
-                }
-            }
-        }
-
-        if (waitingRight == 0)
-            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
-
-        if (waitingLeft == 0 && leftInBuf.isEmpty())
-            sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
-
-        if (requested > 0 && waitingLeft == NOT_WAITING && waitingRight == NOT_WAITING && left == null
-            && leftInBuf.isEmpty()) {
-            downstream.end();
-            requested = 0;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
index 3b96914..5887e8e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
@@ -18,13 +18,19 @@
 package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.NotNull;
 
 /**
  * A node with a single input
  */
 public interface SingleNode<Row> extends Node<Row> {
     /** */
-    default void register(Node<Row> src) {
+    default void register(@NotNull Node<Row> src) {
         register(F.asList(src));
     }
+
+    /** */
+    default @NotNull Node<Row> source() {
+        return F.first(sources());
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SortNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SortNode.java
index 64c236d..5c686fc 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SortNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SortNode.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 import java.util.Comparator;
 import java.util.PriorityQueue;
 
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.util.typedef.F;
 
@@ -49,6 +50,13 @@ public class SortNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
     }
 
     /** {@inheritDoc} */
+    @Override protected void onRewind() {
+        requested = 0;
+        waiting = 0;
+        rows.clear();
+    }
+
+    /** {@inheritDoc} */
     @Override protected Downstream<Row> requestDownstream(int idx) {
         if (idx != 0)
             throw new IndexOutOfBoundsException();
@@ -58,77 +66,74 @@ public class SortNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert !F.isEmpty(sources()) && sources().size() == 1;
         assert rowsCnt > 0 && requested == 0;
         assert waiting <= 0;
 
-        requested = rowsCnt;
+        try {
+            checkState();
+
+            requested = rowsCnt;
 
-        if (waiting == 0)
-            F.first(sources).request(waiting = IN_BUFFER_SIZE);
-        else if (!inLoop)
-            context().execute(this::flushFromBuffer);
+            if (waiting == 0)
+                source().request(waiting = IN_BUFFER_SIZE);
+            else if (!inLoop)
+                context().execute(this::doFlush);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
+    private void doFlush() {
+        try {
+            flush();
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void push(Row row) {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting--;
-
         try {
+            checkState();
+
+            waiting--;
+
             rows.add(row);
 
             if (waiting == 0)
-                F.first(sources).request(waiting = IN_BUFFER_SIZE);
+                source().request(waiting = IN_BUFFER_SIZE);
         }
         catch (Exception e) {
-            downstream.onError(e);
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        if (isClosed())
-            return;
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting = -1;
-
         try {
-            flushFromBuffer();
+            checkState();
+
+            waiting = -1;
+
+            flush();
         }
         catch (Exception e) {
-            downstream.onError(e);
+            downstream().onError(e);
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
-
-        downstream.onError(e);
-    }
-
     /** */
-    private void flushFromBuffer() {
+    private void flush() throws IgniteCheckedException {
         assert waiting == -1;
 
         int processed = 0;
@@ -136,20 +141,22 @@ public class SortNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
         inLoop = true;
         try {
             while (requested > 0 && !rows.isEmpty()) {
+                checkState();
+
                 requested--;
 
-                downstream.push(rows.poll());
+                downstream().push(rows.poll());
 
                 if (++processed >= IN_BUFFER_SIZE && requested > 0) {
                     // allow others to do their job
-                    context().execute(this::flushFromBuffer);
+                    context().execute(this::doFlush);
 
                     return;
                 }
             }
 
             if (requested >= 0) {
-                downstream.end();
+                downstream().end();
                 requested = 0;
             }
         }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/UnionAllNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/UnionAllNode.java
index d9e4f1b..9935b68 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/UnionAllNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/UnionAllNode.java
@@ -39,60 +39,72 @@ public class UnionAllNode<Row> extends AbstractNode<Row> implements Downstream<R
 
     /** {@inheritDoc} */
     @Override protected Downstream<Row> requestDownstream(int idx) {
-        assert sources != null;
-        assert idx >= 0 && idx < sources.size();
+        assert sources() != null;
+        assert idx >= 0 && idx < sources().size();
 
         return this;
     }
 
     /** {@inheritDoc} */
     @Override public void request(int rowsCnt) {
-        checkThread();
-
-        assert !F.isEmpty(sources);
+        assert !F.isEmpty(sources());
         assert rowsCnt > 0 && waiting == 0;
 
-        source().request(waiting = rowsCnt);
+        try {
+            checkState();
+
+            source().request(waiting = rowsCnt);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void push(Row row) {
-        checkThread();
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        waiting--;
+        try {
+            checkState();
+
+            waiting--;
 
-        downstream.push(row);
+            downstream().push(row);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        checkThread();
-
-        assert downstream != null;
+        assert downstream() != null;
         assert waiting > 0;
 
-        if (++curSrc < sources.size())
-            source().request(waiting);
-        else {
-            waiting = -1;
-            downstream.end();
+        try {
+            checkState();
+
+            if (++curSrc < sources().size())
+                source().request(waiting);
+            else {
+                waiting = -1;
+                downstream().end();
+            }
+        }
+        catch (Exception e) {
+            onError(e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public void onError(Throwable e) {
-        checkThread();
-
-        assert downstream != null;
-
-        downstream.onError(e);
+    @Override protected void onRewind() {
+        curSrc = 0;
+        waiting = 0;
     }
 
     /** */
     private Node<Row> source() {
-        return sources.get(curSrc);
+        return sources().get(curSrc);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdCumulativeCost.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdCumulativeCost.java
new file mode 100644
index 0000000..82fdd3f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdCumulativeCost.java
@@ -0,0 +1,81 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.plan.volcano.VolcanoUtils;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.any;
+import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.distribution;
+
+/** */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
+public class IgniteMdCumulativeCost implements MetadataHandler<BuiltInMetadata.CumulativeCost> {
+    /** */
+    public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(
+        BuiltInMethod.CUMULATIVE_COST.method, new IgniteMdCumulativeCost());
+
+    /** {@inheritDoc} */
+    @Override public MetadataDef<BuiltInMetadata.CumulativeCost> getDef() {
+        return BuiltInMetadata.CumulativeCost.DEF;
+    }
+
+    /** */
+    public RelOptCost getCumulativeCost(RelSubset rel, RelMetadataQuery mq) {
+        return VolcanoUtils.bestCost(rel);
+    }
+
+    /** */
+    public RelOptCost getCumulativeCost(RelNode rel, RelMetadataQuery mq) {
+        RelOptCost cost = nonCumulativeCost(rel, mq);
+
+        if (cost.isInfinite())
+            return cost;
+
+        for (RelNode input : rel.getInputs())
+            cost = cost.plus(mq.getCumulativeCost(input));
+
+        return cost;
+    }
+
+    /** */
+    private static RelOptCost nonCumulativeCost(RelNode rel, RelMetadataQuery mq) {
+        RelOptCost cost = mq.getNonCumulativeCost(rel);
+
+        if (cost.isInfinite())
+            return cost;
+
+        RelOptCostFactory costFactory = rel.getCluster().getPlanner().getCostFactory();
+
+        if (rel.getConvention() == Convention.NONE || distribution(rel) == any())
+            return costFactory.makeInfiniteCost();
+
+        return costFactory.makeZeroCost().isLt(cost) ? cost : costFactory.makeTinyCost();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
index e4f2c0f..c68ab89 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 /**
  * Implementation class for {@link RelMetadataQuery#distribution(RelNode)} method call.
  */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
 public class IgniteMdDistribution implements MetadataHandler<BuiltInMetadata.Distribution> {
     /**
      * Metadata provider, responsible for distribution type request. It uses this implementation class under the hood.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNodesMapping.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNodesMapping.java
index 00c6f14..692c1a7 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNodesMapping.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNodesMapping.java
@@ -31,8 +31,8 @@ import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.NodesMappingMetadata;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
 import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
 /**
  * Implementation class for {@link RelMetadataQueryEx#nodesMapping(RelNode)} method call.
  */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
 public class IgniteMdNodesMapping implements MetadataHandler<NodesMappingMetadata> {
     /**
      * Metadata provider, responsible for nodes mapping request. It uses this implementation class under the hood.
@@ -162,7 +163,7 @@ public class IgniteMdNodesMapping implements MetadataHandler<NodesMappingMetadat
     /**
      * See {@link IgniteMdNodesMapping#nodesMapping(RelNode, RelMetadataQuery)}
      */
-    public NodesMapping nodesMapping(IgniteTableScan rel, RelMetadataQuery mq) {
+    public NodesMapping nodesMapping(IgniteIndexScan rel, RelMetadataQuery mq) {
         return rel.getTable().unwrap(IgniteTable.class).mapping(Commons.context(rel));
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNonCumulativeCost.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNonCumulativeCost.java
new file mode 100644
index 0000000..004e89b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdNonCumulativeCost.java
@@ -0,0 +1,46 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+
+/** */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
+public class IgniteMdNonCumulativeCost implements MetadataHandler<BuiltInMetadata.NonCumulativeCost> {
+    /** */
+    public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(
+        BuiltInMethod.NON_CUMULATIVE_COST.method, new IgniteMdNonCumulativeCost());
+
+    /** */
+    @Override public MetadataDef<BuiltInMetadata.NonCumulativeCost> getDef() {
+        return BuiltInMetadata.NonCumulativeCost.DEF;
+    }
+
+    /** */
+    public RelOptCost getNonCumulativeCost(RelNode rel, RelMetadataQuery mq) {
+        return rel.computeSelfCost(rel.getCluster().getPlanner(), mq);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdPercentageOriginalRows.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdPercentageOriginalRows.java
new file mode 100644
index 0000000..bbcdbdb
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdPercentageOriginalRows.java
@@ -0,0 +1,161 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+
+/**
+ * See {@link org.apache.calcite.rel.metadata.RelMdPercentageOriginalRows}
+ */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
+public class IgniteMdPercentageOriginalRows implements MetadataHandler<BuiltInMetadata.PercentageOriginalRows> {
+    /** */
+    public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(
+        BuiltInMethod.PERCENTAGE_ORIGINAL_ROWS.method, new IgniteMdPercentageOriginalRows());
+
+    /** {@inheritDoc} */
+    @Override public MetadataDef<BuiltInMetadata.PercentageOriginalRows> getDef() {
+        return BuiltInMetadata.PercentageOriginalRows.DEF;
+    }
+
+    /** */
+    public Double getPercentageOriginalRows(Aggregate rel, RelMetadataQuery mq) {
+        // REVIEW jvs 28-Mar-2006: The assumption here seems to be that
+        // aggregation does not apply any filtering, so it does not modify the
+        // percentage.  That's very much oversimplified.
+        return mq.getPercentageOriginalRows(rel.getInput());
+    }
+
+    /** */
+    public Double getPercentageOriginalRows(Union rel, RelMetadataQuery mq) {
+        double numerator = 0.0;
+        double denominator = 0.0;
+
+        // Ignore rel.isDistinct() because it's the same as an aggregate.
+
+        // REVIEW jvs 28-Mar-2006: The original Broadbase formula was broken.
+        // It was multiplying percentage into the numerator term rather than
+        // than dividing it out of the denominator term, which would be OK if
+        // there weren't summation going on.  Probably the cause of the error
+        // was the desire to avoid division by zero, which I don't know how to
+        // handle so I punt, meaning we return a totally wrong answer in the
+        // case where a huge table has been completely filtered away.
+
+        for (RelNode input : rel.getInputs()) {
+            Double rowCount = mq.getRowCount(input);
+            if (rowCount == null) {
+                continue;
+            }
+            Double percentage = mq.getPercentageOriginalRows(input);
+            if (percentage == null) {
+                continue;
+            }
+            if (percentage != 0.0) {
+                denominator += rowCount / percentage;
+                numerator += rowCount;
+            }
+        }
+
+        return quotientForPercentage(numerator, denominator);
+    }
+
+    /** */
+    public Double getPercentageOriginalRows(Join rel, RelMetadataQuery mq) {
+        // Assume any single-table filter conditions have already
+        // been pushed down.
+
+        // REVIEW jvs 28-Mar-2006: As with aggregation, this is
+        // oversimplified.
+
+        // REVIEW jvs 28-Mar-2006:  need any special casing for SemiJoin?
+
+        Double left = mq.getPercentageOriginalRows(rel.getLeft());
+        if (left == null) {
+            return null;
+        }
+        Double right = mq.getPercentageOriginalRows(rel.getRight());
+        if (right == null) {
+            return null;
+        }
+        return left * right;
+    }
+
+    /** */
+    public Double getPercentageOriginalRows(RelNode rel, RelMetadataQuery mq) {
+        if (rel.getInputs().size() > 1) {
+            // No generic formula available for multiple inputs.
+            return null;
+        }
+
+        if (rel.getInputs().size() == 0) {
+            // Assume no filtering happening at leaf.
+            return 1.0;
+        }
+
+        RelNode child = rel.getInputs().get(0);
+
+        Double childPercentage = mq.getPercentageOriginalRows(child);
+        if (childPercentage == null) {
+            return null;
+        }
+
+        // Compute product of percentage filtering from this rel (assuming any
+        // filtering is the effect of single-table filters) with the percentage
+        // filtering performed by the child.
+        Double relPercentage =
+            quotientForPercentage(mq.getRowCount(rel), mq.getRowCount(child));
+        if (relPercentage == null) {
+            return null;
+        }
+        double percent = relPercentage * childPercentage;
+
+        // this check is needed in cases where this method is called on a
+        // physical rel
+        if ((percent < 0.0) || (percent > 1.0)) {
+            return null;
+        }
+        return relPercentage * childPercentage;
+    }
+
+    /** */
+    private static Double quotientForPercentage(
+        Double numerator,
+        Double denominator) {
+        if ((numerator == null) || (denominator == null)) {
+            return null;
+        }
+
+        // may need epsilon instead
+        if (denominator == 0.0) {
+            // cap at 100%
+            return 1.0;
+        } else {
+            return numerator / denominator;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdPredicates.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdPredicates.java
new file mode 100644
index 0000000..f520df5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdPredicates.java
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMdPredicates;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
+
+/** */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
+public class IgniteMdPredicates extends RelMdPredicates {
+    /** */
+    public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider
+        .reflectiveSource(BuiltInMethod.PREDICATES.method, new IgniteMdPredicates());
+
+    /**
+     * See {@link RelMdPredicates#getPredicates(org.apache.calcite.rel.RelNode, org.apache.calcite.rel.metadata.RelMetadataQuery)}
+     */
+    public RelOptPredicateList getPredicates(IgniteIndexScan rel, RelMetadataQuery mq) {
+        if (rel.condition() == null)
+            return RelOptPredicateList.EMPTY;
+
+        return RelOptPredicateList.of(rel.getCluster().getRexBuilder(),
+                    RexUtil.retainDeterministic(
+                        RelOptUtil.conjunctions(rel.condition().accept(new LocalRefReplacer()))));
+    }
+
+    /** Visitor for replacing scan local refs to input refs. */
+    private static class LocalRefReplacer extends RexShuttle {
+        @Override public RexNode visitLocalRef(RexLocalRef inputRef) {
+            return new RexInputRef(inputRef.getIndex(), inputRef.getType());
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
new file mode 100644
index 0000000..d7e9387
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
@@ -0,0 +1,107 @@
+/*
+ * 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.metadata;
+
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMdRowCount;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.calcite.util.NumberUtil.multiply;
+
+/** */
+@SuppressWarnings("unused") // actually all methods are used by runtime generated classes
+public class IgniteMdRowCount extends RelMdRowCount {
+    public static final RelMetadataProvider SOURCE =
+        ReflectiveRelMetadataProvider.reflectiveSource(
+            BuiltInMethod.ROW_COUNT.method, new IgniteMdRowCount());
+
+    /** {@inheritDoc} */
+    @Override public Double getRowCount(Join rel, RelMetadataQuery mq) {
+        return joinRowCount(mq, rel);
+    }
+
+    /** */
+    @Nullable public static Double joinRowCount(RelMetadataQuery mq, Join rel) {
+        if (!rel.getJoinType().projectsRight()) {
+          // Create a RexNode representing the selectivity of the
+          // semijoin filter and pass it to getSelectivity
+          RexNode semiJoinSelectivity =
+              RelMdUtil.makeSemiJoinSelectivityRexNode(mq, rel);
+
+          return multiply(mq.getSelectivity(rel.getLeft(), semiJoinSelectivity),
+              mq.getRowCount(rel.getLeft()));
+        }
+
+        // Row count estimates of 0 will be rounded up to 1.
+        // So, use maxRowCount where the product is very small.
+        final Double left = mq.getRowCount(rel.getLeft());
+        final Double right = mq.getRowCount(rel.getRight());
+
+        if (left == null || right == null)
+            return null;
+
+        if (left <= 1D || right <= 1D) {
+          Double max = mq.getMaxRowCount(rel);
+          if (max != null && max <= 1D)
+              return max;
+        }
+
+        JoinInfo joinInfo = rel.analyzeCondition();
+
+        ImmutableIntList leftKeys = joinInfo.leftKeys;
+        ImmutableIntList rightKeys = joinInfo.rightKeys;
+
+        double selectivity = mq.getSelectivity(rel, rel.getCondition());
+
+        if (F.isEmpty(leftKeys) || F.isEmpty(rightKeys))
+            return left * right * selectivity;
+
+        double leftDistinct = Util.first(
+            mq.getDistinctRowCount(rel.getLeft(), ImmutableBitSet.of(leftKeys), null), left);
+        double rightDistinct = Util.first(
+            mq.getDistinctRowCount(rel.getRight(), ImmutableBitSet.of(rightKeys), null), right);
+
+        double leftCardinality = leftDistinct / left;
+        double rightCardinality = rightDistinct / right;
+
+        double rowsCount = (Math.min(left, right) / (leftCardinality * rightCardinality)) * selectivity;
+
+        JoinRelType type = rel.getJoinType();
+
+        if (type == JoinRelType.LEFT)
+            rowsCount += left;
+        else if (type == JoinRelType.RIGHT)
+            rowsCount += right;
+        else if (type == JoinRelType.FULL)
+            rowsCount += left + right;
+
+        return rowsCount;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java
index e50fa5a..cb3c762 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java
@@ -20,10 +20,26 @@ package org.apache.ignite.internal.processors.query.calcite.metadata;
 import com.google.common.collect.ImmutableList;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
-import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.Metadata;
 import org.apache.calcite.rel.metadata.MetadataDef;
 import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.RelMdAllPredicates;
+import org.apache.calcite.rel.metadata.RelMdCollation;
+import org.apache.calcite.rel.metadata.RelMdColumnOrigins;
+import org.apache.calcite.rel.metadata.RelMdColumnUniqueness;
+import org.apache.calcite.rel.metadata.RelMdDistinctRowCount;
+import org.apache.calcite.rel.metadata.RelMdExplainVisibility;
+import org.apache.calcite.rel.metadata.RelMdExpressionLineage;
+import org.apache.calcite.rel.metadata.RelMdMaxRowCount;
+import org.apache.calcite.rel.metadata.RelMdMemory;
+import org.apache.calcite.rel.metadata.RelMdMinRowCount;
+import org.apache.calcite.rel.metadata.RelMdNodeTypes;
+import org.apache.calcite.rel.metadata.RelMdParallelism;
+import org.apache.calcite.rel.metadata.RelMdPopulationSize;
+import org.apache.calcite.rel.metadata.RelMdSelectivity;
+import org.apache.calcite.rel.metadata.RelMdSize;
+import org.apache.calcite.rel.metadata.RelMdTableReferences;
+import org.apache.calcite.rel.metadata.RelMdUniqueKeys;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
@@ -35,10 +51,37 @@ public class IgniteMetadata {
     public static final RelMetadataProvider METADATA_PROVIDER =
         ChainedRelMetadataProvider.of(
             ImmutableList.of(
-                IgniteMdDistribution.SOURCE,
+                // Ignite specific providers
                 IgniteMdNodesMapping.SOURCE,
-                DefaultRelMetadataProvider.INSTANCE));
 
+                // Ignite overriden providers
+                IgniteMdDistribution.SOURCE,
+                IgniteMdPercentageOriginalRows.SOURCE,
+                IgniteMdCumulativeCost.SOURCE,
+                IgniteMdNonCumulativeCost.SOURCE,
+                IgniteMdRowCount.SOURCE,
+                IgniteMdPredicates.SOURCE,
+
+                // Basic providers
+                RelMdColumnOrigins.SOURCE,
+                RelMdExpressionLineage.SOURCE,
+                RelMdTableReferences.SOURCE,
+                RelMdNodeTypes.SOURCE,
+                RelMdMaxRowCount.SOURCE,
+                RelMdMinRowCount.SOURCE,
+                RelMdUniqueKeys.SOURCE,
+                RelMdColumnUniqueness.SOURCE,
+                RelMdPopulationSize.SOURCE,
+                RelMdSize.SOURCE,
+                RelMdParallelism.SOURCE,
+                RelMdMemory.SOURCE,
+                RelMdDistinctRowCount.SOURCE,
+                RelMdSelectivity.SOURCE,
+                RelMdExplainVisibility.SOURCE,
+                RelMdAllPredicates.SOURCE,
+                RelMdCollation.SOURCE));
+
+    /** */
     public interface NodesMappingMetadata extends Metadata {
         MetadataDef<NodesMappingMetadata> DEF = MetadataDef.of(NodesMappingMetadata.class,
             NodesMappingMetadata.Handler.class, IgniteMethod.NODES_MAPPING.method());
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
index 32f37ea..d42e00b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
@@ -17,39 +17,37 @@
 
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
-import com.google.common.collect.ImmutableList;
+import java.lang.reflect.Modifier;
+import java.util.List;
+import java.util.stream.Collectors;
+
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
-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.IgniteJoin;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.reflections.Reflections;
+import org.reflections.scanners.SubTypesScanner;
+import org.reflections.util.ConfigurationBuilder;
 
 /**
  * See {@link RelMetadataQuery}
  */
 public class RelMetadataQueryEx extends RelMetadataQuery {
     static {
-        JaninoRelMetadataProvider.DEFAULT.register(
-            ImmutableList.of(
-                IgniteExchange.class,
-                IgniteReceiver.class,
-                IgniteSender.class,
-                IgniteFilter.class,
-                IgniteProject.class,
-                IgniteJoin.class,
-                IgniteTableScan.class,
-                IgniteValues.class,
-                IgniteTableModify.class,
-                IgniteSort.class));
+        ConfigurationBuilder cfg = new ConfigurationBuilder()
+            .forPackages("org.apache.ignite.internal.processors.query.calcite.rel")
+            .addClassLoaders(U.gridClassLoader())
+            .addScanners(new SubTypesScanner());
+
+        List<Class<? extends RelNode>> types = new Reflections(cfg)
+            .getSubTypesOf(IgniteRel.class).stream()
+            .filter(type -> !type.isInterface())
+            .filter(type -> !Modifier.isAbstract(type.getModifiers()))
+            .collect(Collectors.toList());
+
+        JaninoRelMetadataProvider.DEFAULT.register(types);
     }
 
     /** */
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 551a816..5ad1bf3 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,10 +24,12 @@ import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.RelNode;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
 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.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMapAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReduceAggregate;
@@ -36,7 +38,6 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTrimExchange;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteUnionAll;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
@@ -113,16 +114,24 @@ class Cloner implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteJoin rel) {
+    @Override public IgniteRel visit(IgniteNestedLoopJoin rel) {
         RelNode left = visit((IgniteRel) rel.getLeft());
         RelNode right = visit((IgniteRel) rel.getRight());
 
-        return new IgniteJoin(cluster, rel.getTraitSet(), left, right, rel.getCondition(), rel.getVariablesSet(), rel.getJoinType());
+        return new IgniteNestedLoopJoin(cluster, rel.getTraitSet(), left, right, rel.getCondition(), rel.getVariablesSet(), rel.getJoinType());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteTableScan rel) {
-        return new IgniteTableScan(cluster, rel.getTraitSet(), rel.getTable(), rel.indexName(), rel.condition());
+    @Override public IgniteRel visit(IgniteCorrelatedNestedLoopJoin rel) {
+        RelNode left = visit((IgniteRel) rel.getLeft());
+        RelNode right = visit((IgniteRel) rel.getRight());
+
+        return new IgniteCorrelatedNestedLoopJoin(cluster, rel.getTraitSet(), left, right, rel.getCondition(), rel.getVariablesSet(), rel.getJoinType());
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteRel visit(IgniteIndexScan rel) {
+        return new IgniteIndexScan(cluster, rel.getTraitSet(), rel.getTable(), rel.indexName(), rel.condition());
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentDescription.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentDescription.java
index 1b836eb..1475c91 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentDescription.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentDescription.java
@@ -42,7 +42,7 @@ public class FragmentDescription implements MarshalableMessage {
     private long fragmentId;
 
     /** */
-    private int[] partitions;
+    private int[] localPartitions;
 
     /** */
     private int partitionsCount;
@@ -63,11 +63,10 @@ public class FragmentDescription implements MarshalableMessage {
     }
 
     /** */
-    public FragmentDescription(long fragmentId, int[] partitions, int partitionsCount,
-        NodesMapping targetMapping,
+    public FragmentDescription(long fragmentId, int[] localPartitions, int partitionsCount, NodesMapping targetMapping,
         Map<Long, List<UUID>> remoteSources) {
         this.fragmentId = fragmentId;
-        this.partitions = partitions;
+        this.localPartitions = localPartitions;
         this.partitionsCount = partitionsCount;
         this.targetMapping = targetMapping;
         this.remoteSources = remoteSources;
@@ -89,8 +88,8 @@ public class FragmentDescription implements MarshalableMessage {
     }
 
     /** */
-    public int[] partitions() {
-        return partitions;
+    public int[] localPartitions() {
+        return localPartitions;
     }
 
     /** */
@@ -122,7 +121,7 @@ public class FragmentDescription implements MarshalableMessage {
                 writer.incrementState();
 
             case 1:
-                if (!writer.writeIntArray("partitions", partitions))
+                if (!writer.writeIntArray("localPartitions", localPartitions))
                     return false;
 
                 writer.incrementState();
@@ -167,7 +166,7 @@ public class FragmentDescription implements MarshalableMessage {
                 reader.incrementState();
 
             case 1:
-                partitions = reader.readIntArray("partitions");
+                localPartitions = reader.readIntArray("localPartitions");
 
                 if (!reader.isLastRead())
                     return false;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentSplitter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentSplitter.java
index 9c73207..2d6b8be 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentSplitter.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentSplitter.java
@@ -28,10 +28,12 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
 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.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMapAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReduceAggregate;
@@ -40,7 +42,6 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTrimExchange;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteUnionAll;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
@@ -95,7 +96,12 @@ public class FragmentSplitter implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteJoin rel) {
+    @Override public IgniteRel visit(IgniteNestedLoopJoin rel) {
+        return processNode(rel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteRel visit(IgniteCorrelatedNestedLoopJoin rel) {
         return processNode(rel);
     }
 
@@ -139,7 +145,7 @@ public class FragmentSplitter implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteTableScan rel) {
+    @Override public IgniteRel visit(IgniteIndexScan rel) {
         return processNode(rel);
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
index 9b96e43..f4eac75 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
@@ -34,7 +34,6 @@ import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.plan.volcano.VolcanoPlanner;
 import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
@@ -60,7 +59,6 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata;
 import org.apache.ignite.internal.processors.query.calcite.metadata.RelMetadataQueryEx;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 
 /**
@@ -341,10 +339,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
 
         /** {@inheritDoc} */
         @Override public RelOptCost getCost(RelNode rel, RelMetadataQuery mq) {
-            if (rel instanceof IgniteRel && ((IgniteRel)rel).distribution().getType() == RelDistribution.Type.ANY)
-                return getCostFactory().makeInfiniteCost(); // force certain distributions
-
-            return super.getCost(rel, mq);
+            return mq.getCumulativeCost(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 dddf93b..7ec3565 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
 import org.apache.calcite.rel.rules.SortRemoveRule;
 import org.apache.calcite.rel.rules.SubQueryRemoveRule;
 import org.apache.calcite.rel.rules.UnionMergeRule;
@@ -25,11 +26,12 @@ import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.RuleSets;
 import org.apache.ignite.internal.processors.query.calcite.rule.AggregateConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rule.CorrelatedNestedLoopJoinConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rule.ExposeIndexRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.FilterConverterRule;
-import org.apache.ignite.internal.processors.query.calcite.rule.JoinConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rule.NestedLoopJoinConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.ProjectConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.PushFilterIntoScanRule;
-import org.apache.ignite.internal.processors.query.calcite.rule.RegisterIndexRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.SortConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.TableModifyConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.UnionConverterRule;
@@ -68,12 +70,14 @@ public enum PlannerPhase {
         /** {@inheritDoc} */
         @Override public RuleSet getRules(PlanningContext ctx) {
             return RuleSets.ofList(
-                RegisterIndexRule.INSTANCE,
+                ExposeIndexRule.INSTANCE,
                 AggregateConverterRule.INSTANCE,
-                JoinConverterRule.INSTANCE,
+                NestedLoopJoinConverterRule.INSTANCE,
+                CorrelatedNestedLoopJoinConverterRule.INSTANCE,
                 FilterJoinRule.PUSH_JOIN_CONDITION,
                 FilterJoinRule.FILTER_ON_JOIN,
                 ProjectConverterRule.INSTANCE,
+                ProjectMergeRule.INSTANCE,
                 FilterConverterRule.INSTANCE,
                 LogicalFilterMergeRule.INSTANCE,
                 LogicalFilterProjectTransposeRule.INSTANCE,
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
index 7ddbd3d..e4cc623 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
@@ -37,26 +37,19 @@ import org.apache.calcite.tools.Frameworks;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
-import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.logger.NullLogger;
 import org.jetbrains.annotations.NotNull;
 
+import static org.apache.calcite.tools.Frameworks.createRootSchema;
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
 /**
  * Planning context.
  */
 public final class PlanningContext implements Context {
     /** */
-    private static final Context EMPTY_CONTEXT = Contexts.empty();
-
-    /** */
-    private static final FrameworkConfig EMPTY_CONFIG =
-        Frameworks.newConfigBuilder(CalciteQueryProcessor.FRAMEWORK_CONFIG)
-        .defaultSchema(Frameworks.createRootSchema(false))
-        .traitDefs()
-        .build();
-
-    /** */
-    public static final PlanningContext EMPTY = new PlanningContext();
+    private static final PlanningContext EMPTY = builder().build();
 
     /** */
     private final FrameworkConfig cfg;
@@ -127,23 +120,6 @@ public final class PlanningContext implements Context {
     }
 
     /**
-     * Constructor for empty context.
-     */
-    private PlanningContext() {
-        cfg = EMPTY_CONFIG;
-        parentCtx = EMPTY_CONTEXT;
-        RelDataTypeSystem typeSys = connectionConfig().typeSystem(RelDataTypeSystem.class, cfg.getTypeSystem());
-        typeFactory = new IgniteTypeFactory(typeSys);
-        locNodeId = null;
-        originatingNodeId = null;
-        qry = null;
-        parameters = null;
-        topVer = null;
-        qryCancel = null;
-        log = null;
-    }
-
-    /**
      * @return Local node ID.
      */
     public UUID localNodeId() {
@@ -327,10 +303,17 @@ public final class PlanningContext implements Context {
     @SuppressWarnings("PublicInnerClass") 
     public static class Builder {
         /** */
+        private static final FrameworkConfig EMPTY_CONFIG =
+            Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                .defaultSchema(createRootSchema(false))
+                .traitDefs()
+                .build();
+
+        /** */
         private FrameworkConfig frameworkCfg = EMPTY_CONFIG;
 
         /** */
-        private Context parentCtx = EMPTY_CONTEXT;
+        private Context parentCtx = Contexts.empty();
 
         /** */
         private UUID locNodeId;
@@ -348,7 +331,7 @@ public final class PlanningContext implements Context {
         private AffinityTopologyVersion topVer;
 
         /** */
-        private IgniteLogger log;
+        private IgniteLogger log = new NullLogger();
 
         /**
          * @param locNodeId Local node ID.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java
index 5a5dee0..4c78273 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java
@@ -25,10 +25,12 @@ import java.util.List;
 import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
 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.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMapAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReduceAggregate;
@@ -37,7 +39,6 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSort;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTrimExchange;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteUnionAll;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
@@ -90,7 +91,12 @@ public class Splitter implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteJoin rel) {
+    @Override public IgniteRel visit(IgniteNestedLoopJoin rel) {
+        return processNode(rel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteRel visit(IgniteCorrelatedNestedLoopJoin rel) {
         return processNode(rel);
     }
 
@@ -125,7 +131,7 @@ public class Splitter implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteTableScan rel) {
+    @Override public IgniteRel visit(IgniteIndexScan rel) {
         return rel;
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteNestedLoopJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteNestedLoopJoin.java
new file mode 100644
index 0000000..e382599
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/AbstractIgniteNestedLoopJoin.java
@@ -0,0 +1,414 @@
+/*
+ * 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.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+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.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelNodes;
+import org.apache.calcite.rel.RelWriter;
+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.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitsAwareIgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static org.apache.calcite.rel.RelDistribution.Type.BROADCAST_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.SINGLETON;
+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.ignite.internal.processors.query.calcite.metadata.IgniteMdRowCount.joinRowCount;
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.broadcast;
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.hash;
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.single;
+
+/** */
+public abstract class AbstractIgniteNestedLoopJoin extends Join implements TraitsAwareIgniteRel {
+    /** */
+    protected AbstractIgniteNestedLoopJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right,
+        RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+        super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    /** {@inheritDoc} */
+    @Override public abstract Join copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right,
+        JoinRelType joinType, boolean semiJoinDone);
+
+    /** {@inheritDoc} */
+    @Override public abstract <T> T accept(IgniteRelVisitor<T> visitor);
+
+    /** {@inheritDoc} */
+    @Override public RelWriter explainTerms(RelWriter pw) {
+        return super.explainTerms(pw)
+            .itemIf("variablesSet", Commons.transform(variablesSet.asList(), CorrelationId::getId), pw.getDetailLevel() == SqlExplainLevel.ALL_ATTRIBUTES);
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // We preserve left collation since it's translated into a nested loop join with an outer loop
+        // over a left edge. The code below checks and projects left collation on an output row type.
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        RelCollation collation = TraitUtils.collation(left);
+
+        // If nulls are possible at left we has to check whether NullDirection.LAST flag is set on sorted fields.
+        // TODO set NullDirection.LAST for insufficient fields instead of erasing collation.
+        if (joinType == RIGHT || joinType == JoinRelType.FULL) {
+            for (RelFieldCollation field : collation.getFieldCollations()) {
+                if (RelFieldCollation.NullDirection.LAST != field.nullDirection) {
+                    collation = RelCollations.EMPTY;
+                    break;
+                }
+            }
+        }
+
+        outTraits = nodeTraits.replace(collation);
+        leftTraits = left.replace(collation);
+        rightTraits = right.replace(RelCollations.EMPTY);
+
+        return ImmutableList.of(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // The node is rewindable only if both sources are rewindable.
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        ImmutableList.Builder<Pair<RelTraitSet, List<RelTraitSet>>> b = ImmutableList.builder();
+
+        RewindabilityTrait leftRewindability = TraitUtils.rewindability(left);
+        RewindabilityTrait rightRewindability = TraitUtils.rewindability(right);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        outTraits = nodeTraits.replace(RewindabilityTrait.ONE_WAY);
+        leftTraits = left.replace(RewindabilityTrait.ONE_WAY);
+        rightTraits = right.replace(RewindabilityTrait.ONE_WAY);
+
+        b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+        if (leftRewindability.rewindable() && rightRewindability.rewindable()) {
+            outTraits = nodeTraits.replace(RewindabilityTrait.REWINDABLE);
+            leftTraits = left.replace(RewindabilityTrait.REWINDABLE);
+            rightTraits = right.replace(RewindabilityTrait.REWINDABLE);
+
+            b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+        }
+
+        return b.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Tere are several rules:
+        // 1) any join is possible on broadcast or single distribution
+        // 2) hash distributed join is possible when join keys equal to source distribution keys
+        // 3) hash and broadcast distributed tables can be joined when join keys equal to hash
+        //    distributed table distribution keys and:
+        //      3.1) it's a left join and a hash distributed table is at left
+        //      3.2) it's a right join and a hash distributed table is at right
+        //      3.3) it's an inner join, this case a hash distributed table may be at any side
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        ImmutableList.Builder<Pair<RelTraitSet, List<RelTraitSet>>> b = ImmutableList.builder();
+
+        IgniteDistribution leftDistr = TraitUtils.distribution(left);
+        IgniteDistribution rightDistr = TraitUtils.distribution(right);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        outTraits = nodeTraits.replace(single());
+        leftTraits = left.replace(single());
+        rightTraits = right.replace(single());
+
+        b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+        outTraits = nodeTraits.replace(broadcast());
+        leftTraits = left.replace(broadcast());
+        rightTraits = right.replace(broadcast());
+
+        b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+        if (!F.isEmpty(joinInfo.pairs())) {
+            Set<DistributionFunction> functions = new HashSet<>();
+
+            if (leftDistr.getType() == HASH_DISTRIBUTED
+                && Objects.equals(joinInfo.leftKeys, leftDistr.getKeys()))
+                functions.add(leftDistr.function());
+
+            if (rightDistr.getType() == HASH_DISTRIBUTED
+                && Objects.equals(joinInfo.rightKeys, rightDistr.getKeys()))
+                functions.add(rightDistr.function());
+
+            functions.add(DistributionFunction.HashDistribution.INSTANCE);
+
+            for (DistributionFunction function : functions) {
+                leftTraits = left.replace(hash(joinInfo.leftKeys, function));
+                rightTraits = right.replace(hash(joinInfo.rightKeys, function));
+
+                // TODO distribution multitrait support
+                outTraits = nodeTraits.replace(hash(joinInfo.leftKeys, function));
+                b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+                outTraits = nodeTraits.replace(hash(joinInfo.rightKeys, function));
+                b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+                if (joinType == INNER || joinType == LEFT) {
+                    outTraits = nodeTraits.replace(hash(joinInfo.leftKeys, function));
+                    leftTraits = left.replace(hash(joinInfo.leftKeys, function));
+                    rightTraits = right.replace(broadcast());
+
+                    b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+                }
+
+                if (joinType == INNER || joinType == RIGHT) {
+                    outTraits = nodeTraits.replace(hash(joinInfo.rightKeys, function));
+                    leftTraits = left.replace(broadcast());
+                    rightTraits = right.replace(hash(joinInfo.rightKeys, function));
+
+                    b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+                }
+            }
+        }
+
+        return b.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // We preserve left collation since it's translated into a nested loop join with an outer loop
+        // over a left edge. The code below checks whether a desired collation is possible and requires
+        // appropriate collation from the left edge.
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        RelCollation collation = TraitUtils.collation(nodeTraits);
+
+        if (!projectsLeft(collation))
+            collation = RelCollations.EMPTY;
+        else if (joinType == RIGHT || joinType == JoinRelType.FULL) {
+            for (RelFieldCollation field : collation.getFieldCollations()) {
+                if (RelFieldCollation.NullDirection.LAST != field.nullDirection) {
+                    collation = RelCollations.EMPTY;
+                    break;
+                }
+            }
+        }
+
+        outTraits = nodeTraits.replace(collation);
+        leftTraits = left.replace(collation);
+        rightTraits = right.replace(RelCollations.EMPTY);
+
+        return ImmutableList.of(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // The node is rewindable only if both sources are rewindable.
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        RewindabilityTrait rewindability = TraitUtils.rewindability(nodeTraits);
+
+        outTraits = nodeTraits.replace(rewindability);
+        leftTraits = left.replace(rewindability);
+        rightTraits = right.replace(rewindability);
+
+        return ImmutableList.of(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Tere are several rules:
+        // 1) any join is possible on broadcast or single distribution
+        // 2) hash distributed join is possible when join keys equal to source distribution keys
+        // 3) hash and broadcast distributed tables can be joined when join keys equal to hash
+        //    distributed table distribution keys and:
+        //      3.1) it's a left join and a hash distributed table is at left
+        //      3.2) it's a right join and a hash distributed table is at right
+        //      3.3) it's an inner join, this case a hash distributed table may be at any side
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        List<Pair<RelTraitSet, List<RelTraitSet>>> res = new ArrayList<>();
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        IgniteDistribution distribution = TraitUtils.distribution(nodeTraits);
+
+        RelDistribution.Type distrType = distribution.getType();
+        switch (distrType) {
+            case BROADCAST_DISTRIBUTED:
+            case SINGLETON:
+                outTraits = nodeTraits.replace(distribution);
+                leftTraits = left.replace(distribution);
+                rightTraits = right.replace(distribution);
+
+                res.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+                break;
+            case HASH_DISTRIBUTED:
+            case RANDOM_DISTRIBUTED:
+                // Such join may be replaced as a cross join with a filter uppon it.
+                // It's impossible to get random or hash distribution from a cross join.
+                if (F.isEmpty(joinInfo.pairs()))
+                    break;
+
+                // We cannot provide random distribution without unique constrain on join keys,
+                // so, we require hash distribution (wich satisfies random distribution) instead.
+                DistributionFunction function = distrType == HASH_DISTRIBUTED
+                    ? distribution.function()
+                    : DistributionFunction.HashDistribution.INSTANCE;
+
+                IgniteDistribution outDistr; // TODO distribution multitrait support
+
+                outDistr = hash(joinInfo.leftKeys, function);
+
+                if (distrType != HASH_DISTRIBUTED || outDistr.satisfies(distribution)) {
+                    outTraits = nodeTraits.replace(outDistr);
+                    leftTraits = left.replace(hash(joinInfo.leftKeys, function));
+                    rightTraits = right.replace(hash(joinInfo.rightKeys, function));
+
+                    res.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+                    if (joinType == INNER || joinType == LEFT) {
+                        outTraits = nodeTraits.replace(outDistr);
+                        leftTraits = left.replace(hash(joinInfo.leftKeys, function));
+                        rightTraits = right.replace(broadcast());
+
+                        res.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+                    }
+                }
+
+                outDistr = hash(joinInfo.rightKeys, function);
+
+                if (distrType != HASH_DISTRIBUTED || outDistr.satisfies(distribution)) {
+                    outTraits = nodeTraits.replace(outDistr);
+                    leftTraits = left.replace(hash(joinInfo.leftKeys, function));
+                    rightTraits = right.replace(hash(joinInfo.rightKeys, function));
+
+                    res.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+                    if (joinType == INNER || joinType == RIGHT) {
+                        outTraits = nodeTraits.replace(outDistr);
+                        leftTraits = left.replace(broadcast());
+                        rightTraits = right.replace(hash(joinInfo.rightKeys, function));
+
+                        res.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+                    }
+                }
+
+                break;
+
+            default:
+                break;
+        }
+
+        if (!res.isEmpty())
+            return res;
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(single()),
+            ImmutableList.of(left.replace(single()), right.replace(single()))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public double estimateRowCount(RelMetadataQuery mq) {
+        return Util.first(joinRowCount(mq, this), 1D);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        double rowCount = mq.getRowCount(this);
+
+        // Joins can be flipped, and for many algorithms, both versions are viable
+        // and have the same cost. To make the results stable between versions of
+        // the planner, make one of the versions slightly more expensive.
+        switch (joinType) {
+            case SEMI:
+            case ANTI:
+                // SEMI and ANTI join cannot be flipped
+                break;
+            case RIGHT:
+                rowCount = RelMdUtil.addEpsilon(rowCount);
+                break;
+            default:
+                if (RelNodes.COMPARATOR.compare(left, right) > 0)
+                    rowCount = RelMdUtil.addEpsilon(rowCount);
+        }
+
+        final double rightRowCount = right.estimateRowCount(mq);
+        final double leftRowCount = left.estimateRowCount(mq);
+
+        if (Double.isInfinite(leftRowCount))
+            rowCount = leftRowCount;
+        if (Double.isInfinite(rightRowCount))
+            rowCount = rightRowCount;
+
+        RelDistribution.Type type = distribution().getType();
+
+        if (type == BROADCAST_DISTRIBUTED || type == SINGLETON)
+            rowCount = RelMdUtil.addEpsilon(rowCount);
+
+        return planner.getCostFactory().makeCost(rowCount, 0, 0);
+    }
+
+    /** */
+    protected boolean projectsLeft(RelCollation collation) {
+        int leftFieldCount = getLeft().getRowType().getFieldCount();
+        for (int field : RelCollations.ordinals(collation)) {
+            if (field >= leftFieldCount)
+                return false;
+        }
+        return true;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java
index 00424d4..119303d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteAggregate.java
@@ -18,17 +18,14 @@
 package org.apache.ignite.internal.processors.query.calcite.rel;
 
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.DeriveMode;
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -39,16 +36,17 @@ import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
-import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitsAwareIgniteRel;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
 
 import static org.apache.calcite.plan.RelOptRule.convert;
 import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
 import static org.apache.calcite.util.ImmutableIntList.range;
-import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.any;
 import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.broadcast;
 import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.hash;
 import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.random;
@@ -58,7 +56,7 @@ import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUti
 /**
  *
  */
-public class IgniteAggregate extends Aggregate implements IgniteRel {
+public class IgniteAggregate extends Aggregate implements TraitsAwareIgniteRel {
     /** {@inheritDoc} */
     public IgniteAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
         super(cluster, traitSet, input, groupSet, groupSets, aggCalls);
@@ -80,155 +78,220 @@ public class IgniteAggregate extends Aggregate implements IgniteRel {
     }
 
     /** {@inheritDoc} */
-    @Override public RelNode passThrough(RelTraitSet required) {
-        IgniteDistribution toDistr = TraitUtils.distribution(required);
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Aggregate is rewindable if its input is rewindable.
 
-        // Hash aggregate erases collation and only distribution trait can be passed through.
-        // So that, it's no use to pass ANY distribution.
-        if (toDistr == any())
-            return null;
+        RewindabilityTrait rewindability = TraitUtils.rewindability(nodeTraits);
 
-        List<Pair<IgniteDistribution, IgniteDistribution>> distributions = new ArrayList<>();
-        RelDistribution.Type distrType = toDistr.getType();
+        return ImmutableList.of(Pair.of(nodeTraits, ImmutableList.of(inputTraits.get(0).replace(rewindability))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Distribution propagation is based on next rules:
+        // 1) Any aggregation is possible on single or broadcast distribution.
+        // 2) hash-distributed aggregation is possible in case it's a simple aggregate having hash distributed input
+        //    and all of input distribution keys are parts of aggregation group and vice versa.
+        // 3) Map-reduce aggregation is possible in case it's a simple aggregate and its input has random distribution.
+
+        RelTraitSet in = inputTraits.get(0);
+
+        ImmutableList.Builder<Pair<RelTraitSet, List<RelTraitSet>>> b = ImmutableList.builder();
+
+        IgniteDistribution distribution = TraitUtils.distribution(nodeTraits);
+
+        RelDistribution.Type distrType = distribution.getType();
 
         switch (distrType) {
             case SINGLETON:
             case BROADCAST_DISTRIBUTED:
-                if (!groupSet.isEmpty() && Group.induce(groupSet, groupSets) == Group.SIMPLE) {
-                    distributions.add(Pair.of(toDistr, random()));
-                    distributions.add(Pair.of(toDistr, hash(groupSet.asList())));
-                }
+                b.add(Pair.of(nodeTraits, ImmutableList.of(in.replace(distribution))));
 
-                distributions.add(Pair.of(toDistr, toDistr));
+                if (isSimple(this))
+                    b.add(Pair.of(nodeTraits, ImmutableList.of(in.replace(random())))); // Map-reduce aggregate
 
                 break;
+
             case RANDOM_DISTRIBUTED:
-            case HASH_DISTRIBUTED:
-                if (!groupSet.isEmpty() && Group.induce(groupSet, groupSets) == Group.SIMPLE)
+                if (!groupSet.isEmpty() && isSimple(this)) {
+                    IgniteDistribution outDistr = hash(range(0, groupSet.cardinality()));
+                    IgniteDistribution inDistr = hash(groupSet.asList());
+
+                    b.add(Pair.of(nodeTraits.replace(outDistr), ImmutableList.of(in.replace(inDistr))));
+
                     break;
+                }
 
-                DistributionFunction function = distrType == HASH_DISTRIBUTED
-                    ? toDistr.function()
-                    : DistributionFunction.HashDistribution.INSTANCE;
+                b.add(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(single()))));
 
-                IgniteDistribution outDistr = hash(range(0, groupSet.cardinality()), function);
+                break;
 
-                if (distrType == HASH_DISTRIBUTED && !outDistr.satisfies(toDistr))
-                    break;
+            case HASH_DISTRIBUTED:
+                ImmutableIntList keys = distribution.getKeys();
+
+                if (isSimple(this) && groupSet.cardinality() == keys.size()) {
+                    Mappings.TargetMapping mapping = groupMapping(
+                        getInput().getRowType().getFieldCount(), groupSet);
+
+                    List<Integer> srcKeys = new ArrayList<>(keys.size());
 
-                IgniteDistribution inDistr = hash(groupSet.asList(), function);
+                    for (int key : keys) {
+                        int src = mapping.getSourceOpt(key);
 
-                distributions.add(Pair.of(outDistr, inDistr));
+                        if (src == -1)
+                            break;
+
+                        srcKeys.add(src);
+                    }
+
+                    if (srcKeys.size() == keys.size()) {
+                        b.add(Pair.of(nodeTraits, ImmutableList.of(in.replace(hash(srcKeys, distribution.function())))));
+
+                        break;
+                    }
+                }
+
+                b.add(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(single()))));
+
+                break;
 
             default:
                 break;
         }
 
-        List<RelNode> nodes = createNodes(distributions);
+        return b.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Since it's a hash aggregate it erases collation.
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            ImmutableList.of(inputTraits.get(0).replace(RelCollations.EMPTY))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Aggregate is rewindable if its input is rewindable.
 
-        RelOptPlanner planner = getCluster().getPlanner();
-        for (int i = 1; i < nodes.size(); i++)
-            planner.register(nodes.get(i), this);
+        RelTraitSet in = inputTraits.get(0);
 
-        return F.first(nodes);
+        RewindabilityTrait rewindability = isMapReduce(nodeTraits, in)
+            ? RewindabilityTrait.ONE_WAY
+            : TraitUtils.rewindability(in);
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(rewindability), ImmutableList.of(in.replace(rewindability))));
     }
 
     /** {@inheritDoc} */
-    @Override public List<RelNode> derive(List<List<RelTraitSet>> inputTraits) {
-        assert inputTraits.size() == 1;
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Distribution propagation is based on next rules:
+        // 1) Any aggregation is possible on single or broadcast distribution.
+        // 2) hash-distributed aggregation is possible in case it's a simple aggregate having hash distributed input
+        //    and all of input distribution keys are parts aggregation group.
+        // 3) Map-reduce aggregation is possible in case it's a simple aggregate and its input has random distribution.
 
-        Set<IgniteDistribution> inDistrs = inputTraits.get(0).stream()
-            .map(TraitUtils::distribution)
-            // Hash aggregate erases collation and only distribution trait can be passed.
-            // So that, it's no use to pass ANY distribution.
-            .filter(d -> d != any())
-            .collect(Collectors.toSet());
+        RelTraitSet in = inputTraits.get(0);
 
-        Set<Pair<IgniteDistribution, IgniteDistribution>> pairs = new HashSet<>();
+        ImmutableList.Builder<Pair<RelTraitSet, List<RelTraitSet>>> b = ImmutableList.builder();
 
-        if (inDistrs.contains(single()))
-            pairs.add(Pair.of(single(), single()));
+        IgniteDistribution distribution = TraitUtils.distribution(in);
 
-        if (inDistrs.contains(broadcast()))
-            pairs.add(Pair.of(broadcast(), broadcast()));
+        RelDistribution.Type distrType = distribution.getType();
 
-        if (inDistrs.contains(random())) {
-            // Map-reduce cases
-            pairs.add(Pair.of(single(), random()));
-            pairs.add(Pair.of(broadcast(), random()));
-        }
+        switch (distrType) {
+            case SINGLETON:
+            case BROADCAST_DISTRIBUTED:
+                b.add(Pair.of(nodeTraits.replace(distribution), ImmutableList.of(in)));
+
+                break;
+
+            case HASH_DISTRIBUTED:
+                if (isSimple(this)) {
+                    ImmutableIntList keys = distribution.getKeys();
+
+                    if (groupSet.cardinality() == keys.size()) {
+                        Mappings.TargetMapping mapping = groupMapping(
+                            getInput().getRowType().getFieldCount(), groupSet);
+
+                        IgniteDistribution outDistr = distribution.apply(mapping);
+
+                        if (outDistr.getType() == HASH_DISTRIBUTED) {
+                            b.add(Pair.of(nodeTraits.replace(outDistr), ImmutableList.of(in)));
+
+                            break;
+                        }
+                    }
 
-        if (!groupSet.isEmpty() && isSimple(this)) {
-            int cardinality = groupSet.cardinality();
+                    b.add(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(random()))));
 
-            // Here we check whether all distribution keys contain in group set
-            for (IgniteDistribution in : inDistrs) {
-                if (in.getType() != HASH_DISTRIBUTED)
-                    continue;
+                    break;
+                }
 
-                ImmutableIntList keys = in.getKeys();
+                b.add(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(single()))));
 
-                if (keys.size() != cardinality)
-                    continue;
+                break;
 
-                Mappings.TargetMapping mapping = partialMapping(Math.max(Commons.max(keys), cardinality), cardinality);
+            case RANDOM_DISTRIBUTED:
+                // Map-reduce aggregates
+                if (isSimple(this)) {
+                    b.add(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(random()))));
+                    b.add(Pair.of(nodeTraits.replace(broadcast()), ImmutableList.of(in.replace(random()))));
 
-                IgniteDistribution out = in.apply(mapping);
+                    break;
+                }
+
+                b.add(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(single()))));
 
-                if (out.getType() != RelDistribution.Type.RANDOM_DISTRIBUTED)
-                    pairs.add(Pair.of(out, in));
-            }
+                break;
 
-            IgniteDistribution in = hash(groupSet.asList());
-            IgniteDistribution out = hash(range(0, cardinality));
-            pairs.add(Pair.of(out, in));
+            default:
+                break;
         }
 
-        return createNodes(pairs);
+        return b.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Since it's a hash aggregate it erases collation.
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            ImmutableList.of(inputTraits.get(0).replace(RelCollations.EMPTY))));
     }
 
     /** {@inheritDoc} */
-    @Override public DeriveMode getDeriveMode() {
-        return DeriveMode.OMAKASE;
+    @Override public @NotNull RelNode createNode(RelTraitSet outTraits, List<RelTraitSet> inTraits) {
+        RelTraitSet in = inTraits.get(0);
+
+        if (!isMapReduce(outTraits, in))
+            return copy(outTraits, ImmutableList.of(convert(getInput(), in)));
+
+        if (U.assertionsEnabled()) {
+            ImmutableList<RelTrait> diff = in.difference(outTraits);
+
+            assert diff.size() == 1 && F.first(diff) == TraitUtils.distribution(outTraits);
+        }
+
+        RelNode map = new IgniteMapAggregate(getCluster(), in, convert(getInput(), in), groupSet, groupSets, aggCalls);
+        return new IgniteReduceAggregate(getCluster(), outTraits, convert(map, outTraits), groupSet, groupSets, aggCalls, getRowType());
     }
 
     /** */
-    private Mappings.TargetMapping partialMapping(int inputFieldCount, int outputFieldCount) {
+    private boolean isMapReduce(RelTraitSet out, RelTraitSet in) {
+        return TraitUtils.distribution(out).satisfies(single())
+            && TraitUtils.distribution(in).satisfies(random());
+    }
+
+    /** */
+    @NotNull private static Mappings.TargetMapping groupMapping(int inputFieldCount, ImmutableBitSet groupSet) {
         Mappings.TargetMapping mapping =
             Mappings.create(MappingType.INVERSE_FUNCTION,
-                inputFieldCount, outputFieldCount);
+                inputFieldCount, groupSet.cardinality());
 
         for (Ord<Integer> group : Ord.zip(groupSet))
             mapping.set(group.e, group.i);
 
         return mapping;
     }
-
-    /** */
-    private List<RelNode> createNodes(Collection<Pair<IgniteDistribution, IgniteDistribution>> distrs) {
-        RelOptCluster cluster = getCluster();
-        List<RelNode> newRels = new ArrayList<>(distrs.size());
-
-        for (Pair<IgniteDistribution, IgniteDistribution> pair : distrs) {
-            RelTraitSet outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                .replace(pair.left);
-            RelTraitSet inTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                .replace(pair.right);
-            RelNode input0 = convert(input, inTraits);
-
-            if (pair.left.satisfies(single()) && pair.right.satisfies(random())) {
-                RelTraitSet mapTraits = input0.getTraitSet()
-                    .replace(IgniteMapAggregate.distribution(pair.right, groupSet, groupSets, aggCalls));
-
-                input0 = new IgniteMapAggregate(cluster, mapTraits, input0, groupSet, groupSets, aggCalls);
-                input0 = convert(input0, pair.left);
-
-                newRels.add(new IgniteReduceAggregate(cluster, outTraits, input0, groupSet, groupSets, aggCalls, getRowType()));
-            }
-            else
-                newRels.add(new IgniteAggregate(cluster, outTraits, input0, groupSet, groupSets, aggCalls));
-        }
-
-        return newRels;
-    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteCorrelatedNestedLoopJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteCorrelatedNestedLoopJoin.java
new file mode 100644
index 0000000..7769a58
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteCorrelatedNestedLoopJoin.java
@@ -0,0 +1,162 @@
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+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.RelCollations;
+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.Pair;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * Relational expression that combines two relational expressions according to
+ * some condition.
+ *
+ * <p>Each output row has columns from the left and right inputs.
+ * The set of output rows is a subset of the cartesian product of the two
+ * inputs; precisely which subset depends on the join condition.
+ */
+public class IgniteCorrelatedNestedLoopJoin extends AbstractIgniteNestedLoopJoin {
+    /**
+     * Creates a Join.
+     *
+     * @param cluster          Cluster
+     * @param traitSet         Trait set
+     * @param left             Left input
+     * @param right            Right input
+     * @param condition        Join condition
+     * @param joinType         Join type
+     * @param variablesSet     Set variables that are set by the
+     *                         LHS and used by the RHS and are not available to
+     *                         nodes above this Join in the tree
+     */
+    public IgniteCorrelatedNestedLoopJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+        super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    /** */
+    public IgniteCorrelatedNestedLoopJoin(RelInput input) {
+        this(input.getCluster(),
+            input.getTraitSet().replace(IgniteConvention.INSTANCE),
+            input.getInputs().get(0),
+            input.getInputs().get(1),
+            input.getExpression("condition"),
+            ImmutableSet.copyOf(Commons.transform(input.getIntegerList("variablesSet"), CorrelationId::new)),
+            input.getEnum("joinType", JoinRelType.class));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Join copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+        return new IgniteCorrelatedNestedLoopJoin(getCluster(), traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // We preserve left edge collation only if batch size == 1
+        if (variablesSet.size() == 1)
+            return super.deriveCollation(nodeTraits, inputTraits);
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            ImmutableList.of(left.replace(RelCollations.EMPTY), right.replace(RelCollations.EMPTY))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Correlated nested loop requires rewindable right edge.
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        ImmutableList.Builder<Pair<RelTraitSet, List<RelTraitSet>>> b = ImmutableList.builder();
+
+        RewindabilityTrait leftRewindability = TraitUtils.rewindability(left);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        outTraits = nodeTraits.replace(RewindabilityTrait.ONE_WAY);
+        leftTraits = left.replace(RewindabilityTrait.ONE_WAY);
+        rightTraits = right.replace(RewindabilityTrait.REWINDABLE);
+
+        b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+
+        if (leftRewindability.rewindable()) {
+            outTraits = nodeTraits.replace(RewindabilityTrait.REWINDABLE);
+            leftTraits = left.replace(RewindabilityTrait.REWINDABLE);
+            rightTraits = right.replace(RewindabilityTrait.REWINDABLE);
+
+            b.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+        }
+
+        return b.build();
+    }
+
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // We preserve left edge collation only if batch size == 1
+        if (variablesSet.size() == 1)
+            return super.passThroughCollation(nodeTraits, inputTraits);
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            ImmutableList.of(left.replace(RelCollations.EMPTY), right.replace(RelCollations.EMPTY))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Correlated nested loop requires rewindable right edge.
+
+        RelTraitSet left = inputTraits.get(0), right = inputTraits.get(1);
+
+        RelTraitSet outTraits, leftTraits, rightTraits;
+
+        RewindabilityTrait rewindability = TraitUtils.rewindability(nodeTraits);
+
+        outTraits = nodeTraits.replace(rewindability);
+        leftTraits = left.replace(rewindability);
+        rightTraits = right.replace(RewindabilityTrait.REWINDABLE);
+
+        return ImmutableList.of(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        // Give it some penalty
+        return super.computeSelfCost(planner, mq).multiplyBy(5);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
index 8e26076..de080cd 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
@@ -31,9 +31,10 @@ import org.apache.calcite.rel.core.Exchange;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.Pair;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.any;
 import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.changeTraits;
 import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.fixTraits;
 
@@ -89,7 +90,12 @@ public class IgniteExchange extends Exchange implements IgniteRel {
         if (!distribution().satisfies(distribution))
             return null;
 
-        return Pair.of(required.replace(distribution()), ImmutableList.of(required.replace(IgniteDistributions.any())));
+        RelTraitSet outTraits = required.replace(distribution())
+            .replace(RewindabilityTrait.ONE_WAY);
+        RelTraitSet inTraits = required.replace(any())
+            .replace(RewindabilityTrait.ONE_WAY);
+
+        return Pair.of(outTraits, ImmutableList.of(inTraits));
     }
 
     /** {@inheritDoc} */
@@ -98,7 +104,12 @@ public class IgniteExchange extends Exchange implements IgniteRel {
 
         childTraits = fixTraits(childTraits);
 
-        return Pair.of(childTraits.replace(distribution()), ImmutableList.of(childTraits.replace(IgniteDistributions.any())));
+        RelTraitSet outTraits = childTraits.replace(distribution())
+            .replace(RewindabilityTrait.ONE_WAY);
+        RelTraitSet inTraits = childTraits.replace(any())
+            .replace(RewindabilityTrait.ONE_WAY);
+
+        return Pair.of(outTraits, ImmutableList.of(inTraits));
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java
index 2915029..d88282e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java
@@ -21,10 +21,14 @@ import java.util.List;
 
 import com.google.common.collect.ImmutableList;
 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.Filter;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.Pair;
 
@@ -82,4 +86,11 @@ public class IgniteFilter extends Filter implements IgniteRel {
 
         return Pair.of(childTraits, ImmutableList.of(childTraits));
     }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        double rowCount = mq.getRowCount(getInput());
+        rowCount = RelMdUtil.addEpsilon(rowCount); // to differ from rel nodes with integrated filter
+        return planner.getCostFactory().makeCost(rowCount, 0, 0);
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java
similarity index 82%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java
index 39c11f4..f918d43 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteIndexScan.java
@@ -38,9 +38,14 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
@@ -59,12 +64,14 @@ import static org.apache.calcite.sql.SqlKind.GREATER_THAN_OR_EQUAL;
 import static org.apache.calcite.sql.SqlKind.LESS_THAN;
 import static org.apache.calcite.sql.SqlKind.LESS_THAN_OR_EQUAL;
 import static org.apache.calcite.sql.SqlKind.OR;
+import static org.apache.ignite.internal.processors.query.calcite.schema.IgniteTableImpl.PK_INDEX_NAME;
 import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.changeTraits;
+import static org.apache.ignite.internal.processors.query.calcite.util.RexUtils.makeCast;
 
 /**
  * Relational operator that returns the contents of a table.
  */
-public class IgniteTableScan extends TableScan implements IgniteRel {
+public class IgniteIndexScan extends TableScan implements IgniteRel {
     /** Supported index operations. */
     public static final Set<SqlKind> TREE_INDEX_COMPARISON =
         EnumSet.of(
@@ -98,13 +105,13 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
      *
      * @param input Serialized representation.
      */
-    public IgniteTableScan(RelInput input) {
+    public IgniteIndexScan(RelInput input) {
         super(changeTraits(input, IgniteConvention.INSTANCE));
+        igniteTbl = getTable().unwrap(IgniteTable.class);
         idxName = input.getString("index");
         cond = input.getExpression("filters");
-        lowerIdxCond = input.getExpressionList("lower");
-        upperIdxCond = input.getExpressionList("upper");
-        igniteTbl = getTable().unwrap(IgniteTable.class);
+        lowerIdxCond = input.get("lower") == null ? ImmutableList.of() : input.getExpressionList("lower");
+        upperIdxCond = input.get("upper") == null ? ImmutableList.of() : input.getExpressionList("upper");
         collation = igniteTbl.getIndex(idxName).collation();
     }
 
@@ -116,7 +123,7 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
      * @param idxName Index name.
      * @param cond Filters for scan.
      */
-    public IgniteTableScan(
+    public IgniteIndexScan(
         RelOptCluster cluster,
         RelTraitSet traits,
         RelOptTable tbl,
@@ -166,7 +173,7 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
             for (RexCall pred : collFldPreds) {
                 RexNode cond = removeCast(pred.operands.get(1));
 
-                assert cond instanceof RexLiteral || cond instanceof RexDynamicParam : cond;
+                assert supports(cond) : cond;
 
                 SqlOperator op = pred.getOperator();
                 switch (op.kind) {
@@ -262,7 +269,7 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
         if (cond == null)
             return false;
 
-        RexCall dnf = ((RexCall)RexUtil.toDnf(getCluster().getRexBuilder(), cond));
+        RexCall dnf = (RexCall)RexUtil.toDnf(getCluster().getRexBuilder(), cond);
 
         if (dnf.isA(OR) && dnf.getOperands().size() > 1) // OR conditions are not supported yet.
             return false;
@@ -280,15 +287,22 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
         leftOp = removeCast(leftOp);
         rightOp = removeCast(rightOp);
 
-        if (leftOp instanceof RexLocalRef && (rightOp instanceof RexLiteral || rightOp instanceof RexDynamicParam))
+        if (leftOp instanceof RexLocalRef && supports(rightOp))
             return leftOp;
-        else if ((leftOp instanceof RexLiteral || leftOp instanceof RexDynamicParam) && rightOp instanceof RexLocalRef)
+        else if (supports(leftOp) && rightOp instanceof RexLocalRef)
             return rightOp;
 
         return null;
     }
 
     /** */
+    private static boolean supports(RexNode op) {
+        return op instanceof RexLiteral
+            || op instanceof RexDynamicParam
+            || op instanceof RexFieldAccess;
+    }
+
+    /** */
     private static boolean refOnTheRight(RexCall predCall) {
         RexNode rightOp = predCall.getOperands().get(1);
 
@@ -320,7 +334,9 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
 
     /** */
     public List<RexNode> buildIndexCondition(Iterable<RexNode> idxCond) {
-        List<RexNode> lowerIdxCond = makeListOfNullLiterals(rowType.getFieldCount());
+        List<RexNode> res = makeListOfNullLiterals(rowType);
+        List<RelDataType> fieldTypes = RelOptUtil.getFieldTypeList(rowType);
+        RexBuilder rexBuilder = getCluster().getRexBuilder();
 
         for (RexNode pred : idxCond) {
             assert pred instanceof RexCall;
@@ -329,22 +345,24 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
             RexLocalRef ref = (RexLocalRef)removeCast(call.operands.get(0));
             RexNode cond = removeCast(call.operands.get(1));
 
-            assert cond instanceof RexLiteral || cond instanceof RexDynamicParam : cond;
+            assert supports(cond) : cond;
 
-            lowerIdxCond.set(ref.getIndex(), cond);
+            res.set(ref.getIndex(), makeCast(rexBuilder, fieldTypes.get(ref.getIndex()), cond));
         }
 
-        return lowerIdxCond;
+        return res;
     }
 
     /** */
-    private List<RexNode> makeListOfNullLiterals(int size) {
-        List<RexNode> list = new ArrayList<>(size);
-        RexNode nullLiteral = getCluster().getRexBuilder()
-            .makeNullLiteral(getCluster().getTypeFactory().createJavaType(Object.class));
-        for (int i = 0; i < size; i++) {
-            list.add(nullLiteral);
-        }
+    private List<RexNode> makeListOfNullLiterals(RelDataType rowType) {
+        assert rowType.isStruct();
+
+        RexBuilder builder = getCluster().getRexBuilder();
+
+        List<RexNode> list = new ArrayList<>(rowType.getFieldCount());
+        for (RelDataTypeField field : rowType.getFieldList())
+            list.add(builder.makeNullLiteral(field.getType()));
+
         return list;
     }
 
@@ -352,10 +370,10 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
     @Override public RelWriter explainTerms(RelWriter pw) {
         super.explainTerms(pw);
         return pw.item("index", idxName )
-            .itemIf("lower", lowerIdxCond, lowerIdxCond != null)
-            .itemIf("upper", upperIdxCond, upperIdxCond != null)
+            .item("collation", collation)
             .itemIf("filters", cond, cond != null)
-            .item("collation", collation);
+            .itemIf("lower", lowerIdxCond, !F.isEmpty(lowerIdxCond))
+            .itemIf("upper", upperIdxCond, !F.isEmpty(upperIdxCond));
     }
 
     /** {@inheritDoc} */
@@ -389,23 +407,21 @@ public class IgniteTableScan extends TableScan implements IgniteRel {
 
     /** {@inheritDoc} */
     @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
-        double rows = estimateRowCount(mq);
+        double tableRows = table.getRowCount() * idxSelectivity;
 
-        return planner.getCostFactory().makeCost(rows, 0, 0);
+        if (!PK_INDEX_NAME.equals(indexName()))
+            tableRows = RelMdUtil.addEpsilon(tableRows);
+
+        return planner.getCostFactory().makeCost(tableRows, 0, 0);
     }
 
     /** {@inheritDoc} */
     @Override public double estimateRowCount(RelMetadataQuery mq) {
-        double rows = table.getRowCount();
-
-        double rowsIn = rows * idxSelectivity;
-        double rowsOut = rowsIn;
+        double rows = table.getRowCount() * idxSelectivity;
 
-        if (cond != null) {
-            Double sel = mq.getSelectivity(this, cond);
-            rowsOut *= sel;
-        }
+        if (cond != null)
+            rows *= mq.getSelectivity(this, cond);
 
-        return rowsIn + rowsOut;
+        return rows;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java
deleted file mode 100644
index 880b8d4..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java
+++ /dev/null
@@ -1,365 +0,0 @@
-/*
- * 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.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import org.apache.calcite.plan.DeriveMode;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-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.rex.RexNode;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
-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.ignite.internal.processors.query.calcite.trait.IgniteDistributions.broadcast;
-import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.hash;
-import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.single;
-import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.fixTraits;
-
-/**
- * Relational expression that combines two relational expressions according to
- * some condition.
- *
- * <p>Each output row has columns from the left and right inputs.
- * The set of output rows is a subset of the cartesian product of the two
- * inputs; precisely which subset depends on the join condition.
- */
-public class IgniteJoin extends Join implements IgniteRel {
-    /**
-     * Creates a Join.
-     *
-     * @param cluster          Cluster
-     * @param traitSet         Trait set
-     * @param left             Left input
-     * @param right            Right input
-     * @param condition        Join condition
-     * @param joinType         Join type
-     * @param variablesSet     Set variables that are set by the
-     *                         LHS and used by the RHS and are not available to
-     *                         nodes above this Join in the tree
-     */
-    public IgniteJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
-        super(cluster, traitSet, left, right, condition, variablesSet, joinType);
-    }
-
-    /** */
-    public IgniteJoin(RelInput input) {
-        this(input.getCluster(),
-            input.getTraitSet().replace(IgniteConvention.INSTANCE),
-            input.getInputs().get(0),
-            input.getInputs().get(1),
-            input.getExpression("condition"),
-            ImmutableSet.copyOf((List<CorrelationId>)input.get("variablesSet")),
-            input.getEnum("joinType", JoinRelType.class));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Join copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
-        return new IgniteJoin(getCluster(), traitSet, left, right, condition, variablesSet, joinType);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
-        return visitor.visit(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelWriter explainTerms(RelWriter pw) {
-        return super.explainTerms(pw)
-            .itemIf("variablesSet", variablesSet.asList(), pw.getDetailLevel() == SqlExplainLevel.ALL_ATTRIBUTES);
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelNode passThrough(RelTraitSet required) {
-        required = fixTraits(required);
-
-        Pair<RelCollation, RelCollation> inCollations = inCollations(TraitUtils.collation(required));
-        if (inCollations == null)
-            return passThrough(required.replace(RelCollations.EMPTY));
-
-        IgniteDistribution toDistr = TraitUtils.distribution(required);
-
-        Set<Pair<RelTraitSet, List<RelTraitSet>>> traits = new HashSet<>();
-
-        RelOptCluster cluster = getCluster();
-        RelTraitSet outTraits, leftTraits, rightTraits;
-
-        RelDistribution.Type distrType = toDistr.getType();
-        switch (distrType) {
-            case BROADCAST_DISTRIBUTED:
-            case SINGLETON:
-                outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(toDistr)
-                    .replace(TraitUtils.collation(required));
-
-                leftTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(toDistr)
-                    .replace(inCollations.left);
-
-                rightTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(toDistr)
-                    .replace(inCollations.right);
-
-                traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-
-                break;
-            case HASH_DISTRIBUTED:
-            case RANDOM_DISTRIBUTED:
-                if (joinType != LEFT && joinType != RIGHT && (joinType != INNER || F.isEmpty(joinInfo.pairs())))
-                    return passThrough(required.replace(IgniteDistributions.single()));
-
-                DistributionFunction function = distrType == HASH_DISTRIBUTED
-                    ? toDistr.function()
-                    : DistributionFunction.HashDistribution.INSTANCE;
-
-                IgniteDistribution outDistr = hash(joinInfo.leftKeys, function);
-
-                if (distrType == HASH_DISTRIBUTED && !outDistr.satisfies(toDistr))
-                    return passThrough(required.replace(IgniteDistributions.single()));
-
-                outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(outDistr)
-                    .replace(TraitUtils.collation(required));
-
-                leftTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(hash(joinInfo.leftKeys, function))
-                    .replace(inCollations.left);
-                rightTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(hash(joinInfo.rightKeys, function))
-                    .replace(inCollations.right);
-
-                traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-
-                if (joinType == INNER || joinType == LEFT) {
-                    leftTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                        .replace(hash(joinInfo.leftKeys, function))
-                        .replace(inCollations.left);
-                    rightTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                        .replace(broadcast())
-                        .replace(inCollations.right);
-
-                    traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-                }
-
-                if (joinType == INNER || joinType == RIGHT) {
-                    leftTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                        .replace(broadcast())
-                        .replace(inCollations.left);
-                    rightTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                        .replace(hash(joinInfo.rightKeys, function))
-                        .replace(inCollations.right);
-
-                    traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-                }
-
-             default:
-                break;
-        }
-
-        List<RelNode> nodes = createNodes(traits);
-
-        RelOptPlanner planner = getCluster().getPlanner();
-        for (int i = 1; i < nodes.size(); i++)
-            planner.register(nodes.get(i), this);
-
-        return F.first(nodes);
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<RelNode> derive(List<List<RelTraitSet>> inTraits) {
-        Set<Pair<RelTraitSet, List<RelTraitSet>>> traits = new HashSet<>();
-
-        RelOptCluster cluster = getCluster();
-
-        for (Pair<RelTraitSet, RelTraitSet> inTraits0 : inputTraits(inTraits)) {
-            RelCollation leftCollation = TraitUtils.collation(inTraits0.left);
-            RelCollation rightCollation = TraitUtils.collation(inTraits0.right);
-
-            IgniteDistribution leftDistr = TraitUtils.distribution(inTraits0.left);
-            IgniteDistribution rightDistr = TraitUtils.distribution(inTraits0.right);
-
-            RelCollation outCollation = outCollation(leftCollation, rightCollation);
-
-            RelTraitSet outTraits, leftTraits, rightTraits;
-
-            outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                    .replace(outCollation)
-                    .replace(single());
-
-            leftTraits = inTraits0.left.replace(single());
-            rightTraits = inTraits0.right.replace(single());
-
-            traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-
-            outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                .replace(outCollation)
-                .replace(broadcast());
-
-            leftTraits = inTraits0.left.replace(broadcast());
-            rightTraits = inTraits0.right.replace(broadcast());
-
-            traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-
-            if (joinType == LEFT || joinType == RIGHT || (joinType == INNER && !F.isEmpty(joinInfo.pairs()))) {
-                Set<DistributionFunction> functions = new HashSet<>();
-
-                if (leftDistr.getType() == RelDistribution.Type.HASH_DISTRIBUTED
-                    && Objects.equals(joinInfo.leftKeys, leftDistr.getKeys()))
-                    functions.add(leftDistr.function());
-
-                if (rightDistr.getType() == RelDistribution.Type.HASH_DISTRIBUTED
-                    && Objects.equals(joinInfo.rightKeys, rightDistr.getKeys()))
-                    functions.add(rightDistr.function());
-
-                functions.add(DistributionFunction.HashDistribution.INSTANCE);
-
-                for (DistributionFunction factory : functions) {
-                    outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                        .replace(outCollation)
-                        .replace(hash(joinInfo.leftKeys, factory));
-
-                    leftTraits = inTraits0.left.replace(hash(joinInfo.leftKeys, factory));
-                    rightTraits = inTraits0.right.replace(hash(joinInfo.rightKeys, factory));
-
-                    traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-
-                    if (joinType == INNER || joinType == LEFT) {
-                        leftTraits = inTraits0.left.replace(hash(joinInfo.leftKeys, factory));
-                        rightTraits = inTraits0.right.replace(broadcast());
-
-                        traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-                    }
-
-                    if (joinType == INNER || joinType == RIGHT) {
-                        leftTraits = inTraits0.left.replace(broadcast());
-                        rightTraits = inTraits0.right.replace(hash(joinInfo.rightKeys, factory));
-
-                        traits.add(Pair.of(outTraits, ImmutableList.of(leftTraits, rightTraits)));
-                    }
-                }
-            }
-        }
-
-        return createNodes(traits);
-    }
-
-    /** {@inheritDoc} */
-    @Override public DeriveMode getDeriveMode() {
-        return DeriveMode.OMAKASE;
-    }
-
-    /** */
-    private RelCollation outCollation(RelCollation left, RelCollation right) {
-        switch (joinType) {
-            case SEMI:
-            case ANTI:
-            case INNER:
-            case LEFT:
-                return left;
-            case RIGHT:
-            case FULL:
-                for (RelFieldCollation field : left.getFieldCollations()) {
-                    if (RelFieldCollation.NullDirection.LAST != field.nullDirection)
-                        return RelCollations.EMPTY;
-                }
-
-                return left;
-        }
-
-        return RelCollations.EMPTY;
-    }
-
-    /** */
-    private Pair<RelCollation, RelCollation> inCollations(RelCollation out) {
-        switch (joinType) {
-            case SEMI:
-            case ANTI:
-            case INNER:
-            case LEFT:
-                return Pair.of(out, RelCollations.EMPTY);
-            case RIGHT:
-            case FULL:
-                for (RelFieldCollation field : out.getFieldCollations()) {
-                    if (RelFieldCollation.NullDirection.LAST != field.nullDirection)
-                        return null;
-                }
-
-                return Pair.of(out, RelCollations.EMPTY);
-        }
-
-        return null;
-    }
-
-    /** */
-    private Collection<Pair<RelTraitSet, RelTraitSet>> inputTraits(List<List<RelTraitSet>> inputTraits) {
-        assert !F.isEmpty(inputTraits) && inputTraits.size() == 2;
-        int size = inputTraits.get(0).size() * inputTraits.get(1).size();
-        Set<Pair<RelTraitSet, RelTraitSet>> pairs = U.newHashSet(size);
-        for (RelTraitSet left : inputTraits.get(0)) {
-            for (RelTraitSet right : inputTraits.get(1))
-                pairs.add(Pair.of(fixTraits(left), fixTraits(right)));
-        }
-        return pairs;
-    }
-
-    /** */
-    private List<RelNode> createNodes(Collection<Pair<RelTraitSet, List<RelTraitSet>>> traits) {
-        List<RelNode> res = new ArrayList<>(traits.size());
-        for (Pair<RelTraitSet, List<RelTraitSet>> p : traits) {
-            int size = getInputs().size();
-            assert size == p.right.size();
-
-            List<RelNode> list = new ArrayList<>(size);
-            for (int i = 0; i < size; i++)
-                list.add(RelOptRule.convert(getInput(i), p.right.get(i)));
-
-            res.add(copy(p.left, list));
-        }
-
-        return res;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteNestedLoopJoin.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteNestedLoopJoin.java
new file mode 100644
index 0000000..68e1418
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteNestedLoopJoin.java
@@ -0,0 +1,87 @@
+/*
+ * 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.Set;
+
+import com.google.common.collect.ImmutableSet;
+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.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * Relational expression that combines two relational expressions according to
+ * some condition.
+ *
+ * <p>Each output row has columns from the left and right inputs.
+ * The set of output rows is a subset of the cartesian product of the two
+ * inputs; precisely which subset depends on the join condition.
+ */
+public class IgniteNestedLoopJoin extends AbstractIgniteNestedLoopJoin {
+    /**
+     * Creates a Join.
+     *
+     * @param cluster          Cluster
+     * @param traitSet         Trait set
+     * @param left             Left input
+     * @param right            Right input
+     * @param condition        Join condition
+     * @param joinType         Join type
+     * @param variablesSet     Set variables that are set by the
+     *                         LHS and used by the RHS and are not available to
+     *                         nodes above this Join in the tree
+     */
+    public IgniteNestedLoopJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+        super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    /** */
+    public IgniteNestedLoopJoin(RelInput input) {
+        this(input.getCluster(),
+            input.getTraitSet().replace(IgniteConvention.INSTANCE),
+            input.getInputs().get(0),
+            input.getInputs().get(1),
+            input.getExpression("condition"),
+            ImmutableSet.copyOf(Commons.transform(input.getIntegerList("variablesSet"), CorrelationId::new)),
+            input.getEnum("joinType", JoinRelType.class));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Join copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+        return new IgniteNestedLoopJoin(getCluster(), traitSet, left, right, condition, variablesSet, joinType);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(10);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java
index eb6ba7e..5864bcf 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java
@@ -28,7 +28,6 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -36,20 +35,24 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.mapping.Mappings;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitsAwareIgniteRel;
 
+import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.hash;
+import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.single;
 import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.changeTraits;
-import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.fixTraits;
 
 /**
  * Relational expression that computes a set of
  * 'select expressions' from its input relational expression.
  */
-public class IgniteProject extends Project implements IgniteRel {
+public class IgniteProject extends Project implements TraitsAwareIgniteRel {
     /**
      * Creates a Project.
      *
@@ -78,108 +81,136 @@ public class IgniteProject extends Project implements IgniteRel {
     }
 
     /** {@inheritDoc} */
-    @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughTraits(RelTraitSet required) {
-        required = fixTraits(required);
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // The node is rewindable if its input is rewindable.
 
-        IgniteDistribution distr;
-        RelCollation collation;
+        RelTraitSet in = inputTraits.get(0);
+        RewindabilityTrait rewindability = TraitUtils.rewindability(nodeTraits);
 
-        if ((distr = inDistribution(TraitUtils.distribution(required))) == null)
-            return passThroughTraits(required.replace(IgniteDistributions.single()));
-
-        if ((collation = inCollation(TraitUtils.collation(required))) == null)
-            return passThroughTraits(required.replace(RelCollations.EMPTY));
-
-        return Pair.of(required, ImmutableList.of(required.replace(distr).replace(collation)));
+        return ImmutableList.of(Pair.of(nodeTraits, ImmutableList.of(in.replace(rewindability))));
     }
 
     /** {@inheritDoc} */
-    @Override public Pair<RelTraitSet, List<RelTraitSet>> deriveTraits(RelTraitSet childTraits, int childId) {
-        assert childId == 0;
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // All distribution types except hash distribution are propagated as is.
+        // In case of hash distribution we need to project distribution keys.
+        // In case one of distribution keys is erased by projection result distribution
+        // becomes default single since we cannot calculate required input distribution.
 
-        childTraits = fixTraits(childTraits);
+        RelTraitSet in = inputTraits.get(0);
+        IgniteDistribution distribution = TraitUtils.distribution(nodeTraits);
 
-        IgniteDistribution distr = outDistribution(TraitUtils.distribution(childTraits));
-        RelCollation collation = outCollation(TraitUtils.collation(childTraits));
+        if (distribution.getType() != HASH_DISTRIBUTED)
+            return ImmutableList.of(Pair.of(nodeTraits, ImmutableList.of(in.replace(distribution))));
 
-        return Pair.of(childTraits.replace(distr).replace(collation), ImmutableList.of(childTraits));
-    }
+        Mappings.TargetMapping mapping = getPartialMapping(
+            input.getRowType().getFieldCount(), getProjects());
 
-    /** */
-    private IgniteDistribution outDistribution(IgniteDistribution inDistr) {
-        if (inDistr.getType() == RelDistribution.Type.HASH_DISTRIBUTED) {
-            Mappings.TargetMapping mapping = Project.getPartialMapping(
-                input.getRowType().getFieldCount(), getProjects());
+        ImmutableIntList keys = distribution.getKeys();
+        List<Integer> srcKeys = new ArrayList<>(keys.size());
 
-            return inDistr.apply(mapping);
-        }
+        for (int key : keys) {
+            int src = mapping.getSourceOpt(key);
 
-        return inDistr;
-    }
+            if (src == -1)
+                break;
 
-    /** */
-    private IgniteDistribution inDistribution(IgniteDistribution outDistr) {
-        if (outDistr.getType() == RelDistribution.Type.HASH_DISTRIBUTED) {
-            Mappings.TargetMapping mapping = Project.getPartialMapping(
-                input.getRowType().getFieldCount(), getProjects());
+            srcKeys.add(src);
+        }
 
-            List<Integer> inKeys = new ArrayList<>(outDistr.getKeys().size());
+        if (srcKeys.size() == keys.size()) {
+            return ImmutableList.of(Pair.of(nodeTraits, ImmutableList.of(in.replace(hash(srcKeys, distribution.function())))));
+        }
 
-            for (int key : outDistr.getKeys()) {
-                int src = mapping.getSourceOpt(key);
-                if (src == -1)
-                    return null;
+        return ImmutableList.of(Pair.of(nodeTraits.replace(single()), ImmutableList.of(in.replace(single()))));
+    }
 
-                inKeys.add(src);
-            }
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // The code below projects required collation. In case we cannot calculate required source collation
+        // (e.g. one of required sorted fields is result of a function call), input and output collations are erased.
 
-            return IgniteDistributions.hash(inKeys);
-        }
+        RelTraitSet in = inputTraits.get(0);
 
-        return outDistr;
-    }
+        List<RelFieldCollation> fieldCollations = TraitUtils.collation(nodeTraits).getFieldCollations();
 
-    /** */
-    private RelCollation outCollation(RelCollation inCollation) {
-        if (inCollation.getFieldCollations().isEmpty())
-            return RelCollations.EMPTY;
+        if (fieldCollations.isEmpty())
+            return ImmutableList.of(Pair.of(nodeTraits, ImmutableList.of(in.replace(RelCollations.EMPTY))));
 
         Map<Integer, Integer> targets = new HashMap<>();
         for (Ord<RexNode> project : Ord.zip(getProjects())) {
             if (project.e instanceof RexInputRef)
-                targets.putIfAbsent(((RexInputRef)project.e).getIndex(), project.i);
+                targets.putIfAbsent(project.i, ((RexInputRef)project.e).getIndex());
         }
 
-        List<RelFieldCollation> outFieldCollations = new ArrayList<>();
-        for (RelFieldCollation inFieldCollation : inCollation.getFieldCollations()) {
+        List<RelFieldCollation> inFieldCollations = new ArrayList<>();
+        for (RelFieldCollation inFieldCollation : fieldCollations) {
             Integer newIndex = targets.get(inFieldCollation.getFieldIndex());
-            if (newIndex != null)
-                outFieldCollations.add(inFieldCollation.withFieldIndex(newIndex));
+            if (newIndex == null)
+                break;
+            else
+                inFieldCollations.add(inFieldCollation.withFieldIndex(newIndex));
         }
 
-        return RelCollations.of(outFieldCollations);
+        if (inFieldCollations.size() == fieldCollations.size())
+            return ImmutableList.of(Pair.of(nodeTraits, ImmutableList.of(in.replace(RelCollations.of(inFieldCollations)))));
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY), ImmutableList.of(in.replace(RelCollations.EMPTY))));
     }
 
-    /** */
-    private RelCollation inCollation(RelCollation outCollation) {
-        if (outCollation.getFieldCollations().isEmpty())
-            return RelCollations.EMPTY;
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // The node is rewindable if its input is rewindable.
+
+        RelTraitSet in = inputTraits.get(0);
+        RewindabilityTrait rewindability = TraitUtils.rewindability(in);
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(rewindability), ImmutableList.of(in)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // All distribution types except hash distribution are propagated as is.
+        // In case of hash distribution we need to project distribution keys.
+        // In case one of distribution keys is erased by projection result distribution
+        // becomes random since we cannot determine where data is without erased key.
+
+        RelTraitSet in = inputTraits.get(0);
+        IgniteDistribution distribution = TraitUtils.distribution(in);
+
+        if (distribution.getType() == HASH_DISTRIBUTED) {
+            Mappings.TargetMapping mapping = Project.getPartialMapping(
+                input.getRowType().getFieldCount(), getProjects());
+
+            return ImmutableList.of(Pair.of(nodeTraits.replace(distribution.apply(mapping)), ImmutableList.of(in)));
+        }
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(distribution), ImmutableList.of(in)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // The code below projects input collation.
+
+        RelTraitSet in = inputTraits.get(0);
+        RelCollation collation = TraitUtils.collation(in);
+
+        if (collation.getFieldCollations().isEmpty())
+            return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY), ImmutableList.of(in)));
 
         Map<Integer, Integer> targets = new HashMap<>();
         for (Ord<RexNode> project : Ord.zip(getProjects())) {
             if (project.e instanceof RexInputRef)
-                targets.putIfAbsent(project.i, ((RexInputRef)project.e).getIndex());
+                targets.putIfAbsent(((RexInputRef)project.e).getIndex(), project.i);
         }
 
-        List<RelFieldCollation> inFieldCollations = new ArrayList<>();
-        for (RelFieldCollation inFieldCollation : outCollation.getFieldCollations()) {
+        List<RelFieldCollation> outFieldCollations = new ArrayList<>();
+        for (RelFieldCollation inFieldCollation : collation.getFieldCollations()) {
             Integer newIndex = targets.get(inFieldCollation.getFieldIndex());
-            if (newIndex == null)
-                return null;
-
-            inFieldCollations.add(inFieldCollation.withFieldIndex(newIndex));
+            if (newIndex != null)
+                outFieldCollations.add(inFieldCollation.withFieldIndex(newIndex));
         }
 
-        return RelCollations.of(inFieldCollations);
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.of(outFieldCollations)), ImmutableList.of(in)));
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java
index 8e73318..97ff53a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.PhysicalNode;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.util.Pair;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 
 /**
@@ -52,6 +53,13 @@ public interface IgniteRel extends PhysicalNode {
         return TraitUtils.collation(getTraitSet());
     }
 
+    /**
+     * @return Node rewindability.
+     */
+    default RewindabilityTrait rewindability() {
+        return TraitUtils.rewindability(getTraitSet());
+    }
+
     /** {@inheritDoc} */
     @Override default Pair<RelTraitSet, List<RelTraitSet>> passThroughTraits(
         RelTraitSet required) {
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 6447cc4..5dcca0d 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
@@ -44,12 +44,17 @@ public interface IgniteRelVisitor<T> {
     /**
      * See {@link IgniteRelVisitor#visit(IgniteRel)}
      */
-    T visit(IgniteJoin rel);
+    T visit(IgniteNestedLoopJoin rel);
 
     /**
      * See {@link IgniteRelVisitor#visit(IgniteRel)}
      */
-    T visit(IgniteTableScan rel);
+    T visit(IgniteCorrelatedNestedLoopJoin rel);
+
+    /**
+     * See {@link IgniteRelVisitor#visit(IgniteRel)}
+     */
+    T visit(IgniteIndexScan rel);
 
     /**
      * See {@link IgniteRelVisitor#visit(IgniteRel)}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTrimExchange.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTrimExchange.java
index d8816a9..47a9867 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTrimExchange.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTrimExchange.java
@@ -21,11 +21,14 @@ import java.util.List;
 
 import com.google.common.collect.ImmutableList;
 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.RelDistribution;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Exchange;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.Pair;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
@@ -93,4 +96,10 @@ public class IgniteTrimExchange extends Exchange implements IgniteRel {
 
         return Pair.of(childTraits.replace(distribution()), ImmutableList.of(childTraits.replace(IgniteDistributions.broadcast())));
     }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        double rowCount = mq.getRowCount(this);
+        return planner.getCostFactory().makeCost(rowCount, rowCount, 0);
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteUnionAll.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteUnionAll.java
index f9fcd04..f6c92da 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteUnionAll.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteUnionAll.java
@@ -17,29 +17,29 @@
 
 package org.apache.ignite.internal.processors.query.calcite.rel;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import org.apache.calcite.plan.DeriveMode;
+import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.util.Pair;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitsAwareIgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
-import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.any;
-
 /**
  *
  */
-public class IgniteUnionAll extends Union implements IgniteRel {
+public class IgniteUnionAll extends Union implements TraitsAwareIgniteRel {
     /** */
     public IgniteUnionAll(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs) {
         super(cluster, traits, inputs, true);
@@ -66,46 +66,70 @@ public class IgniteUnionAll extends Union implements IgniteRel {
     }
 
     /** {@inheritDoc} */
-    @Override public RelNode passThrough(RelTraitSet required) {
-        IgniteDistribution toDistr = TraitUtils.distribution(required);
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Union node requires the same traits from all its inputs.
+
+        RewindabilityTrait rewindability = TraitUtils.rewindability(nodeTraits);
+
+        return ImmutableList.of(Pair.of(nodeTraits,
+            Commons.transform(inputTraits, t -> t.replace(rewindability))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Union node requires the same traits from all its inputs.
+
+        IgniteDistribution distribution = TraitUtils.distribution(nodeTraits);
+
+        return ImmutableList.of(Pair.of(nodeTraits,
+            Commons.transform(inputTraits, t -> t.replace(distribution))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Union node erases collation. TODO union all using merge sort algorythm
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            Commons.transform(inputTraits, t -> t.replace(RelCollations.EMPTY))));
+    }
 
-        // Union erases collation and only distribution trait can be passed through.
-        // So that, it's no use to pass ANY distribution.
-        if (toDistr == any())
-            return null;
+    /** {@inheritDoc} */
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Union node requires the same traits from all its inputs.
 
-        RelTraitSet traits = getCluster().traitSetOf(IgniteConvention.INSTANCE)
-            .replace(toDistr);
+        boolean rewindable = inputTraits.stream()
+            .map(TraitUtils::rewindability)
+            .allMatch(RewindabilityTrait::rewindable);
 
-        List<RelNode> inputs0 = Commons.transform(inputs,
-            input -> RelOptRule.convert(input, traits));
+        if (rewindable)
+            return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
 
-        return copy(traits, inputs0);
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
+            Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
     }
 
     /** {@inheritDoc} */
-    @Override public List<RelNode> derive(List<List<RelTraitSet>> inputTraits) {
-        RelOptCluster cluster = getCluster();
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Union node requires the same traits from all its inputs.
 
-        Set<RelTraitSet> traits = inputTraits.stream()
-            .flatMap(List::stream)
+        Set<IgniteDistribution> distributions = inputTraits.stream()
             .map(TraitUtils::distribution)
-            .filter(d -> d != any())
-            .map(distr -> cluster.traitSetOf(IgniteConvention.INSTANCE).replace(distr))
             .collect(Collectors.toSet());
 
-        List<RelNode> res = new ArrayList<>(traits.size());
-        for (RelTraitSet traits0 : traits) {
-            List<RelNode> inputs0 = Commons.transform(inputs,
-                input -> RelOptRule.convert(input, traits0));
-            res.add(copy(traits0, inputs0));
-        }
+        ImmutableList.Builder<Pair<RelTraitSet, List<RelTraitSet>>> b = ImmutableList.builder();
 
-        return res;
+        for (IgniteDistribution distribution : distributions)
+            b.add(Pair.of(nodeTraits.replace(distribution),
+                Commons.transform(inputTraits, t -> t.replace(distribution))));
+
+        return b.build();
     }
 
     /** {@inheritDoc} */
-    @Override public DeriveMode getDeriveMode() {
-        return DeriveMode.OMAKASE;
+    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits, List<RelTraitSet> inputTraits) {
+        // Union node erases collation. TODO union all using merge sort algorythm
+
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            Commons.transform(inputTraits, t -> t.replace(RelCollations.EMPTY))));
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/AbstractIgniteConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/AbstractIgniteConverterRule.java
index 404ea4c..efdef2a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/AbstractIgniteConverterRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/AbstractIgniteConverterRule.java
@@ -29,7 +29,12 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
 public abstract class AbstractIgniteConverterRule<T extends RelNode> extends ConverterRule {
     /** */
     protected AbstractIgniteConverterRule(Class<T> clazz) {
-        super(clazz, Convention.NONE, IgniteConvention.INSTANCE, clazz.getName() + "Converter");
+        this(clazz, clazz.getName() + "Converter");
+    }
+
+    /** */
+    protected AbstractIgniteConverterRule(Class<T> clazz, String descriptionPreffix) {
+        super(clazz, Convention.NONE, IgniteConvention.INSTANCE, descriptionPreffix);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/CorrelatedNestedLoopJoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/CorrelatedNestedLoopJoinConverterRule.java
new file mode 100644
index 0000000..1a5cdf0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/CorrelatedNestedLoopJoinConverterRule.java
@@ -0,0 +1,130 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+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.core.CorrelationId;
+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.RexBuilder;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
+
+/** */
+public class CorrelatedNestedLoopJoinConverterRule extends AbstractIgniteConverterRule<LogicalJoin> {
+    /** */
+    public static final RelOptRule INSTANCE = new CorrelatedNestedLoopJoinConverterRule(1);
+
+    /** */
+    public static final RelOptRule INSTANCE_BATCHED = new CorrelatedNestedLoopJoinConverterRule(100);
+
+    /** */
+    private final int batchSize;
+
+    /** */
+    public CorrelatedNestedLoopJoinConverterRule(int batchSize) {
+        super(LogicalJoin.class, "CorrelatedNestedLoopConverter");
+
+        assert batchSize >= 0;
+
+        this.batchSize = batchSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean matches(RelOptRuleCall call) {
+        LogicalJoin join = call.rel(0);
+        JoinRelType joinType = join.getJoinType();
+        return joinType == JoinRelType.INNER; // TODO LEFT, SEMI, ANTI
+    }
+
+    /** {@inheritDoc} */
+    @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, LogicalJoin rel) {
+        final int leftFieldCount = rel.getLeft().getRowType().getFieldCount();
+        final RelOptCluster cluster = rel.getCluster();
+        final RexBuilder rexBuilder = cluster.getRexBuilder();
+        final RelBuilder relBuilder = relBuilderFactory.create(rel.getCluster(), null);
+
+        final Set<CorrelationId> correlationIds = new HashSet<>();
+        final ArrayList<RexNode> corrVar = new ArrayList<>();
+
+        for (int i = 0; i < batchSize; i++) {
+            CorrelationId correlationId = cluster.createCorrel();
+            correlationIds.add(correlationId);
+            corrVar.add(rexBuilder.makeCorrel(rel.getLeft().getRowType(), correlationId));
+        }
+
+        // Generate first condition
+        final RexNode condition = rel.getCondition().accept(new RexShuttle() {
+            @Override public RexNode visitInputRef(RexInputRef input) {
+                int field = input.getIndex();
+                if (field >= leftFieldCount)
+                    return rexBuilder.makeInputRef(input.getType(), input.getIndex() - leftFieldCount);
+
+                return rexBuilder.makeFieldAccess(corrVar.get(0), field);
+            }
+        });
+
+        List<RexNode> conditionList = new ArrayList<>();
+        conditionList.add(condition);
+
+        // Add batchSize-1 other conditions
+        for (int i = 1; i < batchSize; i++) {
+            final int corrIndex = i;
+            final RexNode condition2 = condition.accept(new RexShuttle() {
+                @Override public RexNode visitCorrelVariable(RexCorrelVariable variable) {
+                    return corrVar.get(corrIndex);
+                }
+            });
+            conditionList.add(condition2);
+        }
+
+        // Push a filter with batchSize disjunctions
+        relBuilder.push(rel.getRight()).filter(relBuilder.or(conditionList));
+        RelNode right = relBuilder.build();
+
+        JoinRelType joinType = rel.getJoinType();
+
+        RelTraitSet outTraits = cluster.traitSetOf(IgniteConvention.INSTANCE);
+        RelTraitSet leftInTraits = cluster.traitSetOf(IgniteConvention.INSTANCE);
+        RelTraitSet rightInTraits = cluster.traitSetOf(IgniteConvention.INSTANCE)
+            .replace(RewindabilityTrait.REWINDABLE);
+
+        RelNode left = convert(rel.getLeft(), leftInTraits);
+        right = convert(right, rightInTraits);
+
+        return new IgniteCorrelatedNestedLoopJoin(cluster, outTraits, left, right, rel.getCondition(), correlationIds, joinType);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/RegisterIndexRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ExposeIndexRule.java
similarity index 56%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/RegisterIndexRule.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ExposeIndexRule.java
index afecdf9..3e84500 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/RegisterIndexRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ExposeIndexRule.java
@@ -17,17 +17,18 @@
 
 package org.apache.ignite.internal.processors.query.calcite.rule;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
+import java.util.Set;
 
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
 import org.apache.ignite.internal.util.typedef.F;
 
@@ -36,37 +37,47 @@ import static org.apache.ignite.internal.processors.query.calcite.schema.IgniteT
 /**
  *
  */
-public class RegisterIndexRule extends RelOptRule {
-    /** Instance. */
-    public static final RelOptRule INSTANCE = new RegisterIndexRule();
-
-    /**
-     * Constructor.
-     */
-    private RegisterIndexRule() {
-        super(operandJ(IgniteTableScan.class, null,
-            scan -> PK_INDEX_NAME.equals(scan.indexName()) && scan.condition() == null, any()));
+public class ExposeIndexRule extends RelOptRule {
+    /** */
+    public static final RelOptRule INSTANCE = new ExposeIndexRule();
+
+    /** */
+    public ExposeIndexRule() {
+        super(operandJ(IgniteIndexScan.class, null, ExposeIndexRule::preMatch, any()));
+    }
+
+    /** */
+    private static boolean preMatch(IgniteIndexScan scan) {
+        return scan.igniteTable().indexes().size() > 1     // has indexes to expose
+            && PK_INDEX_NAME.equals(scan.indexName())      // is PK index scan
+            && scan.condition() == null;                   // was not modified by PushFilterIntoScanRule
     }
 
     /** {@inheritDoc} */
     @Override public void onMatch(RelOptRuleCall call) {
-        IgniteTableScan rel = call.rel(0);
-        RelOptCluster cluster = rel.getCluster();
-        RelOptTable table = rel.getTable();
-        IgniteTable igniteTable = rel.igniteTable();
+        IgniteIndexScan scan = call.rel(0);
+        RelOptCluster cluster = scan.getCluster();
 
-        List<IgniteTableScan> indexes = igniteTable.indexes().keySet().stream()
-            .filter(idxName -> !PK_INDEX_NAME.equals(idxName))
-            .map(idxName -> igniteTable.toRel(cluster, table, idxName))
-            .collect(Collectors.toList());
+        RelOptTable optTable = scan.getTable();
+        IgniteTable igniteTable = scan.igniteTable();
 
-        if (indexes.isEmpty())
-            return;
+        assert PK_INDEX_NAME.equals(scan.indexName());
 
-        Map<RelNode, RelNode> equivMap = new HashMap<>();
+        Set<String> indexNames = igniteTable.indexes().keySet();
 
+        assert indexNames.size() > 1;
+
+        List<IgniteIndexScan> indexes = new ArrayList<>();
+        for (String idxName : indexNames) {
+            if (PK_INDEX_NAME.equals(idxName))
+                continue;
+
+            indexes.add(igniteTable.toRel(cluster, optTable, idxName));
+        }
+
+        Map<RelNode, RelNode> equivMap = new HashMap<>();
         for (int i = 1; i < indexes.size(); i++)
-            equivMap.put(indexes.get(i), rel);
+            equivMap.put(indexes.get(i), scan);
 
         call.transformTo(F.first(indexes), equivMap);
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/NestedLoopJoinConverterRule.java
similarity index 82%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverterRule.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/NestedLoopJoinConverterRule.java
index ca46017..c8d6d18 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverterRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/NestedLoopJoinConverterRule.java
@@ -26,20 +26,20 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
 
 /**
  * Ignite Join converter.
  */
-public class JoinConverterRule extends AbstractIgniteConverterRule<LogicalJoin> {
+public class NestedLoopJoinConverterRule extends AbstractIgniteConverterRule<LogicalJoin> {
     /** */
-    public static final RelOptRule INSTANCE = new JoinConverterRule();
+    public static final RelOptRule INSTANCE = new NestedLoopJoinConverterRule();
 
     /**
      * Creates a converter.
      */
-    public JoinConverterRule() {
-        super(LogicalJoin.class);
+    public NestedLoopJoinConverterRule() {
+        super(LogicalJoin.class, "NestedLoopJoinConverter");
     }
 
     /** {@inheritDoc} */
@@ -51,6 +51,6 @@ public class JoinConverterRule extends AbstractIgniteConverterRule<LogicalJoin>
         RelNode left = convert(rel.getLeft(), leftInTraits);
         RelNode right = convert(rel.getRight(), rightInTraits);
 
-        return new IgniteJoin(cluster, outTraits, left, right, rel.getCondition(), rel.getVariablesSet(), rel.getJoinType());
+        return new IgniteNestedLoopJoin(cluster, outTraits, left, right, rel.getCondition(), rel.getVariablesSet(), rel.getJoinType());
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/PushFilterIntoScanRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/PushFilterIntoScanRule.java
index 06dcd19..49d1751 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/PushFilterIntoScanRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/PushFilterIntoScanRule.java
@@ -16,29 +16,35 @@
  */
 package org.apache.ignite.internal.processors.query.calcite.rule;
 
-import com.google.common.collect.ImmutableList;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitor;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.util.typedef.F;
 
+import static org.apache.ignite.internal.processors.query.calcite.util.RexUtils.builder;
+import static org.apache.ignite.internal.processors.query.calcite.util.RexUtils.simplifier;
+
 /**
  * Rule that pushes filter into the scan. This might be useful for index range scans.
  */
 public class PushFilterIntoScanRule extends RelOptRule {
     /** Instance. */
     public static final PushFilterIntoScanRule FILTER_INTO_SCAN =
-        new PushFilterIntoScanRule(Filter.class, "IgniteFilterIntoScanRule");
+        new PushFilterIntoScanRule(LogicalFilter.class, "IgniteFilterIntoScanRule");
 
     /**
      * Constructor.
@@ -48,42 +54,50 @@ public class PushFilterIntoScanRule extends RelOptRule {
      */
     private PushFilterIntoScanRule(Class<? extends RelNode> clazz, String desc) {
         super(operand(clazz,
-            operand(IgniteTableScan.class, none())),
+            operand(IgniteIndexScan.class, none())),
             RelFactories.LOGICAL_BUILDER,
             desc);
     }
 
     /** {@inheritDoc} */
     @Override public void onMatch(RelOptRuleCall call) {
-        Filter filter = call.rel(0);
-        IgniteTableScan scan = call.rel(1);
+        LogicalFilter filter = call.rel(0);
+        IgniteIndexScan scan = call.rel(1);
+
+        RelOptCluster cluster = scan.getCluster();
+        RelMetadataQuery mq = call.getMetadataQuery();
 
         RexNode cond = filter.getCondition();
 
-        // We need to replace RexInputRef with RexLocalRef because TableScan doesn't have inputs.
-        RexVisitor<RexNode> inputRefReplacer = new InputRefReplacer();
-        cond = cond.accept(inputRefReplacer);
+        RexSimplify simplifier = simplifier(cluster);
 
-        RexNode cond0 = scan.condition();
+        // Let's remove from the condition common with the scan filter parts.
+        cond = simplifier
+            .withPredicates(mq.getPulledUpPredicates(scan))
+            .simplifyUnknownAsFalse(cond);
 
-        if (cond0 != null) {
-            ImmutableList<RexNode> nodes = RexUtil.flattenAnd(ImmutableList.of(cond0));
-            if (nodes.contains(cond))
-                return;
+        // We need to replace RexInputRef with RexLocalRef because TableScan doesn't have inputs.
+        cond = cond.accept(new InputRefReplacer());
 
-            RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
-            cond = RexUtil.composeConjunction(rexBuilder, F.concat(true, cond, nodes));
-        }
+        // Combine the condition with the scan filter.
+        cond = RexUtil.composeConjunction(builder(cluster), F.asList(cond, scan.condition()));
+
+        // Final simplification. We need several phases because simplifier sometimes
+        // (see RexSimplify.simplifyGenericNode) leaves UNKNOWN nodes that can be
+        // eliminated on next simplify attempt. We limit attempts count not to break
+        // planning performance on complex condition.
+        Set<RexNode> nodes = new HashSet<>();
+        while (nodes.add(cond) && nodes.size() < 3)
+            cond = simplifier.simplifyUnknownAsFalse(cond);
 
         call.transformTo(
-            new IgniteTableScan(scan.getCluster(), scan.getTraitSet(), scan.getTable(), scan.indexName(), cond));
+            new IgniteIndexScan(cluster, scan.getTraitSet(), scan.getTable(), scan.indexName(), cond));
     }
 
     /** Visitor for replacing input refs to local refs. We need it for proper plan serialization. */
     private static class InputRefReplacer extends RexShuttle {
         @Override public RexNode visitInputRef(RexInputRef inputRef) {
-            int idx = inputRef.getIndex();
-            return new RexLocalRef(idx, inputRef.getType());
+            return new RexLocalRef(inputRef.getIndex(), inputRef.getType());
         }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ValuesConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ValuesConverterRule.java
index 9e4bddb..d89319b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ValuesConverterRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ValuesConverterRule.java
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 
 import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions.broadcast;
 
@@ -46,7 +47,9 @@ public class ValuesConverterRule extends AbstractIgniteConverterRule<LogicalValu
     /** {@inheritDoc} */
     @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, LogicalValues rel) {
         RelOptCluster cluster = rel.getCluster();
-        RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE).replace(broadcast());
+        RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE)
+            .replace(broadcast())
+            .replace(RewindabilityTrait.REWINDABLE);
 
         return new IgniteValues(cluster, rel.getRowType(), rel.getTuples(), traits);
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalFilterProjectTransposeRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalFilterProjectTransposeRule.java
index 3410df3..9415185 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalFilterProjectTransposeRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalFilterProjectTransposeRule.java
@@ -32,7 +32,7 @@ public class LogicalFilterProjectTransposeRule extends FilterProjectTransposeRul
 
     /** */
     public LogicalFilterProjectTransposeRule() {
-        super(LogicalFilter.class, LogicalProject.class, true, true,
+        super(LogicalFilter.class, filter -> true, LogicalProject.class, project -> true, true, true,
             RelFactories.LOGICAL_BUILDER);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java
index 2dd202d..40b2bb9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/logical/LogicalOrToUnionRule.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.calcite.rule.logical;
 
 import java.util.List;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
@@ -28,11 +27,9 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.RelBuilder;
 
 /**
@@ -87,22 +84,14 @@ public class LogicalOrToUnionRule extends RelOptRule {
      * @return UnionAll expression.
      */
     private RelNode createUnionAll(RelOptCluster cluster, RelNode input, RexNode op1, RexNode op2) {
-        final RelBuilder builder = relBuilderFactory.create(cluster, null);
-        final RexBuilder rexBuilder = cluster.getRexBuilder();
+        RelBuilder relBldr = relBuilderFactory.create(cluster, null);
 
-        builder.push(input).filter(op1);
-        builder.push(input).filter(
-            builder.and(op2,
-                // LNNVL is used here. We must treat 'null' values as valid.
-                rexBuilder.makeCall(op1.getType(), SqlStdOperatorTable.CASE,
-                    ImmutableList.of(
-                        rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, op1),
-                        RexUtil.not(op1),
-                        rexBuilder.makeLiteral(true))
-                )
-            )
-        );
-
-        return builder.union(true).build();
+        return relBldr
+            .push(input).filter(op1)
+            .push(input).filter(
+                relBldr.and(op2,
+                    relBldr.or(relBldr.isNull(op1), relBldr.not(op1))))
+            .union(true)
+            .build();
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteIndex.java
index d746ec5..5267ce3 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteIndex.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteIndex.java
@@ -17,6 +17,7 @@
 package org.apache.ignite.internal.processors.query.calcite.schema;
 
 import java.util.function.Predicate;
+import java.util.function.Supplier;
 
 import org.apache.calcite.rel.RelCollation;
 import org.apache.ignite.internal.processors.query.GridIndex;
@@ -72,8 +73,8 @@ public class IgniteIndex {
     public <Row> Iterable<Row> scan(
         ExecutionContext<Row> execCtx,
         Predicate<Row> filters,
-        Row lowerIdxConditions,
-        Row upperIdxConditions) {
+        Supplier<Row> lowerIdxConditions,
+        Supplier<Row> upperIdxConditions) {
         return new IndexScan<>(execCtx, this, filters, lowerIdxConditions, upperIdxConditions);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
index 00a9e74..ea67e81 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
@@ -26,7 +26,7 @@ import org.apache.calcite.schema.ProjectableFilterableTable;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
 import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 
 /**
@@ -45,7 +45,7 @@ public interface IgniteTable extends TranslatableTable, ProjectableFilterableTab
      * @param relOptTbl Table.
      * @return Table relational expression.
      */
-    IgniteTableScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName);
+    IgniteIndexScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName);
 
     /**
      * Returns nodes mapping.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
index 1b83abf..aac4d0e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
@@ -48,8 +48,9 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid
 import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
 import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -115,9 +116,10 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteTableScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName) {
+    @Override public IgniteIndexScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName) {
         RelTraitSet traitSet = cluster.traitSetOf(IgniteConvention.INSTANCE)
-            .replace(distribution());
+            .replace(distribution())
+            .replace(RewindabilityTrait.REWINDABLE);
 
         IgniteIndex idx = getIndex(idxName);
 
@@ -126,7 +128,7 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable {
 
         traitSet = traitSet.replace(idx.collation());
 
-        return new IgniteTableScan(cluster, traitSet, relOptTbl, idxName, null);
+        return new IgniteIndexScan(cluster, traitSet, relOptTbl, idxName, null);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
index 1f9eeea..9110fa4 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
@@ -57,7 +57,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 /**
  *
  */
-@SuppressWarnings({"AssignmentOrReturnOfFieldWithMutableType", "rawtypes", "unchecked"})
+@SuppressWarnings({"AssignmentOrReturnOfFieldWithMutableType", "rawtypes"})
 public class TableDescriptorImpl extends NullInitializerExpressionFactory
     implements TableDescriptor {
     /** */
@@ -144,8 +144,7 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory
             }
         }
 
-        Map<String, ColumnDescriptor> descriptorsMap = U.newHashMap(fields.size() + 2);
-
+        Map<String, ColumnDescriptor> descriptorsMap = U.newHashMap(descriptors.size());
         for (ColumnDescriptor descriptor : descriptors)
             descriptorsMap.put(descriptor.name(), descriptor);
 
@@ -187,12 +186,18 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory
 
     /** {@inheritDoc} */
     @Override public <Row> Row toRow(ExecutionContext<Row> ectx, CacheDataRow row, RowHandler.RowFactory<Row> factory) throws IgniteCheckedException {
-        Object[] res = new Object[descriptors.length];
+        RowHandler<Row> handler = factory.handler();
+
+        assert handler == ectx.rowHandler();
+
+        Row res = factory.create();
+
+        assert handler.columnCount(res) == descriptors.length;
 
         for (int i = 0; i < descriptors.length; i++)
-            res[i] = descriptors[i].value(ectx, cctx, row);
+            handler.set(i, res, descriptors[i].value(ectx, cctx, row));
 
-        return factory.create(res);
+        return res;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java
index c974e0b..b3e9c52 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java
@@ -21,7 +21,7 @@ import java.util.List;
 import java.util.UUID;
 
 /** */
-public final class AllNodes implements Destination {
+public final class AllNodes<Row> implements Destination<Row> {
     /** */
     private final List<UUID> nodes;
 
@@ -31,7 +31,7 @@ public final class AllNodes implements Destination {
     }
 
     /** {@inheritDoc} */
-    @Override public List<UUID> targets(Object row) {
+    @Override public List<UUID> targets(Row row) {
         return nodes;
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
index 69f0a06..60a1a4f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
@@ -23,12 +23,12 @@ import java.util.UUID;
 /**
  * Determines where to send a row to.
  */
-public interface Destination {
+public interface Destination<Row> {
     /**
      * @param row Row.
      * @return Target nodes list for specific row.
      */
-    List<UUID> targets(Object row);
+    List<UUID> targets(Row row);
 
     /**
      * @return All target nodes.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
index 8db896d..8920015 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
@@ -28,6 +28,8 @@ import java.util.function.ToIntFunction;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.util.ImmutableIntList;
+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.metadata.NodesMapping;
 import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
 import org.apache.ignite.internal.util.typedef.F;
@@ -52,22 +54,26 @@ public abstract class DistributionFunction implements Serializable {
     /**
      * Creates a destination based on this function algorithm, given nodes mapping and given distribution keys.
      *
+     *
+     * @param ctx Execution context.
      * @param partitionService Affinity function source.
      * @param mapping Target mapping.
      * @param keys Distribution keys.
      * @return Destination function.
      */
-    public abstract Destination destination(PartitionService partitionService, NodesMapping mapping, ImmutableIntList keys);
+    public abstract <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+        NodesMapping mapping, ImmutableIntList keys);
 
     /**
      * Creates a partition.
      *
+     * @param ctx Execution context.
      * @param partitionService Affinity function source.
-     * @param partitionsCount Expected partitions count.
      * @param keys Distribution keys.
      * @return Partition function.
      */
-    public ToIntFunction<Object> partitionFunction(PartitionService partitionService, int partitionsCount, ImmutableIntList keys) {
+    public <Row> ToIntFunction<Row> partitionFunction(ExecutionContext<Row> ctx, PartitionService partitionService,
+        ImmutableIntList keys) {
         throw new UnsupportedOperationException();
     }
 
@@ -118,7 +124,8 @@ public abstract class DistributionFunction implements Serializable {
         }
 
         /** {@inheritDoc} */
-        @Override public Destination destination(PartitionService partitionService, NodesMapping m, ImmutableIntList k) {
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+            NodesMapping m, ImmutableIntList k) {
             throw new AssertionError();
         }
 
@@ -139,10 +146,11 @@ public abstract class DistributionFunction implements Serializable {
         }
 
         /** {@inheritDoc} */
-        @Override public Destination destination(PartitionService partitionService, NodesMapping m, ImmutableIntList k) {
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+            NodesMapping m, ImmutableIntList k) {
             assert m != null && !F.isEmpty(m.nodes());
 
-            return new AllNodes(m.nodes());
+            return new AllNodes<>(m.nodes());
         }
 
         /** */
@@ -162,10 +170,11 @@ public abstract class DistributionFunction implements Serializable {
         }
 
         /** {@inheritDoc} */
-        @Override public Destination destination(PartitionService partitionService, NodesMapping m, ImmutableIntList k) {
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+            NodesMapping m, ImmutableIntList k) {
             assert m != null && !F.isEmpty(m.nodes());
 
-            return new RandomNode(m.nodes());
+            return new RandomNode<>(m.nodes());
         }
 
         /** */
@@ -186,11 +195,12 @@ public abstract class DistributionFunction implements Serializable {
         }
 
         /** {@inheritDoc} */
-        @Override public Destination destination(PartitionService partitionService, NodesMapping m, ImmutableIntList k) {
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+            NodesMapping m, ImmutableIntList k) {
             if (m == null || m.nodes() == null || m.nodes().size() != 1)
                 throw new AssertionError();
 
-            return new AllNodes(Collections
+            return new AllNodes<>(Collections
                 .singletonList(Objects
                     .requireNonNull(F
                         .first(m.nodes()))));
@@ -212,7 +222,8 @@ public abstract class DistributionFunction implements Serializable {
         }
 
         /** {@inheritDoc} */
-        @Override public Destination destination(PartitionService partitionService, NodesMapping m, ImmutableIntList k) {
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+            NodesMapping m, ImmutableIntList k) {
             assert m != null && !F.isEmpty(m.assignments()) && !k.isEmpty();
 
             List<List<UUID>> assignments = m.assignments();
@@ -222,12 +233,14 @@ public abstract class DistributionFunction implements Serializable {
                     assert F.isEmpty(assignment) || assignment.size() == 1;
             }
 
-            return new Partitioned(m.nodes(), assignments, partitionFunction(partitionService, assignments.size(), k));
+            return new Partitioned<>(assignments, partitionFunction(ctx, partitionService, k));
         }
 
         /** {@inheritDoc} */
-        @Override public ToIntFunction<Object> partitionFunction(PartitionService partitionService, int partitionsCount, ImmutableIntList k) {
-            return DistributionFunction.rowToPart(partitionService.partitionFunction(CU.UNDEFINED_CACHE_ID), partitionsCount, k.toIntArray());
+        @Override public <Row> ToIntFunction<Row> partitionFunction(ExecutionContext<Row> ctx,
+            PartitionService partitionService, ImmutableIntList k) {
+            return DistributionFunction.rowToPart(partitionService.partitionFunction(CU.UNDEFINED_CACHE_ID),
+                ctx.partitionsCount(), k.toIntArray(), ctx.rowHandler());
         }
 
         /** */
@@ -264,7 +277,8 @@ public abstract class DistributionFunction implements Serializable {
         }
 
         /** {@inheritDoc} */
-        @Override public Destination destination(PartitionService partitionService, NodesMapping m, ImmutableIntList k) {
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, PartitionService partitionService,
+            NodesMapping m, ImmutableIntList k) {
             assert m != null && !F.isEmpty(m.assignments()) && k.size() == 1;
 
             List<List<UUID>> assignments = m.assignments();
@@ -274,12 +288,16 @@ public abstract class DistributionFunction implements Serializable {
                     assert F.isEmpty(assignment) || assignment.size() == 1;
             }
 
-            return new Partitioned(m.nodes(), assignments, partitionFunction(partitionService, assignments.size(), k));
+            return new Partitioned<>(assignments, partitionFunction(ctx, partitionService, k));
         }
 
         /** {@inheritDoc} */
-        @Override public ToIntFunction<Object> partitionFunction(PartitionService partitionService, int partitionsCount, ImmutableIntList k) {
-            return DistributionFunction.rowToPart(partitionService.partitionFunction(cacheId), partitionsCount, k.toIntArray());
+        @Override public <Row> ToIntFunction<Row> partitionFunction(ExecutionContext<Row> ctx,
+            PartitionService partitionService, ImmutableIntList k) {
+            assert k.size() == 1;
+
+            return DistributionFunction.rowToPart(partitionService.partitionFunction(cacheId),
+                ctx.partitionsCount(), k.toIntArray(), ctx.rowHandler());
         }
 
         /** {@inheritDoc} */
@@ -289,17 +307,15 @@ public abstract class DistributionFunction implements Serializable {
     }
 
     /** */
-    private static ToIntFunction<Object> rowToPart(ToIntFunction<Object> keyToPart, int parts, int[] keys) {
+    private static <Row> ToIntFunction<Row> rowToPart(ToIntFunction<Object> keyToPart, int parts, int[] keys, RowHandler<Row> hndlr) {
         return r -> {
-            Object[] row = (Object[]) r;
-
-            if (F.isEmpty(row))
+            if (F.isEmpty(keys))
                 return 0;
 
-            int hash = keyToPart.applyAsInt(row[keys[0]]);
+            int hash = keyToPart.applyAsInt(hndlr.get(keys[0], r));
 
             for (int i = 1; i < keys.length; i++)
-                hash = 31 * hash + keyToPart.applyAsInt(row[keys[i]]);
+                hash = 31 * hash + keyToPart.applyAsInt(hndlr.get(keys[i], r));
 
             return U.safeAbs(hash) % parts;
         };
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java
index 3fd0aed..56e65c4 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java
@@ -17,35 +17,42 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 import java.util.UUID;
 import java.util.function.ToIntFunction;
 
-/** */
-public final class Partitioned implements Destination {
-    /** */
-    private final List<UUID> nodes;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
+/** */
+public final class Partitioned<Row> implements Destination<Row> {
     /** */
     private final List<List<UUID>> assignments;
 
     /** */
-    private final ToIntFunction<Object> partFun;
+    private final ToIntFunction<Row> partFun;
 
     /** */
-    public Partitioned(List<UUID> nodes, List<List<UUID>> assignments, ToIntFunction<Object> partFun) {
-        this.nodes = nodes;
+    public Partitioned(List<List<UUID>> assignments, ToIntFunction<Row> partFun) {
         this.assignments = assignments;
         this.partFun = partFun;
     }
 
     /** {@inheritDoc} */
-    @Override public List<UUID> targets(Object row) {
+    @Override public List<UUID> targets(Row row) {
         return assignments.get(partFun.applyAsInt(row));
     }
 
     /** {@inheritDoc} */
     @Override public List<UUID> targets() {
-        return nodes;
+        Set<UUID> targets = U.newHashSet(assignments.size());
+        for (List<UUID> assignment : assignments) {
+            if (!F.isEmpty(assignment))
+                targets.addAll(assignment);
+        }
+
+        return new ArrayList<>(targets);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java
index ab9cb9e..9e0f438 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java
@@ -23,7 +23,7 @@ import java.util.Random;
 import java.util.UUID;
 
 /** */
-public final class RandomNode implements Destination {
+public final class RandomNode<Row> implements Destination<Row> {
     /** */
     private final Random random;
 
@@ -38,7 +38,7 @@ public final class RandomNode implements Destination {
     }
 
     /** {@inheritDoc} */
-    @Override public List<UUID> targets(Object row) {
+    @Override public List<UUID> targets(Row row) {
         return Collections.singletonList(nodes.get(random.nextInt(nodes.size())));
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RelFactory.java
similarity index 68%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RelFactory.java
index 69f0a06..aea9408 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RelFactory.java
@@ -18,20 +18,18 @@
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
 import java.util.List;
-import java.util.UUID;
 
-/**
- * Determines where to send a row to.
- */
-public interface Destination {
-    /**
-     * @param row Row.
-     * @return Target nodes list for specific row.
-     */
-    List<UUID> targets(Object row);
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
 
+/** */
+public interface RelFactory {
     /**
-     * @return All target nodes.
+     * Creates a node for given traits combination.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return Relational node for given traits combination.
      */
-    List<UUID> targets();
+    RelNode create(RelTraitSet outTraits, List<RelTraitSet> inTraits);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RewindabilityTrait.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RewindabilityTrait.java
new file mode 100644
index 0000000..cfc9284
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RewindabilityTrait.java
@@ -0,0 +1,102 @@
+/*
+ * 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.trait;
+
+import org.apache.calcite.plan.RelMultipleTrait;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.jetbrains.annotations.NotNull;
+
+/** */
+public class RewindabilityTrait implements RelMultipleTrait {
+    /** */
+    public static final RewindabilityTrait ONE_WAY = canonize(new RewindabilityTrait(false));
+
+    /** */
+    public static final RewindabilityTrait REWINDABLE = canonize(new RewindabilityTrait(true));
+
+    /** */
+    private final boolean rewindable;
+
+    /** */
+    private RewindabilityTrait(boolean rewindable) {
+        this.rewindable = rewindable;
+    }
+
+    /** */
+    public boolean rewindable() {
+        return rewindable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isTop() {
+        return !rewindable();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(@NotNull RelMultipleTrait o) {
+        RewindabilityTrait that = (RewindabilityTrait)o;
+        return Boolean.compare(that.rewindable, rewindable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (!(o instanceof RewindabilityTrait))
+            return false;
+        return compareTo((RewindabilityTrait)o) == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return (rewindable ? 1 : 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return rewindable ? "rewindable" : "one-way";
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelTraitDef<RewindabilityTrait> getTraitDef() {
+        return RewindabilityTraitDef.INSTANCE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean satisfies(RelTrait trait) {
+        if (this == trait)
+            return true;
+
+        if (!(trait instanceof RewindabilityTrait))
+            return false;
+
+        RewindabilityTrait trait0 = (RewindabilityTrait)trait;
+
+        return !trait0.rewindable() || rewindable();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(RelOptPlanner planner) {
+        // no-op
+    }
+
+    /** */
+    private static RewindabilityTrait canonize(RewindabilityTrait trait) {
+        return RewindabilityTraitDef.INSTANCE.canonize(trait);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RewindabilityTraitDef.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RewindabilityTraitDef.java
new file mode 100644
index 0000000..68398b3
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RewindabilityTraitDef.java
@@ -0,0 +1,52 @@
+/*
+ * 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.trait;
+
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.rel.RelNode;
+
+public class RewindabilityTraitDef extends RelTraitDef<RewindabilityTrait> {
+    /** */
+    public static final RewindabilityTraitDef INSTANCE = new RewindabilityTraitDef();
+
+    /** {@inheritDoc} */
+    @Override public Class<RewindabilityTrait> getTraitClass() {
+        return RewindabilityTrait.class;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSimpleName() {
+        return "rewindability";
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelNode convert(RelOptPlanner planner, RelNode rel, RewindabilityTrait toTrait, boolean allowInfiniteCostConverters) {
+        return TraitUtils.convertRewindability(planner, toTrait, rel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canConvert(RelOptPlanner planner, RewindabilityTrait fromTrait, RewindabilityTrait toTrait) {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RewindabilityTrait getDefault() {
+        return RewindabilityTrait.ONE_WAY;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitUtils.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitUtils.java
index e9ad7af..602db23 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitUtils.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitUtils.java
@@ -17,9 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
@@ -37,10 +40,13 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexLiteral;
 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.rel.IgniteConvention;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+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.rel.IgniteTrimExchange;
+import org.apache.ignite.internal.util.typedef.F;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.calcite.rel.RelDistribution.Type.BROADCAST_DISTRIBUTED;
@@ -53,7 +59,7 @@ import static org.apache.ignite.internal.processors.query.calcite.trait.IgniteDi
  */
 public class TraitUtils {
     /** */
-    public static RelNode enforce(RelNode rel, RelTraitSet toTraits) {
+    @Nullable public static RelNode enforce(RelNode rel, RelTraitSet toTraits) {
         RelOptPlanner planner = rel.getCluster().getPlanner();
         RelTraitSet fromTraits = rel.getTraitSet();
         if (!fromTraits.satisfies(toTraits)) {
@@ -79,8 +85,9 @@ public class TraitUtils {
         return rel;
     }
 
+    /** */
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private static RelNode convertTrait(RelOptPlanner planner, RelTrait fromTrait, RelTrait toTrait, RelNode rel) {
+    @Nullable private static RelNode convertTrait(RelOptPlanner planner, RelTrait fromTrait, RelTrait toTrait, RelNode rel) {
         assert fromTrait.getTraitDef() == toTrait.getTraitDef();
 
         RelTraitDef converter = fromTrait.getTraitDef();
@@ -89,8 +96,12 @@ public class TraitUtils {
             return convertCollation(planner, (RelCollation)toTrait, rel);
         else if (converter == DistributionTraitDef.INSTANCE)
             return convertDistribution(planner, (IgniteDistribution)toTrait, rel);
-        else
+        else if (converter == RewindabilityTraitDef.INSTANCE)
+            return convertRewindability(planner, (RewindabilityTrait)toTrait, rel);
+        else if (converter.canConvert(planner, fromTrait, toTrait))
             return converter.convert(planner, rel, toTrait, true);
+
+        return null;
     }
 
     /** */
@@ -114,17 +125,28 @@ public class TraitUtils {
 
         RelNode result;
         IgniteDistribution fromTrait = distribution(rel.getTraitSet());
+        RelTraitSet traits = rel.getTraitSet().replace(toTrait);
+
         if (fromTrait.getType() == BROADCAST_DISTRIBUTED && toTrait.getType() == HASH_DISTRIBUTED)
-            result = new IgniteTrimExchange(rel.getCluster(), rel.getTraitSet().replace(toTrait), rel, toTrait);
+            result = new IgniteTrimExchange(rel.getCluster(), traits, rel, toTrait);
         else {
-            result = new IgniteExchange(rel.getCluster(), rel.getTraitSet().replace(toTrait),
-                RelOptRule.convert(rel, any()), toTrait);
+            traits = traits.replace(RewindabilityTrait.ONE_WAY);
+            result = new IgniteExchange(rel.getCluster(), traits, RelOptRule.convert(rel, any()), toTrait);
         }
 
         return planner.register(result, rel);
     }
 
     /** */
+    @Nullable public static RelNode convertRewindability(RelOptPlanner planner,
+        RewindabilityTrait toTrait, RelNode rel) {
+        if (toTrait.rewindable() && !rewindability(rel.getTraitSet()).rewindable())
+            return null; // TODO IndexSpoon
+
+        return rel;
+    }
+
+    /** */
     public static RelTraitSet fixTraits(RelTraitSet traits) {
         if (distribution(traits) == any())
             traits = traits.replace(single());
@@ -132,15 +154,43 @@ public class TraitUtils {
         return traits.replace(IgniteConvention.INSTANCE);
     }
 
+    /** */
+    public static IgniteDistribution distribution(RelNode rel) {
+        return rel instanceof IgniteRel
+            ? ((IgniteRel)rel).distribution()
+            : distribution(rel.getTraitSet());
+    }
+
+    /** */
     public static IgniteDistribution distribution(RelTraitSet traits) {
         return traits.getTrait(DistributionTraitDef.INSTANCE);
     }
 
+    /** */
+    public static RelCollation collation(RelNode rel) {
+        return rel instanceof IgniteRel
+            ? ((IgniteRel)rel).collation()
+            : collation(rel.getTraitSet());
+    }
+
+    /** */
     public static RelCollation collation(RelTraitSet traits) {
         return traits.getTrait(RelCollationTraitDef.INSTANCE);
     }
 
     /** */
+    public static RewindabilityTrait rewindability(RelNode rel) {
+        return rel instanceof IgniteRel
+            ? ((IgniteRel)rel).rewindability()
+            : rewindability(rel.getTraitSet());
+    }
+
+    /** */
+    public static RewindabilityTrait rewindability(RelTraitSet traits) {
+        return traits.getTrait(RewindabilityTraitDef.INSTANCE);
+    }
+
+    /** */
     public static RelInput changeTraits(RelInput input, RelTrait... traits) {
         RelTraitSet traitSet = input.getTraitSet();
 
@@ -244,4 +294,42 @@ public class TraitUtils {
             }
         };
     }
+
+    /**
+     * Replaces input traits into a set of source traits combinations.
+     */
+    public static Collection<List<RelTraitSet>> inputTraits(List<List<RelTraitSet>> inTraits) {
+        assert !F.isEmpty(inTraits);
+
+        try {
+            return fill(inTraits, ImmutableSet.builder(), ImmutableList.builder(), 0).build();
+        }
+        catch (Util.FoundOne e) {
+            return Collections.emptySet();
+        }
+    }
+
+    /** */
+    private static ImmutableSet.Builder<List<RelTraitSet>> fill(List<List<RelTraitSet>> in,
+        ImmutableSet.Builder<List<RelTraitSet>> out, ImmutableList.Builder<RelTraitSet> template, int srcIdx) {
+        if (srcIdx < in.size()) {
+            List<RelTraitSet> sourceTraits = in.get(srcIdx);
+
+            if (F.isEmpty(sourceTraits))
+                // see a special case in OptimizeTask.RelNodeOptTask#execute
+                throw Util.FoundOne.NULL;
+
+            for (RelTraitSet traits : sourceTraits) {
+                ImmutableList.Builder<RelTraitSet> newTemplate = ImmutableList.<RelTraitSet>builder()
+                    .addAll(template.build())
+                    .add(traits);
+
+                fill(in, out, newTemplate, srcIdx + 1);
+            }
+        }
+        else
+            out.add(template.build());
+
+        return out;
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java
new file mode 100644
index 0000000..2e2096d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsAwareIgniteRel.java
@@ -0,0 +1,153 @@
+/*
+ * 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.trait;
+
+import java.util.List;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.DeriveMode;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public interface TraitsAwareIgniteRel extends IgniteRel {
+    /** {@inheritDoc} */
+    @Override public default RelNode passThrough(RelTraitSet required) {
+        List<RelNode> nodes = TraitsPropagationContext.forPassingThrough(this, required)
+            .propagate(this::passThroughCollation)
+            .propagate(this::passThroughDistribution)
+            .propagate(this::passThroughRewindability)
+            .nodes(this::createNode);
+
+        if (U.assertionsEnabled()) {
+            RelNode first = F.first(nodes);
+
+            if (first != null) {
+                RelTraitSet traits = first.getTraitSet();
+
+                for (int i = 1; i < nodes.size(); i++) {
+                    if (!traits.equals(nodes.get(i).getTraitSet()))
+                        throw new AssertionError("All produced nodes must have equal traits. [nodes=" + nodes + "]");
+                }
+            }
+        }
+
+        RelOptPlanner planner = getCluster().getPlanner();
+        for (int i = 1; i < nodes.size(); i++)
+            planner.register(nodes.get(i), this);
+
+        return F.first(nodes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public default List<RelNode> derive(List<List<RelTraitSet>> inTraits) {
+        return TraitsPropagationContext.forDerivation(this, inTraits)
+            .propagate(this::deriveCollation)
+            .propagate(this::deriveDistribution)
+            .propagate(this::deriveRewindability)
+            .nodes(this::createNode);
+    }
+
+    /** {@inheritDoc} */
+    @Override default Pair<RelTraitSet, List<RelTraitSet>> passThroughTraits(RelTraitSet required) {
+        throw new RuntimeException(getClass().getName() + "#passThroughTraits() is not implemented.");
+    }
+
+    /** {@inheritDoc} */
+    @Override default Pair<RelTraitSet, List<RelTraitSet>> deriveTraits(RelTraitSet childTraits, int childId) {
+        throw new RuntimeException(getClass().getName() + "#deriveTraits() is not implemented.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public default DeriveMode getDeriveMode() {
+        return DeriveMode.OMAKASE;
+    }
+
+    /**
+     * Creates a node for given traits combination.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return Relational node for given traits combination.
+     */
+    default RelNode createNode(RelTraitSet outTraits, List<RelTraitSet> inTraits) {
+        return copy(outTraits, Commons.transform(Ord.zip(inTraits),
+            o -> RelOptRule.convert(getInput(o.i), o.e)));
+    }
+
+    /**
+     * Propagates rewindability trait in up-to-bottom manner.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return List of possible input-output traits combinations.
+     */
+    List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughRewindability(RelTraitSet outTraits, List<RelTraitSet> inTraits);
+
+    /**
+     * Propagates distribution trait in up-to-bottom manner.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return List of possible input-output traits combinations.
+     */
+    List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughDistribution(RelTraitSet outTraits, List<RelTraitSet> inTraits);
+
+    /**
+     * Propagates collation trait in up-to-bottom manner.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return List of possible input-output traits combinations.
+     */
+    List<Pair<RelTraitSet, List<RelTraitSet>>> passThroughCollation(RelTraitSet outTraits, List<RelTraitSet> inTraits);
+
+    /**
+     * Propagates rewindability trait in bottom-up manner.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return List of possible input-output traits combinations.
+     */
+    List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(RelTraitSet outTraits, List<RelTraitSet> inTraits);
+
+    /**
+     * Propagates distribution trait in bottom-up manner.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return List of possible input-output traits combinations.
+     */
+    List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(RelTraitSet outTraits, List<RelTraitSet> inTraits);
+
+    /**
+     * Propagates collation trait in bottom-up manner.
+     *
+     * @param outTraits Relational node output traits.
+     * @param inTraits Relational node input traits.
+     * @return List of possible input-output traits combinations.
+     */
+    List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet outTraits, List<RelTraitSet> inTraits);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsPropagationContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsPropagationContext.java
new file mode 100644
index 0000000..df5c579
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/TraitsPropagationContext.java
@@ -0,0 +1,100 @@
+/*
+ * 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.trait;
+
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.fixTraits;
+import static org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils.inputTraits;
... 1646 lines suppressed ...