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 2019/12/13 13:03:46 UTC

[ignite] 01/01: IGNITE-12248: Apache Calcite based query execution engine. Initial commit.

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

commit 1958c3e542b8b0d8a9491678f5270cd53f56abd1
Author: Igor Seliverstov <gv...@gmail.com>
AuthorDate: Fri Dec 13 16:02:54 2019 +0300

    IGNITE-12248: Apache Calcite based query execution engine.
    Initial commit.
---
 modules/calcite/README.txt                         |   35 +
 modules/calcite/pom.xml                            |  112 ++
 .../calcite/interpreter/InterpreterUtils.java      |   28 +
 .../apache/calcite/plan/volcano/VolcanoUtils.java  |   34 +
 .../query/calcite/CalciteQueryProcessor.java       |  167 +++
 .../query/calcite/cluster/MappingServiceImpl.java  |  134 +++
 .../query/calcite/exchange/EndMarker.java          |   33 +
 .../query/calcite/exchange/ExchangeProcessor.java  |   36 +
 .../processors/query/calcite/exchange/Inbox.java   |   70 ++
 .../processors/query/calcite/exchange/Outbox.java  |  163 +++
 .../query/calcite/exec/AbstractNode.java           |   44 +
 .../query/calcite/exec/ConsumerNode.java           |   75 ++
 .../processors/query/calcite/exec/FilterNode.java  |   47 +
 .../processors/query/calcite/exec/Implementor.java |  174 +++
 .../processors/query/calcite/exec/JoinNode.java    |  101 ++
 .../processors/query/calcite/exec/Node.java        |   27 +
 .../processors/query/calcite/exec/ProjectNode.java |   47 +
 .../query/calcite/exec/ScalarFactory.java          |  151 +++
 .../processors/query/calcite/exec/ScanNode.java    |   70 ++
 .../processors/query/calcite/exec/SingleNode.java  |   33 +
 .../processors/query/calcite/exec/Sink.java        |   38 +
 .../processors/query/calcite/exec/Source.java      |   24 +
 .../query/calcite/metadata/FragmentInfo.java       |   75 ++
 .../metadata/IgniteMdDerivedDistribution.java      |  149 +++
 .../calcite/metadata/IgniteMdDistribution.java     |  123 ++
 .../calcite/metadata/IgniteMdFragmentInfo.java     |  110 ++
 .../query/calcite/metadata/IgniteMetadata.java     |   68 ++
 .../calcite/metadata/LocationMappingException.java |   26 +
 .../query/calcite/metadata/MappingService.java     |   28 +
 .../query/calcite/metadata/NodesMapping.java       |  196 ++++
 .../metadata/OptimisticPlanningException.java      |   35 +
 .../query/calcite/metadata/RelMetadataQueryEx.java |  107 ++
 .../query/calcite/prepare/ContextValue.java        |   44 +
 .../query/calcite/prepare/DataContextImpl.java     |   67 ++
 .../calcite/prepare/DistributedExecution.java      |  110 ++
 .../query/calcite/prepare/IgnitePlanner.java       |  436 +++++++
 .../query/calcite/prepare/IgniteSqlValidator.java  |   55 +
 .../query/calcite/prepare/PlannerContext.java      |  194 ++++
 .../query/calcite/prepare/PlannerPhase.java        |   60 +
 .../query/calcite/prepare/PlannerType.java         |   26 +
 .../processors/query/calcite/prepare/Query.java    |   73 ++
 .../query/calcite/prepare/QueryExecution.java      |   28 +
 .../query/calcite/rel/IgniteConvention.java        |   46 +
 .../query/calcite/rel/IgniteExchange.java          |   40 +
 .../processors/query/calcite/rel/IgniteFilter.java |   40 +
 .../processors/query/calcite/rel/IgniteJoin.java   |   43 +
 .../query/calcite/rel/IgniteProject.java           |   42 +
 .../query/calcite/rel/IgniteReceiver.java          |   61 +
 .../processors/query/calcite/rel/IgniteRel.java    |   26 +
 .../query/calcite/rel/IgniteRelVisitor.java        |   38 +
 .../processors/query/calcite/rel/IgniteSender.java |   57 +
 .../query/calcite/rel/IgniteTableScan.java         |   41 +
 .../processors/query/calcite/rel/RelOp.java        |   27 +
 .../query/calcite/rule/FilterConverter.java        |   62 +
 .../query/calcite/rule/IgniteConverter.java        |   74 ++
 .../query/calcite/rule/JoinConverter.java          |   68 ++
 .../query/calcite/rule/ProjectConverter.java       |   63 +
 .../query/calcite/rule/TableScanConverter.java     |   44 +
 .../query/calcite/schema/CalciteSchemaHolder.java  |   69 ++
 .../query/calcite/schema/IgniteSchema.java         |   88 ++
 .../query/calcite/schema/IgniteTable.java          |  137 +++
 .../processors/query/calcite/serialize/Graph.java  |   78 ++
 .../query/calcite/serialize/GraphNode.java         |   25 +
 .../serialize/expression/CallExpression.java       |   40 +
 .../expression/DynamicParamExpression.java         |   37 +
 .../serialize/expression/ExpImplementor.java       |   32 +
 .../serialize/expression/ExpToRexTranslator.java   |  100 ++
 .../calcite/serialize/expression/Expression.java   |   26 +
 .../serialize/expression/InputRefExpression.java   |   37 +
 .../serialize/expression/LiteralExpression.java    |   37 +
 .../serialize/expression/LocalRefExpression.java   |   37 +
 .../serialize/expression/RexToExpTranslator.java   |  101 ++
 .../serialize/relation/ConversionContext.java      |   36 +
 .../calcite/serialize/relation/FilterNode.java     |   58 +
 .../serialize/relation/GraphToRelConverter.java    |   90 ++
 .../query/calcite/serialize/relation/JoinNode.java |   66 ++
 .../calcite/serialize/relation/ProjectNode.java    |   62 +
 .../calcite/serialize/relation/ReceiverNode.java   |   53 +
 .../query/calcite/serialize/relation/RelGraph.java |   25 +
 .../calcite/serialize/relation/RelGraphNode.java   |   38 +
 .../serialize/relation/RelToGraphConverter.java    |  109 ++
 .../calcite/serialize/relation/SenderNode.java     |   56 +
 .../serialize/relation/SerializedTraits.java       |   65 ++
 .../calcite/serialize/relation/TableScanNode.java  |   44 +
 .../query/calcite/serialize/type/DataType.java     |   32 +
 .../query/calcite/serialize/type/SimpleType.java   |   59 +
 .../query/calcite/serialize/type/StructType.java   |   51 +
 .../processors/query/calcite/splitter/Edge.java    |   46 +
 .../query/calcite/splitter/Fragment.java           |   92 ++
 .../query/calcite/splitter/QueryPlan.java          |   76 ++
 .../query/calcite/splitter/RelSource.java          |   47 +
 .../query/calcite/splitter/RelSourceImpl.java      |   41 +
 .../query/calcite/splitter/RelTarget.java          |   29 +
 .../query/calcite/splitter/RelTargetImpl.java      |   48 +
 .../query/calcite/splitter/Splitter.java           |  122 ++
 .../trait/AbstractDestinationFunctionFactory.java  |   35 +
 .../query/calcite/trait/AffinityFactory.java       |   60 +
 .../query/calcite/trait/AllTargetsFactory.java     |   45 +
 .../query/calcite/trait/DestinationFunction.java   |   27 +
 .../calcite/trait/DestinationFunctionFactory.java  |   31 +
 .../query/calcite/trait/DistributionTrait.java     |  174 +++
 .../query/calcite/trait/DistributionTraitDef.java  |   78 ++
 .../query/calcite/trait/HashFunctionFactory.java   |   72 ++
 .../query/calcite/trait/IgniteDistribution.java    |   28 +
 .../query/calcite/trait/IgniteDistributions.java   |  272 +++++
 .../query/calcite/trait/NoOpFactory.java           |   41 +
 .../query/calcite/trait/RandomTargetFactory.java   |   47 +
 .../query/calcite/trait/SingleTargetFactory.java   |   48 +
 .../query/calcite/type/IgniteTypeFactory.java      |   33 +
 .../query/calcite/type/IgniteTypeSystem.java       |   28 +
 .../processors/query/calcite/type/RowType.java     |  140 +++
 .../processors/query/calcite/util/Commons.java     |  145 +++
 .../query/calcite/util/IgniteMethod.java           |   42 +
 .../query/calcite/util/ListFieldsQueryCursor.java  |   92 ++
 .../query/calcite/util/TableScanIterator.java      |  149 +++
 .../query/calcite/CalciteQueryProcessorTest.java   | 1203 ++++++++++++++++++++
 .../query/calcite/exchange/OutboxTest.java         |  146 +++
 .../query/calcite/exec/ExecutionTest.java          |   77 ++
 .../ignite/testsuites/IgniteCalciteTestSuite.java  |   35 +
 .../ignite/internal/IgniteComponentType.java       |    8 +
 .../processors/query/GridQueryProcessor.java       |   31 +-
 .../processors/query/GridQueryTypeDescriptor.java  |    5 +
 .../internal/processors/query/QueryContext.java    |   73 ++
 .../internal/processors/query/QueryEngine.java     |   61 +
 .../processors/query/QueryTypeDescriptorImpl.java  |    6 +-
 .../query/schema/SchemaChangeListener.java         |   35 +
 .../GridInternalSubscriptionProcessor.java         |   16 +
 .../processors/query/h2/SchemaManager.java         |   67 +-
 parent/pom.xml                                     |    2 +
 pom.xml                                            |    1 +
 130 files changed, 9960 insertions(+), 7 deletions(-)

diff --git a/modules/calcite/README.txt b/modules/calcite/README.txt
new file mode 100644
index 0000000..e144a77
--- /dev/null
+++ b/modules/calcite/README.txt
@@ -0,0 +1,35 @@
+Apache Ignite Calcite Module
+--------------------------
+
+Apache Ignite Calcite module provides experimental Apache Calcite based query engine.
+
+To enable Calcite module when starting a standalone node, move 'optional/ignite-calcite' folder to
+'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will
+be added to classpath in this case.
+
+Note: At now some logic from ignite-indexing module is reused, this means ignite-indexing module also
+has to be present at classpath.
+
+Importing Calcite Module In Maven Project
+---------------------------------------
+
+If you are using Maven to manage dependencies of your project, you can add Calcite module
+dependency like this (replace '${ignite.version}' with actual Apache Ignite version you are
+interested in):
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                        http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    ...
+    <dependencies>
+        ...
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-calcite</artifactId>
+            <version>${ignite.version}</version>
+        </dependency>
+        ...
+    </dependencies>
+    ...
+</project>
diff --git a/modules/calcite/pom.xml b/modules/calcite/pom.xml
new file mode 100644
index 0000000..d654edb
--- /dev/null
+++ b/modules/calcite/pom.xml
@@ -0,0 +1,112 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-calcite</artifactId>
+    <version>2.8.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <!--
+            At now the new calcite engine reuses some logic
+            and doesn't work without "old" indexing module.
+        -->
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-core</artifactId>
+            <version>${calcite.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <version>${slf4j.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Generate the OSGi MANIFEST.MF for this bundle. -->
+            <plugin>
+                <groupId>org.apache.felix</groupId>
+                <artifactId>maven-bundle-plugin</artifactId>
+            </plugin>
+          </plugins>
+    </build>
+</project>
diff --git a/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java b/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java
new file mode 100644
index 0000000..edae3cc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.calcite.interpreter;
+
+import org.apache.calcite.DataContext;
+
+/**
+ *
+ */
+public class InterpreterUtils {
+    public static Context createContext(DataContext ctx) {
+        return new Context(ctx);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java b/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
new file mode 100644
index 0000000..a1874dc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/calcite/plan/volcano/VolcanoUtils.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.calcite.plan.volcano;
+
+import org.apache.calcite.plan.RelTraitSet;
+
+/**
+ *
+ */
+public class VolcanoUtils {
+    public static RelSubset subset(RelSubset subset, RelTraitSet traits) {
+        return subset.set.getOrCreateSubset(subset.getCluster(), traits.simplify());
+    }
+
+    public static VolcanoPlanner impatient(VolcanoPlanner planner) {
+        planner.impatient = true;
+
+        return planner;
+    }
+}
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
new file mode 100644
index 0000000..86a9a2e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
@@ -0,0 +1,167 @@
+/*
+ * 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;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.function.BiFunction;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.sql.fun.SqlLibrary;
+import org.apache.calcite.sql.fun.SqlLibraryOperatorTableFactory;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.calcite.cluster.MappingServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.DistributedExecution;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Query;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryExecution;
+import org.apache.ignite.internal.processors.query.calcite.schema.CalciteSchemaHolder;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
+import org.apache.ignite.resources.LoggerResource;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class CalciteQueryProcessor implements QueryEngine {
+    /** */
+    private final CalciteSchemaHolder schemaHolder = new CalciteSchemaHolder();
+
+    /** */
+    private final FrameworkConfig config;
+
+    /** */
+    private IgniteLogger log;
+
+    /** */
+    private GridKernalContext kernalContext;
+
+    public CalciteQueryProcessor() {
+        config = Frameworks.newConfigBuilder()
+            .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.MYSQL)
+                .build())
+            // 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.of(this))
+            // Custom cost factory to use during optimization
+            .costFactory(null)
+            .typeSystem(IgniteTypeSystem.DEFAULT)
+            .build();
+    }
+
+    /**
+     * @param log Logger.
+     */
+    @LoggerResource
+    public void setLogger(IgniteLogger log) {
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start(@NotNull GridKernalContext ctx) {
+        kernalContext = ctx;
+
+        GridInternalSubscriptionProcessor prc = ctx.internalSubscriptionProcessor();
+
+        if (prc != null) // Stubbed context doesn't have such processor
+            prc.registerSchemaChangeListener(schemaHolder);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+    }
+
+    @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String query, Object... params) throws IgniteSQLException {
+        PlannerContext context = context(Commons.convert(ctx), query, params, this::buildContext);
+        QueryExecution execution = prepare(context);
+        FieldsQueryCursor<List<?>> cur = execution.execute();
+        return Collections.singletonList(cur);
+    }
+
+    public FrameworkConfig config() {
+        return config;
+    }
+
+    public IgniteLogger log() {
+        return log;
+    }
+
+    /** */
+    public IgnitePlanner planner(RelTraitDef[] traitDefs, PlannerContext ctx0) {
+        FrameworkConfig cfg = Frameworks.newConfigBuilder(config())
+                .defaultSchema(ctx0.schema())
+                .traitDefs(traitDefs)
+                .context(ctx0)
+                .build();
+
+        return new IgnitePlanner(cfg);
+    }
+
+    /**
+     * @param ctx External context.
+     * @param query Query string.
+     * @param params Query parameters.
+     * @return Query execution context.
+     */
+    PlannerContext context(@NotNull Context ctx, String query, Object[] params, BiFunction<Context, Query, PlannerContext> clo) { // Package private visibility for tests.
+        return clo.apply(Contexts.chain(ctx, config.getContext()), new Query(query, params));
+    }
+
+    private PlannerContext buildContext(@NotNull Context parent, @NotNull Query query) {
+        return PlannerContext.builder()
+            .logger(log)
+            .kernalContext(kernalContext)
+            .queryProcessor(this)
+            .parentContext(parent)
+            .query(query)
+            .schema(schemaHolder.schema())
+            .topologyVersion(readyAffinityVersion())
+            .mappingService(new MappingServiceImpl(kernalContext))
+            .build();
+    }
+
+    private QueryExecution prepare(PlannerContext ctx) {
+        return new DistributedExecution(ctx);
+    }
+
+    private AffinityTopologyVersion readyAffinityVersion() {
+        return kernalContext.cache().context().exchange().readyAffinityVersion();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
new file mode 100644
index 0000000..3e91219
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.cluster;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+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.metadata.NodesMapping.DEDUPLICATED;
+
+/**
+ *
+ */
+public class MappingServiceImpl implements MappingService {
+    private final GridKernalContext ctx;
+
+    public MappingServiceImpl(GridKernalContext ctx) {
+        this.ctx = ctx;
+    }
+
+    @Override public NodesMapping local() {
+        return new NodesMapping(Collections.singletonList(ctx.discovery().localNode().id()), null, DEDUPLICATED);
+    }
+
+    @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+        List<ClusterNode> nodes = ctx.discovery().discoCache(topVer).serverNodes();
+
+        return new NodesMapping(Commons.transform(nodes, ClusterNode::id), null, DEDUPLICATED);
+    }
+
+    @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+        GridCacheContext<?,?> cctx = ctx.cache().context().cacheContext(cacheId);
+
+        return cctx.isReplicated() ? replicatedLocation(cctx, topVer) : partitionedLocation(cctx, topVer);
+    }
+
+    private NodesMapping partitionedLocation(GridCacheContext<?,?> cctx, AffinityTopologyVersion topVer) {
+        byte flags = NodesMapping.HAS_PARTITIONED_CACHES;
+
+        List<List<ClusterNode>> assignments = cctx.affinity().assignments(topVer);
+        List<List<UUID>> res;
+
+        if (cctx.config().getWriteSynchronizationMode() == CacheWriteSynchronizationMode.PRIMARY_SYNC) {
+            res = new ArrayList<>(assignments.size());
+
+            for (List<ClusterNode> partNodes : assignments)
+                res.add(F.isEmpty(partNodes) ? Collections.emptyList() : Collections.singletonList(F.first(partNodes).id()));
+        }
+        else if (!cctx.topology().rebalanceFinished(topVer)) {
+            res = new ArrayList<>(assignments.size());
+
+            flags |= NodesMapping.HAS_MOVING_PARTITIONS;
+
+            for (int part = 0; part < assignments.size(); part++) {
+                List<ClusterNode> partNodes = assignments.get(part);
+                List<UUID> partIds = new ArrayList<>(partNodes.size());
+
+                for (ClusterNode node : partNodes) {
+                    if (cctx.topology().partitionState(node.id(), part) == GridDhtPartitionState.OWNING)
+                        partIds.add(node.id());
+                }
+
+                res.add(partIds);
+            }
+        }
+        else
+            res = Commons.transform(assignments, nodes -> Commons.transform(nodes, ClusterNode::id));
+
+        return new NodesMapping(null, res, flags);
+    }
+
+    private NodesMapping replicatedLocation(GridCacheContext<?,?> cctx, AffinityTopologyVersion topVer) {
+        byte flags = NodesMapping.HAS_REPLICATED_CACHES;
+
+        if (cctx.config().getNodeFilter() != null)
+            flags |= NodesMapping.PARTIALLY_REPLICATED;
+
+        GridDhtPartitionTopology topology = cctx.topology();
+
+        List<ClusterNode> nodes = cctx.discovery().discoCache(topVer).cacheGroupAffinityNodes(cctx.cacheId());
+        List<UUID> res;
+
+        if (!topology.rebalanceFinished(topVer)) {
+            flags |= NodesMapping.PARTIALLY_REPLICATED;
+
+            res = new ArrayList<>(nodes.size());
+
+            int parts = topology.partitions();
+
+            for (ClusterNode node : nodes) {
+                if (isOwner(node.id(), topology, parts))
+                    res.add(node.id());
+            }
+        }
+        else
+            res = Commons.transform(nodes, ClusterNode::id);
+
+        return new NodesMapping(res, null, flags);
+    }
+
+    private boolean isOwner(UUID nodeId, GridDhtPartitionTopology topology, int parts) {
+        for (int p = 0; p < parts; p++) {
+            if (topology.partitionState(nodeId, p) != GridDhtPartitionState.OWNING)
+                return false;
+        }
+        return true;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
new file mode 100644
index 0000000..ecbfef2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class EndMarker implements Serializable {
+    public static final EndMarker INSTANCE = new EndMarker();
+
+    private EndMarker(){}
+
+    private Object readResolve() throws ObjectStreamException {
+        return INSTANCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
new file mode 100644
index 0000000..803417c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+
+/**
+ *
+ */
+public interface ExchangeProcessor {
+    int BATCH_SIZE = 200;
+    int PER_NODE_BATCH_COUNT = 10;
+
+    <T> Outbox<T> register(Outbox<T> outbox);
+    <T> void unregister(Outbox<T> outbox);
+    <T> Inbox<T> register(Inbox<T> inbox);
+    <T> void unregister(Inbox<T> inbox);
+    void send(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId, List<?> rows);
+    void acknowledge(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
new file mode 100644
index 0000000..bd515a5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exec.SingleNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.Sink;
+import org.apache.ignite.internal.processors.query.calcite.exec.Source;
+
+/**
+ * TODO
+ */
+public class Inbox<T> implements SingleNode<T> {
+    private final GridCacheVersion queryId;
+    private final long exchangeId;
+
+    private Sink<T> target;
+    private Collection<UUID> sources;
+    private Comparator<T> comparator;
+    private ExchangeProcessor srvc;
+
+    public Inbox(GridCacheVersion queryId, long exchangeId) {
+        this.queryId = queryId;
+        this.exchangeId = exchangeId;
+    }
+
+    public void bind(Sink<T> target, Collection<UUID> sources, Comparator<T> comparator) {
+        this.target = target;
+        this.sources = sources;
+        this.comparator = comparator;
+    }
+
+    void init(ExchangeProcessor srvc) {
+        this.srvc = srvc;
+    }
+
+    @Override public void signal() {
+        // No-op.
+    }
+
+    @Override public void sources(List<Source> sources) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Sink<T> sink(int idx) {
+        throw new UnsupportedOperationException();
+    }
+
+    public void push(UUID source, int batchId, List<?> rows) {
+
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
new file mode 100644
index 0000000..f118330
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.exchange;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exec.AbstractNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.SingleNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.Sink;
+import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T> {
+    private final Map<UUID, Destination> perNode = new HashMap<>();
+
+    private final GridCacheVersion queryId;
+    private final long exchangeId;
+    private final Collection<UUID> targets;
+    private final DestinationFunction function;
+
+    private ExchangeProcessor srvc;
+
+    public Outbox(GridCacheVersion queryId, long exchangeId, Collection<UUID> targets, DestinationFunction function) {
+        super(Sink.noOp());
+        this.queryId = queryId;
+        this.exchangeId = exchangeId;
+
+        this.targets = targets;
+        this.function = function;
+    }
+
+    public void init(ExchangeProcessor srvc) {
+        this.srvc = srvc;
+
+        srvc.register(this);
+
+        signal();
+    }
+
+    public void acknowledge(UUID nodeId, int batchId) {
+        perNode.get(nodeId).acknowledge(batchId);
+    }
+
+    @Override public Sink<T> sink(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    @Override public boolean push(T row) {
+        List<UUID> nodes = function.destination(row);
+
+        if (F.isEmpty(nodes))
+            return true;
+
+        List<Destination> destinations = new ArrayList<>(nodes.size());
+
+        for (UUID node : nodes) {
+            Destination dest = perNode.computeIfAbsent(node, Destination::new);
+
+            if (!dest.ready()) {
+                dest.needSignal();
+
+                return false;
+            }
+
+            destinations.add(dest);
+        }
+
+        for (Destination dest : destinations)
+            dest.add(row);
+
+        return true;
+    }
+
+    @Override public void end() {
+        for (UUID node : targets)
+            perNode.computeIfAbsent(node, Destination::new).end();
+
+        srvc.unregister(this);
+    }
+
+    private final class Destination {
+        private final UUID nodeId;
+
+        private int hwm = -1;
+        private int lwm = -1;
+
+        private ArrayList<Object> curr = new ArrayList<>(ExchangeProcessor.BATCH_SIZE + 1); // extra space for end marker;
+
+        private boolean needSignal;
+
+        private Destination(UUID nodeId) {
+            this.nodeId = nodeId;
+        }
+
+        public void add(T row) {
+            if (curr.size() == ExchangeProcessor.BATCH_SIZE) {
+                assert ready() && srvc != null;
+
+                srvc.send(queryId, exchangeId, nodeId, ++hwm, curr);
+
+                curr = new ArrayList<>(ExchangeProcessor.BATCH_SIZE);
+            }
+
+            curr.add(row);
+        }
+
+        public void end() {
+            curr.add(EndMarker.INSTANCE);
+
+            assert srvc != null;
+
+            srvc.send(queryId, exchangeId, nodeId, hwm, curr);
+
+            curr = null;
+            hwm = Integer.MAX_VALUE;
+        }
+
+        boolean ready() {
+            return hwm - lwm < ExchangeProcessor.PER_NODE_BATCH_COUNT || curr.size() < ExchangeProcessor.BATCH_SIZE;
+        }
+
+        void acknowledge(int id) {
+            if (lwm < id) {
+                lwm = id;
+
+                if (needSignal) {
+                    needSignal = false;
+
+                    signal();
+                }
+            }
+        }
+
+        public void needSignal() {
+            needSignal = true;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
new file mode 100644
index 0000000..66ba475
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ *
+ */
+public abstract class AbstractNode<T> implements Node<T> {
+    protected final Sink<T> target;
+    protected List<Source> sources;
+
+    protected AbstractNode(Sink<T> target) {
+        this.target = target;
+    }
+
+    @Override public void sources(List<Source> sources) {
+        this.sources = Collections.unmodifiableList(sources);
+    }
+
+    public void signal(int idx) {
+        sources.get(idx).signal();
+    }
+
+    @Override public void signal() {
+        sources.forEach(Source::signal);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
new file mode 100644
index 0000000..100f21f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.ArrayDeque;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+
+/**
+ *
+ */
+public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]> {
+    private static final int DEFAULT_BUFFER_SIZE = 1000;
+    private static final Object[] END = new Object[0];
+
+    private ArrayDeque<Object[]> buff;
+
+    public ConsumerNode() {
+        super(Sink.noOp());
+
+        buff = new ArrayDeque<>(DEFAULT_BUFFER_SIZE);
+    }
+
+    @Override public Sink<Object[]> sink(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    @Override public boolean push(Object[] row) {
+        if (buff.size() == DEFAULT_BUFFER_SIZE)
+            return false;
+
+        buff.add(row);
+
+        return true;
+    }
+
+    @Override public void end() {
+        buff.add(END);
+    }
+
+    @Override public boolean hasNext() {
+        if (buff.isEmpty())
+            signal();
+
+        return buff.peek() != END;
+    }
+
+    @Override public Object[] next() {
+        if (buff.isEmpty())
+            signal();
+
+        if(!hasNext())
+            throw new NoSuchElementException();
+
+        return Objects.requireNonNull(buff.poll());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java
new file mode 100644
index 0000000..3075ee4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.function.Predicate;
+
+/**
+ *
+ */
+public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
+    private final Predicate<Object[]> predicate;
+
+    public FilterNode(Sink<Object[]> target, Predicate<Object[]> predicate) {
+        super(target);
+
+        this.predicate = predicate;
+    }
+
+    @Override public Sink<Object[]> sink(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    @Override public boolean push(Object[] row) {
+        return !predicate.test(row) || target.push(row);
+    }
+
+    @Override public void end() {
+        target.end();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
new file mode 100644
index 0000000..cc8ed64
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
@@ -0,0 +1,174 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.UUID;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exchange.Outbox;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+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.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.IgniteRel;
+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.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
+import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunctionFactory;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+import static org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue.PLANNER_CONTEXT;
+import static org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue.QUERY_ID;
+
+/**
+ *
+ */
+public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<IgniteRel, Node<Object[]>> {
+    private final PlannerContext ctx;
+    private final DataContext root;
+    private final ScalarFactory factory;
+    private Deque<Sink<Object[]>> stack;
+
+    public Implementor(DataContext root) {
+        this.root = root;
+
+        ctx = PLANNER_CONTEXT.get(root);
+        factory = new ScalarFactory(new RexBuilder(ctx.typeFactory()));
+        stack = new ArrayDeque<>();
+    }
+
+    @Override public Node<Object[]> visit(IgniteSender rel) {
+        assert stack.isEmpty();
+
+        GridCacheVersion id = QUERY_ID.get(root);
+        long exchangeId = rel.target().exchangeId();
+        NodesMapping mapping = rel.target().mapping();
+        List<UUID> targets = mapping.nodes();
+        IgniteDistribution distribution = rel.target().distribution();
+        DestinationFunctionFactory destFactory = distribution.destinationFunctionFactory();
+        DestinationFunction function = destFactory.create(ctx, mapping, ImmutableIntList.copyOf(distribution.getKeys()));
+
+        Outbox<Object[]> res = new Outbox<>(id, exchangeId, targets, function);
+
+        stack.push(res.sink());
+
+        res.source(source(rel.getInput()));
+
+        return res;
+    }
+
+    @Override public Node<Object[]> visit(IgniteFilter rel) {
+        assert !stack.isEmpty();
+
+        FilterNode res = new FilterNode(stack.pop(), factory.filterPredicate(root, rel.getCondition(), rel.getRowType()));
+
+        stack.push(res.sink());
+
+        res.source(source(rel.getInput()));
+
+        return res;
+    }
+
+    @Override public Node<Object[]> visit(IgniteProject rel) {
+        assert !stack.isEmpty();
+
+        ProjectNode res = new ProjectNode(stack.pop(), factory.projectExpression(root, rel.getProjects(), rel.getInput().getRowType()));
+
+        stack.push(res.sink());
+
+        res.source(source(rel.getInput()));
+
+        return res;
+    }
+
+    @Override public Node<Object[]> visit(IgniteJoin rel) {
+        assert !stack.isEmpty();
+
+        JoinNode res = new JoinNode(stack.pop(), factory.joinExpression(root, rel.getCondition(), rel.getLeft().getRowType(), rel.getRight().getRowType()));
+
+        stack.push(res.sink(1));
+        stack.push(res.sink(0));
+
+        res.sources(sources(rel.getInputs()));
+
+        return res;
+    }
+
+    @Override public Node<Object[]> visit(IgniteTableScan rel) {
+        assert !stack.isEmpty();
+
+        Iterable<Object[]> source = rel.getTable().unwrap(ScannableTable.class).scan(root);
+
+        return new ScanNode(stack.pop(), source);
+    }
+
+    @Override public Node<Object[]> visit(IgniteReceiver rel) {
+        throw new AssertionError(); // TODO
+    }
+
+    @Override public Node<Object[]> visit(IgniteExchange rel) {
+        throw new AssertionError();
+    }
+
+    @Override public Node<Object[]> visit(IgniteRel other) {
+        throw new AssertionError();
+    }
+
+    private Source source(RelNode rel) {
+        if (rel.getConvention() != IgniteConvention.INSTANCE)
+            throw new IllegalStateException("INTERPRETABLE is required.");
+
+        return ((IgniteRel) rel).accept(this);
+    }
+
+    private List<Source> sources(List<RelNode> rels) {
+        ArrayList<Source> res = new ArrayList<>(rels.size());
+
+        for (RelNode rel : rels) {
+            res.add(source(rel));
+        }
+
+        return res;
+    }
+
+    @Override public Node<Object[]> go(IgniteRel rel) {
+        if (rel instanceof IgniteSender)
+            return visit((IgniteSender) rel);
+
+        ConsumerNode res = new ConsumerNode();
+
+        stack.push(res.sink());
+
+        res.source(source(rel));
+
+        return res;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java
new file mode 100644
index 0000000..5188148
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.ArrayList;
+import java.util.function.BiFunction;
+
+/**
+ *
+ */
+public class JoinNode extends AbstractNode<Object[]> {
+    private final BiFunction<Object[], Object[], Object[]> expression;
+    private final ArraySink<Object[]> left;
+    private final ArraySink<Object[]> right;
+
+    private int leftIdx;
+    private int rightIdx;
+    private boolean end;
+
+    public JoinNode(Sink<Object[]> target, BiFunction<Object[], Object[], Object[]> expression) {
+        super(target);
+        this.expression = expression;
+
+        left = new ArraySink<>();
+        right = new ArraySink<>();
+    }
+
+    @Override public Sink<Object[]> sink(int idx) {
+        switch (idx) {
+            case 0:
+                return left;
+            case 1:
+                return right;
+            default:
+                throw new IndexOutOfBoundsException();
+        }
+    }
+
+    @Override public void signal() {
+        if (end)
+            return;
+
+        if (left.end && right.end)
+            tryFlush();
+
+        assert sources != null && sources.size() == 2;
+
+        if (!left.end)
+            signal(0);
+        if (!right.end)
+            signal(1);
+    }
+
+    public void tryFlush() {
+        if (left.end && right.end) {
+            for (int i = leftIdx; i < left.size(); i++) {
+                for (int j = rightIdx; j < right.size(); j++) {
+                    Object[] row = expression.apply(left.get(i), right.get(j));
+
+                    if (row != null && !target.push(row)) {
+                        leftIdx = i;
+                        rightIdx = j;
+
+                        return;
+                    }
+                }
+            }
+
+            end = true;
+            target.end();
+        }
+    }
+
+    private final class ArraySink<T> extends ArrayList<T> implements Sink<T> {
+        private boolean end;
+
+        @Override public boolean push(T row) {
+            return add(row);
+        }
+
+        @Override public void end() {
+            end = true;
+
+            tryFlush();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
new file mode 100644
index 0000000..2472e8e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.List;
+
+/**
+ *
+ */
+public interface Node<T> extends Source {
+    Sink<T> sink(int idx);
+    void sources(List<Source> sources);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
new file mode 100644
index 0000000..d7b7e57
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.function.Function;
+
+/**
+ *
+ */
+public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
+    private final Function<Object[], Object[]> projection;
+
+    public ProjectNode(Sink<Object[]> target, Function<Object[], Object[]> projection) {
+        super(target);
+
+        this.projection = projection;
+    }
+
+    @Override public Sink<Object[]> sink(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    @Override public boolean push(Object[] row) {
+        return target.push(projection.apply(row));
+    }
+
+    @Override public void end() {
+        target.end();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
new file mode 100644
index 0000000..e3ff512
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
@@ -0,0 +1,151 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.InterpreterUtils;
+import org.apache.calcite.interpreter.JaninoRexCompiler;
+import org.apache.calcite.interpreter.Scalar;
+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.RexNode;
+
+/**
+ *
+ */
+public class ScalarFactory {
+    private final JaninoRexCompiler rexCompiler;
+    private final RexBuilder builder;
+
+    public ScalarFactory(RexBuilder builder) {
+        rexCompiler = new JaninoRexCompiler(builder);
+        this.builder = builder;
+    }
+
+    public <T> Predicate<T> filterPredicate(DataContext root, RexNode filter, RelDataType rowType) {
+        Scalar scalar = rexCompiler.compile(ImmutableList.of(filter), rowType);
+        Context ctx = InterpreterUtils.createContext(root);
+
+        return new FilterPredicate<>(ctx, scalar);
+    }
+
+    public <T> Function<T, T> projectExpression(DataContext root, List<RexNode> projects, RelDataType rowType) {
+        Scalar scalar = rexCompiler.compile(projects, rowType);
+        Context ctx = InterpreterUtils.createContext(root);
+        int count = projects.size();
+
+        return new ProjectExpression<>(ctx, scalar, count);
+    }
+
+    public <T> BiFunction<T, T, T> joinExpression(DataContext root, RexNode expression, RelDataType leftType, RelDataType rightType) {
+        RelDataType rowType = combinedType(leftType, rightType);
+
+        Scalar scalar = rexCompiler.compile(ImmutableList.of(expression), rowType);
+        Context ctx = InterpreterUtils.createContext(root);
+        ctx.values = new Object[rowType.getFieldCount()];
+
+        return new JoinExpression<>(ctx, scalar);
+    }
+
+    private RelDataType combinedType(RelDataType... types) {
+        RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(builder.getTypeFactory());
+
+        for (RelDataType type : types)
+            typeBuilder.addAll(type.getFieldList());
+
+        return typeBuilder.build();
+    }
+
+    private static class FilterPredicate<T> implements Predicate<T> {
+        private final Context ctx;
+        private final Scalar scalar;
+        private final Object[] vals;
+
+        private FilterPredicate(Context ctx, Scalar scalar) {
+            this.ctx = ctx;
+            this.scalar = scalar;
+
+            vals = new Object[1];
+        }
+
+        @Override public boolean test(T r) {
+            ctx.values = (Object[]) r;
+            scalar.execute(ctx, vals);
+            return (Boolean) vals[0];
+        }
+    }
+
+    private static class JoinExpression<T> implements BiFunction<T, T, T> {
+        private final Object[] vals;
+        private final Context ctx;
+        private final Scalar scalar;
+
+        private Object[] left0;
+
+        private JoinExpression(Context ctx, Scalar scalar) {
+            this.ctx = ctx;
+            this.scalar = scalar;
+
+            vals = new Object[1];
+        }
+
+        @Override public T apply(T left, T right) {
+            if (left0 != left) {
+                left0 = (Object[]) left;
+                System.arraycopy(left0, 0, ctx.values, 0, left0.length);
+            }
+
+            Object[] right0 = (Object[]) right;
+            System.arraycopy(right0, 0, ctx.values, left0.length, right0.length);
+
+            scalar.execute(ctx, vals);
+
+            if ((Boolean) vals[0])
+                return (T) Arrays.copyOf(ctx.values, ctx.values.length);
+
+            return null;
+        }
+    }
+
+    private static class ProjectExpression<T> implements Function<T, T> {
+        private final Context ctx;
+        private final Scalar scalar;
+        private final int count;
+
+        private ProjectExpression(Context ctx, Scalar scalar, int count) {
+            this.ctx = ctx;
+            this.scalar = scalar;
+            this.count = count;
+        }
+
+        @Override public T apply(T r) {
+            ctx.values = (Object[]) r;
+            Object[] res = new Object[count];
+            scalar.execute(ctx, res);
+
+            return (T) res;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
new file mode 100644
index 0000000..6acaa2b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ *
+ */
+public class ScanNode implements SingleNode<Object[]> {
+    private static final Object[] END = new Object[0];
+
+    /** */
+    private final Sink<Object[]> target;
+    private final Iterable<Object[]> source;
+
+    private Iterator<Object[]> it;
+    private Object[] row;
+
+    public ScanNode(Sink<Object[]> target, Iterable<Object[]> source) {
+        this.target = target;
+        this.source = source;
+    }
+
+    @Override public void signal() {
+        if (row == END)
+            return;
+
+        if (row != null && !target.push(row))
+            return;
+
+        row = null;
+
+        if (it == null)
+            it = source.iterator();
+
+        while (it.hasNext()) {
+            row = it.next();
+
+            if (!target.push(row))
+                return;
+        }
+
+        row = END;
+        target.end();
+    }
+
+    @Override public void sources(List<Source> sources) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Sink<Object[]> sink(int idx) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
new file mode 100644
index 0000000..a0476a0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ *
+ */
+public interface SingleNode<T> extends Node<T> {
+    default Sink<T> sink() {
+        return Objects.requireNonNull(sink(0));
+    }
+
+    default void source(Source source) {
+        sources(Collections.singletonList(source));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
new file mode 100644
index 0000000..842a973
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+/**
+ *
+ */
+public interface Sink<T> {
+    Sink NO_OP = new Sink() {
+        @Override public boolean push(Object row) {
+            return true;
+        }
+
+        @Override public void end() {}
+    };
+
+    boolean push(T row);
+    void end();
+
+    @SuppressWarnings("unchecked")
+    static <T> Sink<T> noOp() {
+        return NO_OP;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
new file mode 100644
index 0000000..95262c7
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+/**
+ *
+ */
+public interface Source {
+    void signal();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
new file mode 100644
index 0000000..de15e4d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 com.google.common.collect.ImmutableList;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
+
+/**
+ *
+ */
+public class FragmentInfo {
+    private final NodesMapping mapping;
+    private final ImmutableList<Pair<IgniteReceiver, RelSource>> sources;
+
+    public FragmentInfo(Pair<IgniteReceiver, RelSource> source) {
+        this(ImmutableList.of(source), null);
+    }
+
+    public FragmentInfo(NodesMapping mapping) {
+        this(null, mapping);
+    }
+
+    public FragmentInfo(ImmutableList<Pair<IgniteReceiver, RelSource>> sources, NodesMapping mapping) {
+        this.sources = sources;
+        this.mapping = mapping;
+    }
+
+    public NodesMapping mapping() {
+        return mapping;
+    }
+
+    public ImmutableList<Pair<IgniteReceiver, RelSource>> sources() {
+        return sources;
+    }
+
+    public FragmentInfo merge(FragmentInfo other) throws LocationMappingException {
+        return new FragmentInfo(
+            merge(sources(), other.sources()),
+            merge(mapping(), other.mapping()));
+    }
+
+    private static NodesMapping merge(NodesMapping left, NodesMapping right) throws LocationMappingException {
+        if (left == null)
+            return right;
+        if (right == null)
+            return left;
+
+        return left.mergeWith(right);
+    }
+
+    private static <T> ImmutableList<T> merge(ImmutableList<T> left, ImmutableList<T> right) {
+        if (left == null)
+            return right;
+        if (right == null)
+            return left;
+
+        return ImmutableList.<T>builder().addAll(left).addAll(right).build();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDerivedDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDerivedDistribution.java
new file mode 100644
index 0000000..6858b13
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDerivedDistribution.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.AbstractConverter;
+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.SingleRel;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.logical.LogicalValues;
+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.mapping.Mappings;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.DerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+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.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class IgniteMdDerivedDistribution implements MetadataHandler<DerivedDistribution> {
+    /** */
+    private static final ThreadLocal<Convention> REQUESTED_CONVENTION = ThreadLocal.withInitial(() -> Convention.NONE);
+
+    public static final RelMetadataProvider SOURCE =
+        ReflectiveRelMetadataProvider.reflectiveSource(
+            IgniteMethod.DERIVED_DISTRIBUTIONS.method(), new IgniteMdDerivedDistribution());
+
+    @Override public MetadataDef<DerivedDistribution> getDef() {
+        return DerivedDistribution.DEF;
+    }
+
+    public List<IgniteDistribution> deriveDistributions(AbstractConverter rel, RelMetadataQuery mq) {
+        return Collections.emptyList();
+    }
+
+    public List<IgniteDistribution> deriveDistributions(RelNode rel, RelMetadataQuery mq) {
+        return F.asList(IgniteMdDistribution._distribution(rel, mq));
+    }
+
+    public List<IgniteDistribution> deriveDistributions(IgniteRel rel, RelMetadataQuery mq) {
+        return F.asList(IgniteMdDistribution._distribution(rel, mq));
+    }
+
+    public List<IgniteDistribution> deriveDistributions(LogicalTableScan rel, RelMetadataQuery mq) {
+        return F.asList(IgniteMdDistribution._distribution(rel, mq));
+    }
+
+    public List<IgniteDistribution> deriveDistributions(LogicalValues rel, RelMetadataQuery mq) {
+        return F.asList(IgniteMdDistribution._distribution(rel, mq));
+    }
+
+    public List<IgniteDistribution> deriveDistributions(LogicalProject rel, RelMetadataQuery mq) {
+        Mappings.TargetMapping mapping =
+            Project.getPartialMapping(rel.getInput().getRowType().getFieldCount(), rel.getProjects());
+
+        return Commons.transform(_deriveDistributions(rel.getInput(), mq), i -> i.apply(mapping));
+    }
+
+    public List<IgniteDistribution> deriveDistributions(SingleRel rel, RelMetadataQuery mq) {
+        if (rel instanceof IgniteRel)
+            return deriveDistributions((IgniteRel)rel, mq);
+
+        return _deriveDistributions(rel.getInput(), mq);
+    }
+
+    public List<IgniteDistribution> deriveDistributions(RelSubset rel, RelMetadataQuery mq) {
+        rel = VolcanoUtils.subset(rel, rel.getTraitSet().replace(REQUESTED_CONVENTION.get()));
+
+        HashSet<IgniteDistribution> res = new HashSet<>();
+
+        for (RelNode rel0 : rel.getRels())
+            res.addAll(_deriveDistributions(rel0, mq));
+
+        if (F.isEmpty(res)) {
+            RelSubset newRel = VolcanoUtils.subset(rel, rel.getTraitSet().replace(Convention.NONE));
+
+            if (newRel != rel) {
+                for (RelNode rel0 : newRel.getRels())
+                    res.addAll(_deriveDistributions(rel0, mq));
+            }
+        }
+
+        return new ArrayList<>(res);
+    }
+
+    public List<IgniteDistribution> deriveDistributions(HepRelVertex rel, RelMetadataQuery mq) {
+        return _deriveDistributions(rel.getCurrentRel(), mq);
+    }
+
+    public List<IgniteDistribution> deriveDistributions(LogicalFilter rel, RelMetadataQuery mq) {
+        return _deriveDistributions(rel.getInput(), mq);
+    }
+
+    public List<IgniteDistribution> deriveDistributions(LogicalJoin rel, RelMetadataQuery mq) {
+        List<IgniteDistribution> left = _deriveDistributions(rel.getLeft(), mq);
+        List<IgniteDistribution> right = _deriveDistributions(rel.getRight(), mq);
+
+        return Commons.transform(IgniteDistributions.suggestJoin(left, right, rel.analyzeCondition(), rel.getJoinType()),
+            IgniteDistributions.BiSuggestion::out);
+    }
+
+    private static List<IgniteDistribution> _deriveDistributions(RelNode rel, RelMetadataQuery mq) {
+        return RelMetadataQueryEx.wrap(mq).derivedDistributions(rel);
+    }
+
+    public static List<IgniteDistribution> deriveDistributions(RelNode rel, Convention convention, RelMetadataQuery mq) {
+        try {
+            REQUESTED_CONVENTION.set(convention);
+
+            return _deriveDistributions(rel, mq);
+        }
+        finally {
+            REQUESTED_CONVENTION.remove();
+        }
+    }
+}
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
new file mode 100644
index 0000000..9673835
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 java.util.List;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Exchange;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+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.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+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.util.typedef.F;
+
+import static org.apache.calcite.rel.RelDistribution.Type.ANY;
+
+/**
+ *
+ */
+public class IgniteMdDistribution implements MetadataHandler<BuiltInMetadata.Distribution> {
+    public static final RelMetadataProvider SOURCE =
+        ReflectiveRelMetadataProvider.reflectiveSource(
+            BuiltInMethod.DISTRIBUTION.method, new IgniteMdDistribution());
+
+    @Override public MetadataDef<BuiltInMetadata.Distribution> getDef() {
+        return BuiltInMetadata.Distribution.DEF;
+    }
+
+    public IgniteDistribution distribution(RelNode rel, RelMetadataQuery mq) {
+        return DistributionTraitDef.INSTANCE.getDefault();
+    }
+
+    public IgniteDistribution distribution(Filter filter, RelMetadataQuery mq) {
+        return filter(mq, filter.getInput(), filter.getCondition());
+    }
+
+    public IgniteDistribution distribution(Project project, RelMetadataQuery mq) {
+        return project(mq, project.getInput(), project.getProjects());
+    }
+
+    public IgniteDistribution distribution(Join join, RelMetadataQuery mq) {
+        return join(mq, join.getLeft(), join.getRight(), join.analyzeCondition(), join.getJoinType());
+    }
+
+    public IgniteDistribution distribution(RelSubset rel, RelMetadataQuery mq) {
+        return rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+    }
+
+    public IgniteDistribution distribution(TableScan rel, RelMetadataQuery mq) {
+        return rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+    }
+
+    public IgniteDistribution distribution(Values values, RelMetadataQuery mq) {
+        return IgniteDistributions.broadcast();
+    }
+
+    public IgniteDistribution distribution(Exchange exchange, RelMetadataQuery mq) {
+        return (IgniteDistribution) exchange.distribution;
+    }
+
+    public IgniteDistribution distribution(HepRelVertex rel, RelMetadataQuery mq) {
+        return _distribution(rel.getCurrentRel(), mq);
+    }
+
+    public static IgniteDistribution project(RelMetadataQuery mq, RelNode input, List<? extends RexNode> projects) {
+        return project(input.getRowType(), _distribution(input, mq), projects);
+    }
+
+    public static IgniteDistribution project(RelDataType inType, IgniteDistribution inDistr, List<? extends RexNode> projects) {
+        return inDistr.apply(Project.getPartialMapping(inType.getFieldCount(), projects));
+    }
+
+    public static IgniteDistribution filter(RelMetadataQuery mq, RelNode input, RexNode condition) {
+        return _distribution(input, mq);
+    }
+
+    public static IgniteDistribution join(RelMetadataQuery mq, RelNode left, RelNode right, JoinInfo joinInfo, JoinRelType joinType) {
+        return join(_distribution(left, mq), _distribution(right, mq), joinInfo, joinType);
+    }
+
+    public static IgniteDistribution join(IgniteDistribution left, IgniteDistribution right, JoinInfo joinInfo, JoinRelType joinType) {
+        return F.first(IgniteDistributions.suggestJoin(left, right, joinInfo, joinType)).out();
+    }
+
+    public static IgniteDistribution _distribution(RelNode rel, RelMetadataQuery mq) {
+        IgniteDistribution distr = rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+
+        if (distr.getType() != ANY)
+            return distr;
+
+        return (IgniteDistribution) mq.distribution(rel);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
new file mode 100644
index 0000000..c5f941e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Join;
+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.Pair;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.FragmentMetadata;
+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.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Edge;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+/**
+ *
+ */
+public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
+    public static final RelMetadataProvider SOURCE =
+        ReflectiveRelMetadataProvider.reflectiveSource(
+            IgniteMethod.FRAGMENT_INFO.method(), new IgniteMdFragmentInfo());
+
+    @Override public MetadataDef<FragmentMetadata> getDef() {
+        return FragmentMetadata.DEF;
+    }
+
+    public FragmentInfo getFragmentInfo(RelNode rel, RelMetadataQuery mq) {
+        throw new AssertionError();
+    }
+
+    public FragmentInfo getFragmentInfo(RelSubset rel, RelMetadataQuery mq) {
+        throw new AssertionError();
+    }
+
+    public FragmentInfo getFragmentInfo(SingleRel rel, RelMetadataQuery mq) {
+        return fragmentInfo(rel.getInput(), mq);
+    }
+
+    public FragmentInfo getFragmentInfo(Join rel, RelMetadataQuery mq) {
+        mq = RelMetadataQueryEx.wrap(mq);
+
+        FragmentInfo left = fragmentInfo(rel.getLeft(), mq);
+        FragmentInfo right = fragmentInfo(rel.getRight(), mq);
+
+        try {
+            return left.merge(right);
+        }
+        catch (LocationMappingException e) {
+            // a replicated cache is cheaper to redistribute
+            if (!left.mapping().hasPartitionedCaches())
+                throw planningException(rel, e, true);
+            else if (!right.mapping().hasPartitionedCaches())
+                throw planningException(rel, e, false);
+
+            // both sub-trees have partitioned sources, less cost is better
+            RelOptCluster cluster = rel.getCluster();
+
+            RelOptCost leftCost = rel.getLeft().computeSelfCost(cluster.getPlanner(), mq);
+            RelOptCost rightCost = rel.getRight().computeSelfCost(cluster.getPlanner(), mq);
+
+            throw planningException(rel, e, leftCost.isLe(rightCost));
+        }
+    }
+
+    public FragmentInfo getFragmentInfo(IgniteReceiver rel, RelMetadataQuery mq) {
+        return new FragmentInfo(Pair.of(rel, rel.source()));
+    }
+
+    public FragmentInfo getFragmentInfo(IgniteTableScan rel, RelMetadataQuery mq) {
+        return rel.getTable().unwrap(IgniteTable.class).fragmentInfo(Commons.plannerContext(rel));
+    }
+
+    public static FragmentInfo fragmentInfo(RelNode rel, RelMetadataQuery mq) {
+        return RelMetadataQueryEx.wrap(mq).getFragmentLocation(rel);
+    }
+
+    private OptimisticPlanningException planningException(BiRel rel, Exception cause, boolean splitLeft) {
+        String msg = "Failed to calculate physical distribution";
+
+        if (splitLeft)
+            return new OptimisticPlanningException(msg, new Edge(rel, rel.getLeft(), 0), cause);
+
+        return new OptimisticPlanningException(msg, new Edge(rel, rel.getRight(), 1), cause);
+    }
+}
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
new file mode 100644
index 0000000..5579a14
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMetadata.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 com.google.common.collect.ImmutableList;
+import java.util.List;
+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.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+/**
+ *
+ */
+public class IgniteMetadata {
+    public static final RelMetadataProvider METADATA_PROVIDER =
+        ChainedRelMetadataProvider.of(
+            ImmutableList.of(
+                IgniteMdDerivedDistribution.SOURCE,
+                IgniteMdDistribution.SOURCE,
+                IgniteMdFragmentInfo.SOURCE,
+                DefaultRelMetadataProvider.INSTANCE));
+
+    public interface FragmentMetadata extends Metadata {
+        MetadataDef<FragmentMetadata> DEF = MetadataDef.of(FragmentMetadata.class,
+            FragmentMetadata.Handler.class, IgniteMethod.FRAGMENT_INFO.method());
+
+        /** Determines how the rows are distributed. */
+        FragmentInfo getFragmentInfo();
+
+        /** Handler API. */
+        interface Handler extends MetadataHandler<FragmentMetadata> {
+            FragmentInfo getFragmentInfo(RelNode r, RelMetadataQuery mq);
+        }
+    }
+
+    public interface DerivedDistribution extends Metadata {
+        MetadataDef<DerivedDistribution> DEF = MetadataDef.of(DerivedDistribution.class,
+            DerivedDistribution.Handler.class, IgniteMethod.DERIVED_DISTRIBUTIONS.method());
+
+        List<IgniteDistribution> deriveDistributions();
+
+        /** Handler API. */
+        interface Handler extends MetadataHandler<DerivedDistribution> {
+            List<IgniteDistribution> deriveDistributions(RelNode r, RelMetadataQuery mq);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/LocationMappingException.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/LocationMappingException.java
new file mode 100644
index 0000000..e1d884a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/LocationMappingException.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+/**
+ *
+ */
+public class LocationMappingException extends Exception {
+    public LocationMappingException(String message) {
+        super(message);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
new file mode 100644
index 0000000..9472cd1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.ignite.internal.processors.affinity.AffinityTopologyVersion;
+
+/**
+ *
+ */
+public interface MappingService {
+    NodesMapping local(); // returns local node with single partition
+    NodesMapping random(AffinityTopologyVersion topVer); // returns random distribution, partitions count depends on nodes count
+    NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer); // returns cache distribution
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
new file mode 100644
index 0000000..b63e3e2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
@@ -0,0 +1,196 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+public class NodesMapping implements Serializable {
+    public static final byte HAS_MOVING_PARTITIONS = 1;
+    public static final byte HAS_REPLICATED_CACHES = 1 << 1;
+    public static final byte HAS_PARTITIONED_CACHES = 1 << 2;
+    public static final byte PARTIALLY_REPLICATED = 1 << 3;
+    public static final byte DEDUPLICATED = 1 << 4;
+
+    private final List<UUID> nodes;
+    private final List<List<UUID>> assignments;
+    private final byte flags;
+
+    public NodesMapping(List<UUID> nodes, List<List<UUID>> assignments, byte flags) {
+        this.nodes = nodes;
+        this.assignments = assignments;
+        this.flags = flags;
+    }
+
+    public List<UUID> nodes() {
+        return nodes;
+    }
+
+    public List<List<UUID>> assignments() {
+        return assignments;
+    }
+
+    public NodesMapping mergeWith(NodesMapping other) throws LocationMappingException {
+        byte flags = (byte) (this.flags | other.flags);
+
+        if ((flags & PARTIALLY_REPLICATED) == 0)
+            return new NodesMapping(U.firstNotNull(nodes, other.nodes), mergeAssignments(other, null), flags);
+
+        List<UUID> nodes;
+
+        if (this.nodes == null)
+            nodes = other.nodes;
+        else if (other.nodes == null)
+            nodes = this.nodes;
+        else
+            nodes = Commons.intersect(this.nodes, other.nodes);
+
+        if (nodes != null && nodes.isEmpty())
+            throw new LocationMappingException("Failed to map fragment to location.");
+
+        return new NodesMapping(nodes, mergeAssignments(other, nodes), flags);
+    }
+
+    public NodesMapping deduplicate() {
+        if (!excessive())
+            return this;
+
+        if (assignments == null) {
+            UUID node = nodes.get(ThreadLocalRandom.current().nextInt(nodes.size()));
+
+            return new NodesMapping(Collections.singletonList(node), null, (byte)(flags | DEDUPLICATED));
+        }
+
+        HashSet<UUID> nodes0 = new HashSet<>();
+        List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
+
+        for (List<UUID> partNodes : assignments) {
+            UUID node = F.first(partNodes);
+
+            if (node == null)
+                assignments0.add(Collections.emptyList());
+            else {
+                assignments0.add(Collections.singletonList(node));
+
+                nodes0.add(node);
+            }
+        }
+
+        return new NodesMapping(new ArrayList<>(nodes0), assignments0, (byte)(flags | DEDUPLICATED));
+    }
+
+    public int[] partitions(UUID node) {
+        if (assignments == null)
+            return null;
+
+        GridIntList parts = new GridIntList(assignments.size());
+
+        for (int i = 0; i < assignments.size(); i++) {
+            List<UUID> assignment = assignments.get(i);
+            if (Objects.equals(node, F.first(assignment)))
+                parts.add(i);
+        }
+
+        return parts.array();
+    }
+
+    public boolean excessive() {
+        return (flags & DEDUPLICATED) == 0;
+    }
+
+    public boolean hasMovingPartitions() {
+        return (flags & HAS_MOVING_PARTITIONS) == HAS_MOVING_PARTITIONS;
+    }
+
+    public boolean hasReplicatedCaches() {
+        return (flags & HAS_REPLICATED_CACHES) == HAS_REPLICATED_CACHES;
+    }
+
+    public boolean hasPartitionedCaches() {
+        return (flags & HAS_PARTITIONED_CACHES) == HAS_PARTITIONED_CACHES;
+    }
+
+    public boolean partiallyReplicated() {
+        return (flags & PARTIALLY_REPLICATED) == PARTIALLY_REPLICATED;
+    }
+
+    private List<List<UUID>> mergeAssignments(NodesMapping other, List<UUID> nodes) throws LocationMappingException {
+        byte flags = (byte) (this.flags | other.flags); List<List<UUID>> left = assignments, right = other.assignments;
+
+        if (left == null && right == null)
+            return null; // nothing to intersect;
+
+        if (left == null || right == null || (flags & HAS_MOVING_PARTITIONS) == 0) {
+            List<List<UUID>> assignments = U.firstNotNull(left, right);
+
+            if (nodes == null || (flags & PARTIALLY_REPLICATED) == 0)
+                return assignments;
+
+            List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
+            HashSet<UUID> nodesSet = new HashSet<>(nodes);
+
+            for (List<UUID> partNodes : assignments) {
+                List<UUID> partNodes0 = new ArrayList<>(partNodes.size());
+
+                for (UUID partNode : partNodes) {
+                    if (nodesSet.contains(partNode))
+                        partNodes0.add(partNode);
+                }
+
+                if (partNodes0.isEmpty())
+                    throw new LocationMappingException("Failed to map fragment to location.");
+
+                assignments0.add(partNodes0);
+            }
+
+            return assignments0;
+        }
+
+        List<List<UUID>> assignments = new ArrayList<>(left.size());
+        HashSet<UUID> nodesSet = nodes != null ? new HashSet<>(nodes) : null;
+
+        for (int i = 0; i < left.size(); i++) {
+            List<UUID> leftNodes = left.get(i), partNodes = new ArrayList<>(leftNodes.size());
+            HashSet<UUID> rightNodesSet = new HashSet<>(right.get(i));
+
+            for (UUID partNode : leftNodes) {
+                if (rightNodesSet.contains(partNode) && (nodesSet == null || nodesSet.contains(partNode)))
+                    partNodes.add(partNode);
+            }
+
+            if (partNodes.isEmpty())
+                throw new LocationMappingException("Failed to map fragment to location.");
+
+            assignments.add(partNodes);
+        }
+
+        return assignments;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
new file mode 100644
index 0000000..f0d3fe3
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.ignite.internal.processors.query.calcite.splitter.Edge;
+
+/**
+ *
+ */
+public class OptimisticPlanningException extends RuntimeException{
+    private final Edge edge;
+
+    public OptimisticPlanningException(String message, Edge edge, Throwable cause) {
+        super(message, cause);
+        this.edge = edge;
+    }
+
+    public Edge edge() {
+        return edge;
+    }
+}
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
new file mode 100644
index 0000000..74d7682
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
+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.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public class RelMetadataQueryEx extends RelMetadataQuery {
+    private static final RelMetadataQueryEx PROTO = new RelMetadataQueryEx();
+    public static final JaninoRelMetadataProvider PROVIDER = JaninoRelMetadataProvider.of(IgniteMetadata.METADATA_PROVIDER);
+
+    static {
+        PROVIDER.register(ImmutableList.of(
+                IgniteExchange.class,
+                IgniteReceiver.class,
+                IgniteSender.class,
+                IgniteFilter.class,
+                IgniteProject.class,
+                IgniteJoin.class,
+                IgniteTableScan.class));
+    }
+
+    private IgniteMetadata.FragmentMetadata.Handler sourceDistributionHandler;
+    private IgniteMetadata.DerivedDistribution.Handler derivedDistributionsHandler;
+
+    @SuppressWarnings("MethodOverridesStaticMethodOfSuperclass")
+    public static RelMetadataQueryEx instance() {
+        return new RelMetadataQueryEx(PROTO);
+    }
+
+    public static RelMetadataQueryEx wrap(@NotNull RelMetadataQuery mq) {
+        if (mq.getClass() == RelMetadataQueryEx.class)
+            return (RelMetadataQueryEx) mq;
+
+        return new RelMetadataQueryEx(mq);
+    }
+
+    private RelMetadataQueryEx(@NotNull RelMetadataQueryEx parent) {
+        super(PROVIDER, parent);
+
+        sourceDistributionHandler = parent.sourceDistributionHandler;
+        derivedDistributionsHandler = parent.derivedDistributionsHandler;
+    }
+
+    private RelMetadataQueryEx(@NotNull RelMetadataQuery parent) {
+        super(PROVIDER, parent);
+
+        sourceDistributionHandler = PROTO.sourceDistributionHandler;
+        derivedDistributionsHandler = PROTO.derivedDistributionsHandler;
+    }
+
+    private RelMetadataQueryEx() {
+        super(JaninoRelMetadataProvider.DEFAULT, RelMetadataQuery.EMPTY);
+
+        sourceDistributionHandler = initialHandler(IgniteMetadata.FragmentMetadata.Handler.class);
+        derivedDistributionsHandler = initialHandler(IgniteMetadata.DerivedDistribution.Handler.class);
+    }
+
+    public FragmentInfo getFragmentLocation(RelNode rel) {
+        for (;;) {
+            try {
+                return sourceDistributionHandler.getFragmentInfo(rel, this);
+            } catch (JaninoRelMetadataProvider.NoHandler e) {
+                sourceDistributionHandler = revise(e.relClass, IgniteMetadata.FragmentMetadata.DEF);
+            }
+        }
+    }
+
+    public List<IgniteDistribution> derivedDistributions(RelNode rel) {
+        for (;;) {
+            try {
+                return derivedDistributionsHandler.deriveDistributions(rel, this);
+            } catch (JaninoRelMetadataProvider.NoHandler e) {
+                derivedDistributionsHandler = revise(e.relClass, IgniteMetadata.DerivedDistribution.DEF);
+            }
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
new file mode 100644
index 0000000..0fe3dc5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.prepare;
+
+import org.apache.calcite.DataContext;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+
+/**
+ *
+ */
+public enum ContextValue {
+    QUERY_ID("_query_id", GridCacheVersion.class),
+    PLANNER_CONTEXT("_planner_context", PlannerContext.class);
+
+    private final String valueName;
+    private final Class type;
+
+    ContextValue(String valueName, Class type) {
+        this.valueName = valueName;
+        this.type = type;
+    }
+
+    public String valueName() {
+        return valueName;
+    }
+
+    public <T> T get(DataContext ctx) {
+        return (T) type.cast(ctx.get(valueName));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
new file mode 100644
index 0000000..3c2d14c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
@@ -0,0 +1,67 @@
+/*
+ * 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.prepare;
+
+import java.util.Map;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
+
+/**
+ *
+ */
+public class DataContextImpl implements DataContext {
+    /** */
+    private final PlannerContext ctx;
+
+    /** */
+    private final Map<String, Object> params;
+
+    /**
+     * @param params Parameters.
+     * @param ctx Query context.
+     */
+    public DataContextImpl(Map<String, Object> params, PlannerContext ctx) {
+        this.params = params;
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SchemaPlus getRootSchema() {
+        return ctx.schema();
+    }
+
+    /** {@inheritDoc} */
+    @Override public JavaTypeFactory getTypeFactory() {
+        return ctx.typeFactory();
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryProvider getQueryProvider() {
+        return ctx.queryProvider();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(String name) {
+        if (ContextValue.PLANNER_CONTEXT.valueName().equals(name))
+            return ctx;
+
+        return params.get(name);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
new file mode 100644
index 0000000..b81961f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
@@ -0,0 +1,110 @@
+/*
+ * 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.prepare;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelDistributions;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+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.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.util.ListFieldsQueryCursor;
+
+/**
+ *
+ */
+public class DistributedExecution implements QueryExecution {
+    /** */
+    private final PlannerContext ctx;
+
+    /**
+     * @param ctx Query context.
+     */
+    public DistributedExecution(PlannerContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public FieldsQueryCursor<List<?>> execute() {
+        CalciteQueryProcessor proc = ctx.queryProcessor();
+        Query query = ctx.query();
+
+        RelTraitDef[] traitDefs = {
+            RelDistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE,
+            RelCollationTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)) {
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getTraitSet()
+                .replace(relRoot.collation)
+                .replace(IgniteConvention.INSTANCE)
+                .replace(RelDistributions.ANY)
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        } catch (SqlParseException | ValidationException e) {
+            String msg = "Failed to parse query.";
+
+            proc.log().error(msg, e);
+
+            throw new IgniteSQLException(msg, IgniteQueryErrorCode.PARSING, e);
+        } catch (Exception e) {
+            String msg = "Failed to create query execution graph.";
+
+            proc.log().error(msg, e);
+
+            throw new IgniteSQLException(msg, IgniteQueryErrorCode.UNKNOWN, e);
+        }
+
+        // TODO physical plan.
+
+        return new ListFieldsQueryCursor<>(relRoot.rel.getRowType(), Linq4j.emptyEnumerable(), Arrays::asList);
+    }
+}
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
new file mode 100644
index 0000000..2f35a65
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
@@ -0,0 +1,436 @@
+/*
+ * 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.prepare;
+
+import com.google.common.collect.ImmutableList;
+import java.io.Reader;
+import java.util.List;
+import java.util.Properties;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.plan.volcano.VolcanoUtils;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
+import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexExecutor;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql2rel.RelDecorrelator;
+import org.apache.calcite.sql2rel.SqlRexConvertletTable;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.Program;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.Graph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.GraphToRelConverter;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Splitter;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
+    private final SqlOperatorTable operatorTable;
+    private final ImmutableList<Program> programs;
+    private final FrameworkConfig frameworkConfig;
+    private final Context context;
+    private final CalciteConnectionConfig connectionConfig;
+    private final ImmutableList<RelTraitDef> traitDefs;
+    private final SqlParser.Config parserConfig;
+    private final SqlToRelConverter.Config sqlToRelConverterConfig;
+    private final SqlRexConvertletTable convertletTable;
+    private final RexExecutor executor;
+    private final SchemaPlus defaultSchema;
+    private final JavaTypeFactory typeFactory;
+
+    private boolean open;
+
+    private RelOptPlanner planner;
+    private RelMetadataProvider metadataProvider;
+    private SqlValidator validator;
+
+    /**
+     * @param config Framework config.
+     */
+    public IgnitePlanner(FrameworkConfig config) {
+        frameworkConfig = config;
+        defaultSchema = config.getDefaultSchema();
+        operatorTable = config.getOperatorTable();
+        programs = config.getPrograms();
+        parserConfig = config.getParserConfig();
+        sqlToRelConverterConfig = config.getSqlToRelConverterConfig();
+        traitDefs = config.getTraitDefs();
+        convertletTable = config.getConvertletTable();
+        executor = config.getExecutor();
+        context = config.getContext();
+        connectionConfig = connConfig();
+
+        RelDataTypeSystem typeSystem = connectionConfig
+            .typeSystem(RelDataTypeSystem.class, IgniteTypeSystem.DEFAULT);
+
+        typeFactory = new IgniteTypeFactory(typeSystem);
+
+        Commons.plannerContext(context).planner(this);
+    }
+
+    private CalciteConnectionConfig connConfig() {
+        CalciteConnectionConfig unwrapped = context.unwrap(CalciteConnectionConfig.class);
+        if (unwrapped != null)
+            return unwrapped;
+
+        Properties properties = new Properties();
+        properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
+            String.valueOf(parserConfig.caseSensitive()));
+        properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
+            String.valueOf(frameworkConfig.getParserConfig().conformance()));
+        return new CalciteConnectionConfigImpl(properties);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelTraitSet getEmptyTraitSet() {
+        return planner.emptyTraitSet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        reset();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        planner = null;
+        metadataProvider = null;
+        validator = null;
+
+        RelMetadataQuery.THREAD_PROVIDERS.remove();
+
+        open = false;
+    }
+
+    private void ready() {
+        if (!open) {
+            planner = VolcanoUtils.impatient(new VolcanoPlanner(frameworkConfig.getCostFactory(), context));
+            planner.setExecutor(executor);
+            metadataProvider = new CachingRelMetadataProvider(IgniteMetadata.METADATA_PROVIDER, planner);
+
+            validator = new IgniteSqlValidator(operatorTable(), createCatalogReader(), typeFactory, conformance());
+            validator.setIdentifierExpansion(true);
+
+            for (RelTraitDef def : traitDefs) {
+                planner.addRelTraitDef(def);
+            }
+
+            open = true;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlNode parse(Reader reader) throws SqlParseException {
+        return SqlParser.create(reader, parserConfig).parseStmt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlNode validate(SqlNode sqlNode) throws ValidationException {
+        ready();
+
+        try {
+            return validator.validate(sqlNode);
+        }
+        catch (RuntimeException e) {
+            throw new ValidationException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode) throws ValidationException {
+        ready();
+
+        SqlNode validatedNode = validate(sqlNode);
+        RelDataType type = validator.getValidatedNodeType(validatedNode);
+        return Pair.of(validatedNode, type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelNode convert(SqlNode sql) {
+        return rel(sql).rel;
+    }
+
+    public RelNode convert(RelGraph graph) {
+        ready();
+
+        RelOptCluster cluster = createCluster(createRexBuilder());
+        RelBuilder relBuilder = createRelBuilder(cluster, createCatalogReader());
+
+        return new GraphToRelConverter(this, relBuilder, operatorTable).convert(graph);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelRoot rel(SqlNode sql) {
+        ready();
+
+        RexBuilder rexBuilder = createRexBuilder();
+        RelOptCluster cluster = createCluster(rexBuilder);
+        SqlToRelConverter.Config config = SqlToRelConverter.configBuilder()
+            .withConfig(sqlToRelConverterConfig)
+            .withTrimUnusedFields(false)
+            .withConvertTableAccess(false)
+            .build();
+        SqlToRelConverter sqlToRelConverter =
+            new SqlToRelConverter(this, validator, createCatalogReader(), cluster, convertletTable, config);
+        RelRoot root = sqlToRelConverter.convertQuery(sql, false, true);
+        root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true));
+        RelBuilder relBuilder = createRelBuilder(cluster, null);
+        root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel, relBuilder));
+        return root;
+    }
+
+    public QueryPlan plan(RelNode rel) {
+        ready();
+
+        if (rel.getConvention() != IgniteConvention.INSTANCE)
+            throw new IllegalArgumentException("Physical node is required.");
+
+        return new Splitter().go((IgniteRel) rel);
+    }
+
+    public Graph graph(RelNode rel) {
+        ready();
+
+        if (rel.getConvention() != IgniteConvention.INSTANCE)
+            throw new IllegalArgumentException("Physical node is required.");
+
+        return new RelToGraphConverter().go((IgniteRel) rel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
+        ready();
+
+        SqlParser parser = SqlParser.create(queryString, parserConfig);
+        SqlNode sqlNode;
+        try {
+            sqlNode = parser.parseQuery();
+        }
+        catch (SqlParseException e) {
+            throw new RuntimeException("parse failed", e);
+        }
+
+        SqlConformance conformance = conformance();
+        CalciteCatalogReader catalogReader =
+            createCatalogReader().withSchemaPath(schemaPath);
+        SqlValidator validator = new IgniteSqlValidator(operatorTable(), catalogReader, typeFactory, conformance);
+        validator.setIdentifierExpansion(true);
+
+        RexBuilder rexBuilder = createRexBuilder();
+        RelOptCluster cluster = createCluster(rexBuilder);
+        SqlToRelConverter.Config config = SqlToRelConverter
+            .configBuilder()
+            .withConfig(sqlToRelConverterConfig)
+            .withTrimUnusedFields(false)
+            .withConvertTableAccess(false)
+            .build();
+        SqlToRelConverter sqlToRelConverter =
+            new SqlToRelConverter(this, validator,
+                catalogReader, cluster, convertletTable, config);
+
+        RelRoot root = sqlToRelConverter.convertQuery(sqlNode, true, false);
+        RelRoot root2 = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true));
+        RelBuilder relBuilder = createRelBuilder(cluster, null);
+        return root2.withRel(RelDecorrelator.decorrelateQuery(root.rel, relBuilder));
+    }
+
+    private RelOptCluster createCluster(RexBuilder rexBuilder) {
+        RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
+
+        cluster.setMetadataProvider(metadataProvider);
+        RelMetadataQuery.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(metadataProvider));
+
+        return cluster;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelNode transform(int programIdx, RelTraitSet targetTraits, RelNode rel) {
+        ready();
+
+        RelTraitSet toTraits = targetTraits.simplify();
+
+        rel.accept(new MetaDataProviderModifier(metadataProvider));
+
+        return programs.get(programIdx).run(planner, rel, toTraits, ImmutableList.of(), ImmutableList.of());
+    }
+
+    public RelNode transform(PlannerType plannerType, PlannerPhase plannerPhase, RelNode input, RelTraitSet targetTraits)  {
+        ready();
+
+        RelTraitSet toTraits = targetTraits.simplify();
+
+        input.accept(new MetaDataProviderModifier(metadataProvider));
+
+        RelNode output;
+
+        switch (plannerType) {
+            case HEP:
+                final HepProgramBuilder programBuilder = new HepProgramBuilder();
+
+                for (RelOptRule rule : plannerPhase.getRules(Commons.plannerContext(context))) {
+                    programBuilder.addRuleInstance(rule);
+                }
+
+                final HepPlanner hepPlanner =
+                    new HepPlanner(programBuilder.build(), context, true, null, RelOptCostImpl.FACTORY);
+
+                hepPlanner.setRoot(input);
+
+                if (!input.getTraitSet().equals(targetTraits))
+                    hepPlanner.changeTraits(input, toTraits);
+
+                output = hepPlanner.findBestExp();
+
+                break;
+            case VOLCANO:
+                Program program = Programs.of(plannerPhase.getRules(Commons.plannerContext(context)));
+
+                output = program.run(planner, input, toTraits,
+                    ImmutableList.of(), ImmutableList.of());
+
+                break;
+            default:
+                throw new AssertionError("Unknown planner type: " + plannerType);
+        }
+
+        return output;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JavaTypeFactory getTypeFactory() {
+        return typeFactory;
+    }
+
+    private SqlConformance conformance() {
+        return connectionConfig.conformance();
+    }
+
+    private SqlOperatorTable operatorTable() {
+        return operatorTable;
+    }
+
+    private RexBuilder createRexBuilder() {
+        return new RexBuilder(typeFactory);
+    }
+
+    private RelBuilder createRelBuilder(RelOptCluster cluster, RelOptSchema schema) {
+        return sqlToRelConverterConfig.getRelBuilderFactory().create(cluster, schema);
+    }
+
+    private CalciteCatalogReader createCatalogReader() {
+        SchemaPlus rootSchema = rootSchema(defaultSchema);
+
+        return new CalciteCatalogReader(
+            CalciteSchema.from(rootSchema),
+            CalciteSchema.from(defaultSchema).path(null),
+            typeFactory, connectionConfig);
+    }
+
+    private static SchemaPlus rootSchema(SchemaPlus schema) {
+        for (; ; ) {
+            if (schema.getParentSchema() == null) {
+                return schema;
+            }
+            schema = schema.getParentSchema();
+        }
+    }
+
+    /** */
+    private static class MetaDataProviderModifier extends RelShuttleImpl {
+        /** */
+        private final RelMetadataProvider metadataProvider;
+
+        /** */
+        private MetaDataProviderModifier(RelMetadataProvider metadataProvider) {
+            this.metadataProvider = metadataProvider;
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelNode visit(TableScan scan) {
+            scan.getCluster().setMetadataProvider(metadataProvider);
+            return super.visit(scan);
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelNode visit(TableFunctionScan scan) {
+            scan.getCluster().setMetadataProvider(metadataProvider);
+            return super.visit(scan);
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelNode visit(LogicalValues values) {
+            values.getCluster().setMetadataProvider(metadataProvider);
+            return super.visit(values);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected RelNode visitChild(RelNode parent, int i, RelNode child) {
+            child.accept(this);
+            parent.getCluster().setMetadataProvider(metadataProvider);
+            return parent;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
new file mode 100644
index 0000000..74bd1df
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
@@ -0,0 +1,55 @@
+/*
+ * 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.prepare;
+
+/**
+ *
+ */
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+
+/** Validator. */
+public class IgniteSqlValidator extends SqlValidatorImpl {
+    public IgniteSqlValidator(SqlOperatorTable opTab,
+        CalciteCatalogReader catalogReader, JavaTypeFactory typeFactory,
+        SqlConformance conformance) {
+        super(opTab, catalogReader, typeFactory, conformance);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RelDataType getLogicalSourceRowType(
+        RelDataType sourceRowType, SqlInsert insert) {
+        final RelDataType superType =
+            super.getLogicalSourceRowType(sourceRowType, insert);
+        return ((JavaTypeFactory) typeFactory).toSql(superType);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RelDataType getLogicalTargetRowType(
+        RelDataType targetRowType, SqlInsert insert) {
+        final RelDataType superType =
+            super.getLogicalTargetRowType(targetRowType, insert);
+        return ((JavaTypeFactory) typeFactory).toSql(superType);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
new file mode 100644
index 0000000..352980f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.prepare;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.exchange.ExchangeProcessor;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+
+/**
+ *
+ */
+public final class PlannerContext implements Context {
+    private final Context parentContext;
+    private final Query query;
+    private final AffinityTopologyVersion topologyVersion;
+    private final SchemaPlus schema;
+    private final IgniteLogger logger;
+    private final GridKernalContext kernalContext;
+    private final CalciteQueryProcessor queryProcessor;
+    private final MappingService mappingService;
+    private final ExchangeProcessor exchangeProcessor;
+
+    private IgnitePlanner planner;
+
+    private PlannerContext(Context parentContext, Query query, AffinityTopologyVersion topologyVersion,
+        SchemaPlus schema, IgniteLogger logger, GridKernalContext kernalContext, CalciteQueryProcessor queryProcessor, MappingService mappingService,
+        ExchangeProcessor exchangeProcessor) {
+        this.parentContext = parentContext;
+        this.query = query;
+        this.topologyVersion = topologyVersion;
+        this.schema = schema;
+        this.logger = logger;
+        this.kernalContext = kernalContext;
+        this.queryProcessor = queryProcessor;
+        this.mappingService = mappingService;
+        this.exchangeProcessor = exchangeProcessor;
+    }
+
+    public Query query() {
+        return query;
+    }
+
+    public AffinityTopologyVersion topologyVersion() {
+        return topologyVersion;
+    }
+
+    public SchemaPlus schema() {
+        return schema;
+    }
+
+    public IgniteLogger logger() {
+        return logger;
+    }
+
+    public GridKernalContext kernalContext() {
+        return kernalContext;
+    }
+
+    public CalciteQueryProcessor queryProcessor() {
+        return queryProcessor;
+    }
+
+    void planner(IgnitePlanner planner) {
+        this.planner = planner;
+    }
+
+    public IgnitePlanner planner() {
+        return planner;
+    }
+
+    public MappingService mappingService() {
+        return mappingService;
+    }
+
+    public ExchangeProcessor exchangeProcessor() {
+        return exchangeProcessor;
+    }
+
+    // Helper methods
+
+    public JavaTypeFactory typeFactory() {
+        return planner.getTypeFactory();
+    }
+
+    public NodesMapping mapForLocal() {
+        return mappingService.local();
+    }
+
+    public NodesMapping mapForRandom(AffinityTopologyVersion topVer) {
+        return mappingService.random(topVer);
+    }
+
+    public NodesMapping mapForCache(int cacheId, AffinityTopologyVersion topVer) {
+        return mappingService.distributed(cacheId, topVer);
+    }
+
+    public QueryProvider queryProvider() {
+        return null; // TODO
+    }
+
+    @Override public <C> C unwrap(Class<C> aClass) {
+        if (aClass == getClass())
+            return aClass.cast(this);
+
+        return parentContext.unwrap(aClass);
+    }
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    public static class Builder {
+        private Context parentContext;
+        private Query query;
+        private AffinityTopologyVersion topologyVersion;
+        private SchemaPlus schema;
+        private IgniteLogger logger;
+        private GridKernalContext kernalContext;
+        private CalciteQueryProcessor queryProcessor;
+        private MappingService mappingService;
+        private ExchangeProcessor exchangeProcessor;
+
+        public Builder parentContext(Context parentContext) {
+            this.parentContext = parentContext;
+            return this;
+        }
+
+        public Builder query(Query query) {
+            this.query = query;
+            return this;
+        }
+
+        public Builder topologyVersion(AffinityTopologyVersion topologyVersion) {
+            this.topologyVersion = topologyVersion;
+            return this;
+        }
+
+        public Builder schema(SchemaPlus schema) {
+            this.schema = schema;
+            return this;
+        }
+
+        public Builder logger(IgniteLogger logger) {
+            this.logger = logger;
+            return this;
+        }
+
+        public Builder kernalContext(GridKernalContext kernalContext) {
+            this.kernalContext = kernalContext;
+            return this;
+        }
+
+        public Builder queryProcessor(CalciteQueryProcessor queryProcessor) {
+            this.queryProcessor = queryProcessor;
+            return this;
+        }
+
+        public Builder mappingService(MappingService mappingService) {
+            this.mappingService = mappingService;
+            return this;
+        }
+
+        public Builder exchangeProcessor(ExchangeProcessor exchangeProcessor) {
+            this.exchangeProcessor = exchangeProcessor;
+            return this;
+        }
+
+        public PlannerContext build() {
+            return new PlannerContext(parentContext, query, topologyVersion, schema, logger, kernalContext, queryProcessor, mappingService, exchangeProcessor);
+        }
+    }
+}
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
new file mode 100644
index 0000000..14ef394
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
@@ -0,0 +1,60 @@
+/*
+ * 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.prepare;
+
+import org.apache.calcite.rel.rules.SubQueryRemoveRule;
+import org.apache.calcite.tools.RuleSet;
+import org.apache.calcite.tools.RuleSets;
+import org.apache.ignite.internal.processors.query.calcite.rule.FilterConverter;
+import org.apache.ignite.internal.processors.query.calcite.rule.JoinConverter;
+import org.apache.ignite.internal.processors.query.calcite.rule.ProjectConverter;
+import org.apache.ignite.internal.processors.query.calcite.rule.TableScanConverter;
+
+/**
+ *
+ */
+public enum PlannerPhase {
+    /** */
+    SUBQUERY_REWRITE("Sub-queries rewrites") {
+        @Override public RuleSet getRules(PlannerContext ctx) {
+            return RuleSets.ofList(
+                SubQueryRemoveRule.FILTER,
+                SubQueryRemoveRule.PROJECT,
+                SubQueryRemoveRule.JOIN);
+        }
+    },
+
+    /** */
+    OPTIMIZATION("Main optimization phase") {
+        @Override public RuleSet getRules(PlannerContext ctx) {
+            return RuleSets.ofList(
+                TableScanConverter.INSTANCE,
+                JoinConverter.INSTANCE,
+                ProjectConverter.INSTANCE,
+                FilterConverter.INSTANCE);
+        }
+    };
+
+    public final String description;
+
+    PlannerPhase(String description) {
+        this.description = description;
+    }
+
+    public abstract RuleSet getRules(PlannerContext ctx);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerType.java
new file mode 100644
index 0000000..cddc4d5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerType.java
@@ -0,0 +1,26 @@
+/*
+ * 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.prepare;
+
+/**
+ *
+ */
+public enum PlannerType {
+    HEP,
+    VOLCANO;
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java
new file mode 100644
index 0000000..d18c82c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java
@@ -0,0 +1,73 @@
+/*
+ * 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.prepare;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class Query {
+    private final String sql;
+    private final Object[] params;
+
+    public Query(String sql, Object[] params) {
+        this.sql = sql;
+        this.params = params;
+    }
+
+    public String sql() {
+        return sql;
+    }
+
+    public Object[] params() {
+        return params;
+    }
+
+    public Map<String, Object> params(Map<String, Object> stashed) {
+        Map<String, Object> res = new HashMap<>(stashed);
+        if (!F.isEmpty(params)) {
+            for (int i = 0; i < params.length; i++) {
+                res.put("?" + i, params[i]);
+            }
+        }
+        return res;
+    }
+
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (!(o instanceof Query))
+            return false;
+
+        Query query = (Query) o;
+
+        if (!sql.equals(query.sql))
+            return false;
+        return Arrays.equals(params, query.params);
+    }
+
+    @Override public int hashCode() {
+        int result = sql.hashCode();
+        result = 31 * result + Arrays.hashCode(params);
+        return result;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
new file mode 100644
index 0000000..bcfed58
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
@@ -0,0 +1,28 @@
+/*
+ * 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.prepare;
+
+import java.util.List;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+
+/**
+ *
+ */
+public interface QueryExecution {
+    FieldsQueryCursor<List<?>> execute();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java
new file mode 100644
index 0000000..a0f7878
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteConvention.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.AbstractConverter;
+
+/**
+ *
+ */
+public class IgniteConvention extends Convention.Impl {
+    public static final IgniteConvention INSTANCE = new IgniteConvention();
+
+    private IgniteConvention() {
+        super("IGNITE", IgniteRel.class);
+    }
+
+    @Override public ConventionTraitDef getTraitDef() {
+        return ConventionTraitDef.INSTANCE;
+    }
+
+    @Override public void register(RelOptPlanner planner) {
+        planner.addRule(AbstractConverter.ExpandConversionRule.INSTANCE);
+    }
+
+    @Override public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+        return toTraits.contains(INSTANCE) && fromTraits.contains(INSTANCE);
+    }
+}
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
new file mode 100644
index 0000000..9d01fbb
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteExchange.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Exchange;
+
+/**
+ *
+ */
+public class IgniteExchange extends Exchange implements IgniteRel {
+    public IgniteExchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RelDistribution distribution) {
+        super(cluster, traitSet, input, distribution);
+    }
+
+    @Override public Exchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution) {
+        return new IgniteExchange(getCluster(), traitSet, newInput, newDistribution);
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}
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
new file mode 100644
index 0000000..c408769
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteFilter.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ *
+ */
+public class IgniteFilter extends Filter implements IgniteRel {
+    public IgniteFilter(RelOptCluster cluster, RelTraitSet traits, RelNode input, RexNode condition) {
+        super(cluster, traits, input, condition);
+    }
+
+    @Override public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+        return new IgniteFilter(getCluster(), traitSet, input, condition);
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}
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
new file mode 100644
index 0000000..b1ad44b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteJoin.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+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.rex.RexNode;
+
+/**
+ *
+ */
+public class IgniteJoin extends Join implements IgniteRel {
+    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);
+    }
+
+    @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);
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}
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
new file mode 100644
index 0000000..42d2af2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteProject.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+/**
+ *
+ */
+public class IgniteProject extends Project implements IgniteRel {
+    public IgniteProject(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
+        super(cluster, traits, input, projects, rowType);
+    }
+
+    @Override public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType) {
+        return new IgniteProject(getCluster(), traitSet, input, projects, rowType);
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
new file mode 100644
index 0000000..545c6e8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public class IgniteReceiver extends AbstractRelNode implements IgniteRel {
+    private RelSource source;
+
+    public IgniteReceiver(RelOptCluster cluster, RelTraitSet traits, RelDataType rowType, RelSource source) {
+        super(cluster, traits);
+
+        this.rowType = rowType;
+        this.source = source;
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return new IgniteReceiver(getCluster(), traitSet, rowType, source);
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    public RelSource source() {
+        return source;
+    }
+
+    public void source(RelSource source) {
+        this.source = source;
+    }
+
+    public IgniteDistribution distribution() {
+        return getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+    }
+}
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
new file mode 100644
index 0000000..8e1567a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRel.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.rel.RelNode;
+
+/**
+ *
+ */
+public interface IgniteRel extends RelNode {
+    <T> T accept(IgniteRelVisitor<T> visitor);
+}
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
new file mode 100644
index 0000000..6c6ffa1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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;
+
+/**
+ *
+ */
+public interface IgniteRelVisitor<T> {
+    T visit(IgniteSender rel);
+
+    T visit(IgniteFilter rel);
+
+    T visit(IgniteProject rel);
+
+    T visit(IgniteJoin rel);
+
+    T visit(IgniteTableScan rel);
+
+    T visit(IgniteReceiver rel);
+
+    T visit(IgniteExchange rel);
+
+    T visit(IgniteRel other);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java
new file mode 100644
index 0000000..4763c38
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteSender.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelTarget;
+
+/**
+ *
+ */
+public class IgniteSender extends SingleRel implements IgniteRel {
+    private RelTarget target;
+
+    public IgniteSender(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelTarget target) {
+        super(cluster, traits, input);
+
+        this.target = target;
+    }
+
+    public IgniteSender(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
+        super(cluster, traits, input);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return new IgniteSender(getCluster(), traitSet, sole(inputs), target);
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    public RelTarget target() {
+        return target;
+    }
+
+    public void target(RelTarget target) {
+        this.target = target;
+    }
+}
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/IgniteTableScan.java
new file mode 100644
index 0000000..59af858
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteTableScan.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+
+/**
+ *
+ */
+public class IgniteTableScan extends TableScan implements IgniteRel {
+    public IgniteTableScan(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+        super(cluster, traitSet, table);
+    }
+
+    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return this;
+    }
+
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java
new file mode 100644
index 0000000..fb073c9
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java
@@ -0,0 +1,27 @@
+/*
+ * 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 org.apache.calcite.rel.RelNode;
+
+/**
+ *
+ */
+public interface RelOp<T extends RelNode, R> {
+    R go(T rel);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/FilterConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/FilterConverter.java
new file mode 100644
index 0000000..56587ba
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/FilterConverter.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class FilterConverter extends IgniteConverter {
+    public static final ConverterRule INSTANCE = new FilterConverter();
+
+    public FilterConverter() {
+        super(LogicalFilter.class, "FilterConverter");
+    }
+
+    @Override protected List<RelNode> convert0(RelNode rel) {
+        LogicalFilter filter = (LogicalFilter) rel;
+
+        RelNode input = convert(filter.getInput(), IgniteConvention.INSTANCE);
+
+        RelOptCluster cluster = rel.getCluster();
+        RelMetadataQuery mq = cluster.getMetadataQuery();
+
+        List<IgniteDistribution> distrs = IgniteMdDerivedDistribution.deriveDistributions(input, IgniteConvention.INSTANCE, mq);
+
+        return Commons.transform(distrs, d -> create(filter, input, d));
+    }
+
+    private static IgniteFilter create(LogicalFilter filter, RelNode input, IgniteDistribution distr) {
+        RelTraitSet traits = filter.getTraitSet()
+            .replace(distr)
+            .replace(IgniteConvention.INSTANCE);
+
+        return new IgniteFilter(filter.getCluster(), traits, convert(input, distr), filter.getCondition());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/IgniteConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/IgniteConverter.java
new file mode 100644
index 0000000..4653fec
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/IgniteConverter.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 com.google.common.collect.ImmutableMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public abstract class IgniteConverter extends ConverterRule {
+    protected IgniteConverter(Class<? extends RelNode> clazz, String descriptionPrefix) {
+        super(clazz, Convention.NONE, IgniteConvention.INSTANCE, descriptionPrefix);
+    }
+
+    @Override public void onMatch(RelOptRuleCall call) {
+        RelNode rel = call.rel(0);
+        if (rel.getTraitSet().contains(Convention.NONE)) {
+            List<RelNode> rels = convert0(rel);
+            if (F.isEmpty(rels))
+                return;
+
+            Map<RelNode, RelNode> equiv = ImmutableMap.of();
+
+            if (rels.size() > 1) {
+                equiv = new HashMap<>();
+
+                for (int i = 1; i < rels.size(); i++) {
+                    equiv.put(rels.get(i), rel);
+                }
+            }
+
+            call.transformTo(F.first(rels), equiv);
+        }
+    }
+
+    @Override public RelNode convert(RelNode rel) {
+        List<RelNode> converted = convert0(rel);
+
+        if (converted.size() > 1) {
+            RelOptPlanner planner = rel.getCluster().getPlanner();
+
+            for (int i = 1; i < converted.size(); i++)
+                planner.ensureRegistered(converted.get(i), rel);
+        }
+
+        return F.first(converted);
+    }
+
+    protected abstract List<RelNode> convert0(RelNode rel);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverter.java
new file mode 100644
index 0000000..b10fe20
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/JoinConverter.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDerivedDistribution;
+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.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class JoinConverter extends IgniteConverter {
+    public static final ConverterRule INSTANCE = new JoinConverter();
+
+    public JoinConverter() {
+        super(LogicalJoin.class, "JoinConverter");
+    }
+
+    @Override protected List<RelNode> convert0(RelNode rel) {
+        LogicalJoin join = (LogicalJoin) rel;
+
+        RelNode left = convert(join.getLeft(), IgniteConvention.INSTANCE);
+        RelNode right = convert(join.getRight(), IgniteConvention.INSTANCE);
+
+        RelOptCluster cluster = join.getCluster();
+        RelMetadataQuery mq = cluster.getMetadataQuery();
+
+        List<IgniteDistribution> leftTraits = IgniteMdDerivedDistribution.deriveDistributions(left, IgniteConvention.INSTANCE, mq);
+        List<IgniteDistribution> rightTraits = IgniteMdDerivedDistribution.deriveDistributions(left, IgniteConvention.INSTANCE, mq);
+
+        List<IgniteDistributions.BiSuggestion> suggestions = IgniteDistributions.suggestJoin(leftTraits, rightTraits, join.analyzeCondition(), join.getJoinType());
+
+        return Commons.transform(suggestions, s -> create(join, left, right, s));
+    }
+
+    private static RelNode create(LogicalJoin join, RelNode left, RelNode right, IgniteDistributions.BiSuggestion suggest) {
+        left = convert(left, suggest.left());
+        right = convert(right, suggest.right());
+
+        RelTraitSet traitSet = join.getTraitSet().replace(IgniteConvention.INSTANCE).replace(suggest.out());
+
+        return new IgniteJoin(join.getCluster(), traitSet, left, right, join.getCondition(), join.getVariablesSet(), join.getJoinType());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ProjectConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ProjectConverter.java
new file mode 100644
index 0000000..2fe1147
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/ProjectConverter.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class ProjectConverter extends IgniteConverter {
+    public static final ConverterRule INSTANCE = new ProjectConverter();
+
+    public ProjectConverter() {
+        super(LogicalProject.class, "ProjectConverter");
+    }
+
+    @Override protected List<RelNode> convert0(RelNode rel) {
+        LogicalProject project = (LogicalProject) rel;
+
+        RelNode input = convert(project.getInput(), IgniteConvention.INSTANCE);
+
+        RelOptCluster cluster = rel.getCluster();
+        RelMetadataQuery mq = cluster.getMetadataQuery();
+
+        List<IgniteDistribution> distrs = IgniteMdDerivedDistribution.deriveDistributions(input, IgniteConvention.INSTANCE, mq);
+
+        return Commons.transform(distrs, d -> create(project, input, d));
+    }
+
+    private static IgniteProject create(LogicalProject project, RelNode input, IgniteDistribution distr) {
+        RelTraitSet traits = project.getTraitSet()
+            .replace(IgniteMdDistribution.project(input.getRowType(), distr, project.getProjects()))
+            .replace(IgniteConvention.INSTANCE);
+
+        return new IgniteProject(project.getCluster(), traits, convert(input, distr), project.getProjects(), project.getRowType());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableScanConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableScanConverter.java
new file mode 100644
index 0000000..2d5aea0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableScanConverter.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+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.util.typedef.F;
+
+/**
+ *
+ */
+public class TableScanConverter extends IgniteConverter {
+    public static final ConverterRule INSTANCE = new TableScanConverter();
+
+    public TableScanConverter() {
+        super(LogicalTableScan.class, "TableScanConverter");
+    }
+
+    @Override protected List<RelNode> convert0(RelNode rel) {
+        LogicalTableScan scan = (LogicalTableScan) rel;
+
+        RelTraitSet traitSet = scan.getTraitSet().replace(IgniteConvention.INSTANCE);
+        return F.asList(new IgniteTableScan(rel.getCluster(), traitSet, scan.getTable()));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
new file mode 100644
index 0000000..bad106f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
@@ -0,0 +1,69 @@
+/*
+ * 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.schema;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+
+/**
+ *
+ */
+public class CalciteSchemaHolder implements SchemaChangeListener {
+    private final Map<String, IgniteSchema> schemas = new HashMap<>();
+    private volatile SchemaPlus schema;
+
+    public void schema(SchemaPlus schema) {
+        this.schema = schema;
+    }
+
+    public SchemaPlus schema() {
+        return schema;
+    }
+
+    @Override public synchronized void onSchemaCreate(String schemaName) {
+        schemas.putIfAbsent(schemaName, new IgniteSchema(schemaName));
+        rebuild();
+    }
+
+    @Override public synchronized void onSchemaDrop(String schemaName) {
+        schemas.remove(schemaName);
+        rebuild();
+    }
+
+    @Override public synchronized void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
+        schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeCreate(typeDescriptor, cacheInfo);
+        rebuild();
+    }
+
+    @Override public synchronized void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
+        schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeDrop(typeDescriptor, cacheInfo);
+        rebuild();
+    }
+
+    private void rebuild() {
+        SchemaPlus schema = Frameworks.createRootSchema(false);
+        schemas.forEach(schema::add);
+        schema(schema.getSubSchema(QueryUtils.DFLT_SCHEMA));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
new file mode 100644
index 0000000..271c5f6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.schema;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class IgniteSchema extends AbstractSchema {
+    /** */
+    private final String schemaName;
+
+    /** */
+    private final Map<String, Table> tableMap = new ConcurrentHashMap<>();
+
+    public IgniteSchema(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    public String getName() {
+        return schemaName;
+    }
+
+    @Override protected Map<String, Table> getTableMap() {
+        return Collections.unmodifiableMap(tableMap);
+    }
+
+    /**
+     * Callback method.
+     *
+     * @param typeDesc Query type descriptor.
+     * @param cacheInfo Cache info.
+     */
+    public void onSqlTypeCreate(GridQueryTypeDescriptor typeDesc, GridCacheContextInfo cacheInfo) {
+        Object identityKey = cacheInfo.config().getCacheMode() == CacheMode.PARTITIONED ?
+            cacheInfo.cacheContext().group().affinity().similarAffinityKey() : null;
+
+        addTable(new IgniteTable(typeDesc.tableName(), cacheInfo.name(), Commons.rowType(typeDesc), identityKey));
+    }
+
+    /**
+     * Callback method.
+     *
+     * @param typeDesc Query type descriptor.
+     * @param cacheInfo Cache info.
+     */
+    public void onSqlTypeDrop(GridQueryTypeDescriptor typeDesc, GridCacheContextInfo cacheInfo) {
+        removeTable(typeDesc.tableName());
+    }
+
+    /**
+     * @param table Table.
+     */
+    public void addTable(IgniteTable table) {
+        tableMap.put(table.tableName(), table);
+    }
+
+    /**
+     * @param tableName Table name.
+     */
+    public void removeTable(String tableName) {
+        tableMap.remove(tableName);
+    }
+}
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
new file mode 100644
index 0000000..56e9302
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
@@ -0,0 +1,137 @@
+/*
+ * 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.schema;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelReferentialConstraint;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentInfo;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.AffinityFactory;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+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.type.RowType;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+/** */
+public class IgniteTable extends AbstractTable implements TranslatableTable, ScannableTable {
+    private final String tableName;
+    private final String cacheName;
+    private final RowType rowType;
+    private final Object identityKey;
+
+    public IgniteTable(String tableName, String cacheName, RowType rowType, Object identityKey) {
+        this.tableName = tableName;
+        this.cacheName = cacheName;
+        this.rowType = rowType;
+        this.identityKey = identityKey;
+    }
+
+    /**
+     * @return Table name;
+     */
+    public String tableName() {
+        return tableName;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+        return rowType.asRelDataType(typeFactory);
+    }
+
+    @Override public Statistic getStatistic() {
+        return new TableStatistics();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
+        RelOptCluster cluster = context.getCluster();
+        RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
+                .replaceIf(DistributionTraitDef.INSTANCE, this::getDistribution);
+
+        return new LogicalTableScan(cluster, traitSet, relOptTable);
+    }
+
+    public IgniteDistribution getDistribution() {
+        Object key = identityKey();
+
+        if (key == null)
+            return IgniteDistributions.broadcast();
+
+        return IgniteDistributions.hash(rowType.distributionKeys(), new AffinityFactory(CU.cacheId(cacheName), key));
+    }
+
+    protected Object identityKey() {
+        return identityKey;
+    }
+
+    public FragmentInfo fragmentInfo(PlannerContext ctx) {
+        return new FragmentInfo(ctx.mapForCache(CU.cacheId(cacheName), ctx.topologyVersion()));
+    }
+
+    @Override public Enumerable<Object[]> scan(DataContext root) {
+        throw new AssertionError(); // TODO
+    }
+
+    private class TableStatistics implements Statistic {
+        @Override public Double getRowCount() {
+            return null;
+        }
+
+        @Override public boolean isKey(ImmutableBitSet columns) {
+            return false;
+        }
+
+        @Override public List<RelReferentialConstraint> getReferentialConstraints() {
+            return ImmutableList.of();
+        }
+
+        @Override public List<RelCollation> getCollations() {
+            return ImmutableList.of();
+        }
+
+        @Override public RelDistribution getDistribution() {
+            return IgniteTable.this.getDistribution();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java
new file mode 100644
index 0000000..0427a5f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.GridIntList;
+
+/**
+ *
+ */
+public class Graph<T extends GraphNode> implements Serializable {
+    private final List<T> nodes = new ArrayList<>();
+    private final List<GridIntList> edges = new ArrayList<>();
+
+    public List<Ord<T>> nodes() {
+        return Ord.zip(nodes);
+    }
+
+    public List<GridIntList> edges() {
+        return Commons.transform(edges, GridIntList::copy);
+    }
+
+    public int addNode(int parentId, T node) {
+        int id = addNode(node);
+
+        addEdge(parentId, id);
+
+        return id;
+    }
+
+    public int addNode(T node) {
+        assert nodes.size() == edges.size();
+
+        int id = nodes.size();
+
+        nodes.add(node);
+        edges.add(new GridIntList());
+
+        return id;
+    }
+
+    public void addEdge(int parentId, int childId) {
+        assert parentId == -1 || (parentId >= 0 && parentId < edges.size());
+        assert nodes.size() == edges.size();
+
+        if (parentId != -1)
+            edges.get(parentId).add(childId);
+    }
+
+    public List<Ord<T>> children(int parentId) {
+        GridIntList children = edges.get(parentId);
+
+        ArrayList<Ord<T>> ords = new ArrayList<>(children.size());
+
+        for (int i = 0; i < children.size(); i++)
+            ords.add(Ord.of(children.get(i), nodes.get(children.get(i))));
+
+        return ords;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
new file mode 100644
index 0000000..82e3f22
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public interface GraphNode extends Serializable {
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
new file mode 100644
index 0000000..ab75446
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.util.List;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSyntax;
+
+/**
+ *
+ */
+public class CallExpression implements Expression {
+    public final String opName;
+    public final SqlSyntax opSyntax;
+    public final List<Expression> operands;
+
+    public CallExpression(SqlOperator op, List<Expression> operands) {
+        this.operands = operands;
+        opName = op.getName();
+        opSyntax = op.getSyntax();
+    }
+
+    @Override public <T> T implement(ExpImplementor<T> implementor) {
+        return implementor.implement(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
new file mode 100644
index 0000000..94b2b85
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class DynamicParamExpression implements Expression {
+    public final DataType type;
+    public final int index;
+
+    public DynamicParamExpression(RelDataType type, int index) {
+        this.type = DataType.fromType(type);
+        this.index = index;
+    }
+
+    @Override public <T> T implement(ExpImplementor<T> implementor) {
+        return implementor.implement(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
new file mode 100644
index 0000000..2997c7d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+/**
+ *
+ */
+public interface ExpImplementor<T> {
+    T implement(CallExpression callExpression);
+
+    T implement(InputRefExpression inputRefExpression);
+
+    T implement(LiteralExpression literalExpression);
+
+    T implement(LocalRefExpression localRefExpression);
+
+    T implement(DynamicParamExpression dynamicParamExpression);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java
new file mode 100644
index 0000000..946e483
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class ExpToRexTranslator implements ExpImplementor<RexNode> {
+    private final RexBuilder builder;
+    private final RelDataTypeFactory typeFactory;
+    private final Map<Pair<String, SqlSyntax>, SqlOperator> ops;
+
+    public ExpToRexTranslator(RexBuilder builder, RelDataTypeFactory typeFactory, SqlOperatorTable opTable) {
+        this.builder = builder;
+        this.typeFactory = typeFactory;
+
+        List<SqlOperator> opList = opTable.getOperatorList();
+
+        HashMap<Pair<String, SqlSyntax>, SqlOperator> ops = new HashMap<>(opList.size());
+
+        for (SqlOperator op : opList) {
+            ops.put(Pair.of(op.getName(), op.getSyntax()), op);
+        }
+
+        this.ops = ops;
+    }
+
+    public List<RexNode> translate(List<Expression> exps) {
+        if (F.isEmpty(exps))
+            return Collections.emptyList();
+
+        if (exps.size() == 1)
+            return F.asList(translate(F.first(exps)));
+
+        List<RexNode> res = new ArrayList<>(exps.size());
+
+        for (Expression exp : exps) {
+            res.add(exp.implement(this));
+        }
+
+        return res;
+    }
+
+    public RexNode translate(Expression exp) {
+        return exp.implement(this);
+    }
+
+    @Override public RexNode implement(CallExpression exp) {
+        return builder.makeCall(op(exp.opName, exp.opSyntax), translate(exp.operands));
+    }
+
+    @Override public RexNode implement(InputRefExpression exp) {
+        return builder.makeInputRef(exp.type.toRelDataType(typeFactory), exp.index);
+    }
+
+    @Override public RexNode implement(LiteralExpression exp) {
+        return builder.makeLiteral(exp.value, exp.type.toRelDataType(typeFactory), false);
+    }
+
+    @Override public RexNode implement(LocalRefExpression exp) {
+        return new RexLocalRef(exp.index, exp.type.toRelDataType(typeFactory));
+    }
+
+    @Override public RexNode implement(DynamicParamExpression exp) {
+        return builder.makeDynamicParam(exp.type.toRelDataType(typeFactory), exp.index);
+    }
+
+    private SqlOperator op(String name, SqlSyntax syntax) {
+        return ops.get(Pair.of(name, syntax));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java
new file mode 100644
index 0000000..7e92b6f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public interface Expression extends Serializable {
+    <T> T implement(ExpImplementor<T> implementor);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
new file mode 100644
index 0000000..b561b1e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class InputRefExpression implements Expression {
+    public final DataType type;
+    public final int index;
+
+    public InputRefExpression(RelDataType type, int index) {
+        this.type = DataType.fromType(type);
+        this.index = index;
+    }
+
+    @Override public <T> T implement(ExpImplementor<T> implementor) {
+        return implementor.implement(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java
new file mode 100644
index 0000000..038b142
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class LiteralExpression implements Expression {
+    public final DataType type;
+    public final Comparable value;
+
+    public LiteralExpression(RelDataType type, Comparable value) {
+        this.type = DataType.fromType(type);
+        this.value = value;
+    }
+
+    @Override public <T> T implement(ExpImplementor<T> implementor) {
+        return implementor.implement(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java
new file mode 100644
index 0000000..e6023b8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+
+/**
+ *
+ */
+public class LocalRefExpression implements Expression {
+    public final DataType type;
+    public final int index;
+
+    public LocalRefExpression(RelDataType type, int index) {
+        this.type = DataType.fromType(type);
+        this.index = index;
+    }
+
+    @Override public <T> T implement(ExpImplementor<T> implementor) {
+        return implementor.implement(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java
new file mode 100644
index 0000000..22db0db
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.expression;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexVisitor;
+
+/**
+ *
+ */
+public class RexToExpTranslator implements RexVisitor<Expression> {
+    public List<Expression> translate(List<RexNode> operands) {
+        ArrayList<Expression> res = new ArrayList<>(operands.size());
+
+        for (RexNode operand : operands) {
+            res.add(translate(operand));
+        }
+
+        return res;
+    }
+
+    public Expression translate(RexNode rex) {
+        return rex.accept(this);
+    }
+
+     @Override public Expression visitInputRef(RexInputRef inputRef) {
+        return new InputRefExpression(inputRef.getType(), inputRef.getIndex());
+    }
+
+    @Override public Expression visitLocalRef(RexLocalRef localRef) {
+        return new LocalRefExpression(localRef.getType(), localRef.getIndex());
+    }
+
+    @Override public Expression visitLiteral(RexLiteral literal) {
+        return new LiteralExpression(literal.getType(), literal.getValue());
+    }
+
+    @Override public Expression visitCall(RexCall call) {
+        return new CallExpression(call.getOperator(), translate(call.getOperands()));
+    }
+
+    @Override public Expression visitOver(RexOver over) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Expression visitCorrelVariable(RexCorrelVariable correlVariable) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Expression visitDynamicParam(RexDynamicParam dynamicParam) {
+        return new DynamicParamExpression(dynamicParam.getType(), dynamicParam.getIndex());
+    }
+
+    @Override public Expression visitRangeRef(RexRangeRef rangeRef) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Expression visitFieldAccess(RexFieldAccess fieldAccess) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Expression visitSubQuery(RexSubQuery subQuery) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Expression visitTableInputRef(RexTableInputRef fieldRef) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override public Expression visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java
new file mode 100644
index 0000000..0d8399e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
+
+/**
+ *
+ */
+public interface ConversionContext extends RelOptTable.ToRelContext {
+    RelDataTypeFactory getTypeFactory();
+
+    RelOptSchema getSchema();
+
+    PlannerContext getContext();
+
+    ExpToRexTranslator getExpressionTranslator();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java
new file mode 100644
index 0000000..1fd3da2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class FilterNode extends RelGraphNode {
+    private final Expression condition;
+
+    private FilterNode(Expression condition) {
+        this.condition = condition;
+    }
+
+    public static FilterNode create(IgniteFilter rel, RexToExpTranslator expTranslator) {
+        return new FilterNode(expTranslator.translate(rel.getCondition()));
+    }
+
+    @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+        RelNode input = F.first(children);
+        RexNode condition = this.condition.implement(ctx.getExpressionTranslator());
+        RelOptCluster cluster = input.getCluster();
+        RelMetadataQuery mq = cluster.getMetadataQuery();
+
+        RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE)
+            .replaceIf(DistributionTraitDef.INSTANCE, () -> IgniteMdDistribution.filter(mq, input, condition));
+
+        return new IgniteFilter(cluster, traits, input, condition);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java
new file mode 100644
index 0000000..f7574bf
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class GraphToRelConverter implements ConversionContext {
+    private final RelOptTable.ViewExpander viewExpander;
+    private final RelBuilder relBuilder;
+    private final ExpToRexTranslator expTranslator;
+
+    public GraphToRelConverter(RelOptTable.ViewExpander viewExpander, RelBuilder relBuilder, SqlOperatorTable operatorTable) {
+        this.viewExpander = viewExpander;
+        this.relBuilder = relBuilder;
+
+        expTranslator = new ExpToRexTranslator(
+            relBuilder.getRexBuilder(),
+            getTypeFactory(),
+            operatorTable);
+    }
+
+    @Override public RelDataTypeFactory getTypeFactory() {
+        return getCluster().getTypeFactory();
+    }
+
+    @Override public RelOptSchema getSchema() {
+        return relBuilder.getRelOptSchema();
+    }
+
+    @Override public PlannerContext getContext() {
+        return Commons.plannerContext(getCluster().getPlanner().getContext());
+    }
+
+    @Override public ExpToRexTranslator getExpressionTranslator() {
+        return expTranslator;
+    }
+
+    @Override public RelOptCluster getCluster() {
+        return relBuilder.getCluster();
+    }
+
+    @Override public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
+        return viewExpander.expandView(rowType, queryString, schemaPath, viewPath);
+    }
+
+    public RelNode convert(RelGraph graph) {
+        return F.first(convertRecursive(this, graph, graph.nodes().subList(0, 1)));
+    }
+
+    private List<RelNode> convertRecursive(ConversionContext ctx, RelGraph graph, List<Ord<RelGraphNode>> src) {
+        ImmutableList.Builder<RelNode> b = ImmutableList.builder();
+
+        for (Ord<RelGraphNode> node : src)
+            b.add(node.e.toRel(ctx, convertRecursive(ctx, graph, graph.children(node.i))));
+
+        return b.build();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java
new file mode 100644
index 0000000..c26d2c7
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+
+/**
+ *
+ */
+public class JoinNode extends RelGraphNode {
+    private final Expression condition;
+    private final int[] variables;
+    private final JoinRelType joinType;
+
+    private JoinNode(RelTraitSet traits, Expression condition, int[] variables, JoinRelType joinType) {
+        super(traits);
+        this.condition = condition;
+        this.variables = variables;
+        this.joinType = joinType;
+    }
+
+    public static JoinNode create(IgniteJoin rel, RexToExpTranslator expTranslator) {
+        return new JoinNode(rel.getTraitSet(),
+            expTranslator.translate(rel.getCondition()),
+            rel.getVariablesSet().stream().mapToInt(CorrelationId::getId).toArray(),
+            rel.getJoinType());
+    }
+
+    @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+        assert children.size() == 2;
+
+        RelNode left = children.get(0);
+        RelNode right = children.get(1);
+
+        return new IgniteJoin(ctx.getCluster(),
+            traitSet.toTraitSet(ctx.getCluster()),
+            left,
+            right,
+            ctx.getExpressionTranslator().translate(condition),
+            Arrays.stream(variables).mapToObj(CorrelationId::new).collect(Collectors.toSet()),
+            joinType);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java
new file mode 100644
index 0000000..56c60df
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class ProjectNode extends RelGraphNode {
+    private final List<Expression> projects;
+    private final DataType dataType;
+
+    private ProjectNode(List<Expression> projects, DataType dataType) {
+        this.projects = projects;
+        this.dataType = dataType;
+    }
+
+    public static ProjectNode create(IgniteProject rel, RexToExpTranslator rexTranslator) {
+        return new ProjectNode(rexTranslator.translate(rel.getProjects()),
+            DataType.fromType(rel.getRowType()));
+    }
+
+    @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+        RelNode input = F.first(children);
+        List<RexNode> projects = ctx.getExpressionTranslator().translate(this.projects);
+        RelOptCluster cluster = input.getCluster();
+        RelMetadataQuery mq = cluster.getMetadataQuery();
+
+        RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE)
+            .replaceIf(DistributionTraitDef.INSTANCE, () -> IgniteMdDistribution.project(mq, input, projects));
+
+        return new IgniteProject(cluster, traits, input, projects, dataType.toRelDataType(ctx.getTypeFactory()));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java
new file mode 100644
index 0000000..96bc0dd
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSourceImpl;
+
+
+/**
+ *
+ */
+public class ReceiverNode extends RelGraphNode {
+    private final DataType dataType;
+    private final RelSource source;
+
+    private ReceiverNode(RelTraitSet traits, DataType dataType, RelSource source) {
+        super(traits);
+        this.dataType = dataType;
+        this.source = source;
+    }
+
+    public static ReceiverNode create(IgniteReceiver rel) {
+        RelSource source = new RelSourceImpl(rel.source().exchangeId(), rel.source().mapping());
+
+        return new ReceiverNode(rel.getTraitSet(), DataType.fromType(rel.getRowType()), source);
+    }
+
+    @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+        return new IgniteReceiver(ctx.getCluster(),
+            traitSet.toTraitSet(ctx.getCluster()),
+            dataType.toRelDataType(ctx.getTypeFactory()),
+            source);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java
new file mode 100644
index 0000000..bb38efe
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import org.apache.ignite.internal.processors.query.calcite.serialize.Graph;
+
+/**
+ *
+ */
+public class RelGraph extends Graph<RelGraphNode> {
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java
new file mode 100644
index 0000000..ab53320
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.calcite.serialize.GraphNode;
+
+/**
+ *
+ */
+public abstract class RelGraphNode implements GraphNode {
+    protected SerializedTraits traitSet;
+
+    protected RelGraphNode() {
+    }
+
+    protected RelGraphNode(RelTraitSet traits) {
+        traitSet = new SerializedTraits(traits);
+    }
+
+    public abstract RelNode toRel(ConversionContext ctx, List<RelNode> children);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java
new file mode 100644
index 0000000..56d77aa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.ArrayDeque;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+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.IgniteRel;
+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.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class RelToGraphConverter implements RelOp<IgniteRel, RelGraph> {
+    private final RexToExpTranslator rexTranslator = new RexToExpTranslator();
+
+    private RelGraph graph;
+    private int curParent;
+
+    private static final class Item {
+        final int parentId;
+        final List<IgniteRel> children;
+
+        private Item(int parentId, List<IgniteRel> children) {
+            this.parentId = parentId;
+            this.children = children;
+        }
+    }
+
+    private final class ItemTranslator implements IgniteRelVisitor<Item> {
+        @Override public Item visit(IgniteFilter rel) {
+            return new Item(graph.addNode(curParent, FilterNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
+        }
+
+        @Override public Item visit(IgniteJoin rel) {
+            return new Item(graph.addNode(curParent, JoinNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
+        }
+
+        @Override public Item visit(IgniteProject rel) {
+            return new Item(graph.addNode(curParent, ProjectNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
+        }
+
+        @Override public Item visit(IgniteTableScan rel) {
+            return new Item(graph.addNode(curParent, TableScanNode.create(rel)), Commons.cast(rel.getInputs()));
+        }
+
+        @Override public Item visit(IgniteReceiver rel) {
+            return new Item(graph.addNode(curParent, ReceiverNode.create(rel)), Collections.emptyList());
+        }
+
+        @Override public Item visit(IgniteSender rel) {
+            return new Item(graph.addNode(curParent, SenderNode.create(rel)), Commons.cast(rel.getInputs()));
+        }
+
+        @Override public Item visit(IgniteRel rel) {
+            return rel.accept(this);
+        }
+
+        @Override public Item visit(IgniteExchange rel) {
+            throw new AssertionError("Unexpected node: " + rel);
+        }
+    }
+
+    @Override public RelGraph go(IgniteRel root) {
+        graph = new RelGraph();
+
+        ItemTranslator itemTranslator = new ItemTranslator();
+        Deque<Item> stack = new ArrayDeque<>();
+        stack.push(new Item(-1, F.asList(root)));
+
+        while (!stack.isEmpty()) {
+            Item item = stack.pop();
+
+            curParent = item.parentId;
+
+            for (IgniteRel child : item.children) {
+                stack.push(itemTranslator.visit(child));
+            }
+        }
+
+        return graph;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java
new file mode 100644
index 0000000..2940015
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdDistribution;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelTarget;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class SenderNode extends RelGraphNode {
+    private final RelTarget target;
+
+    private SenderNode(RelTarget target) {
+        this.target = target;
+    }
+
+    public static SenderNode create(IgniteSender rel) {
+        return new SenderNode(rel.target());
+    }
+
+    @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+        RelNode input = F.first(children);
+        RelOptCluster cluster = input.getCluster();
+        RelMetadataQuery mq = cluster.getMetadataQuery();
+
+        RelTraitSet traits = cluster.traitSet()
+            .replace(IgniteConvention.INSTANCE)
+            .replaceIf(DistributionTraitDef.INSTANCE, () -> IgniteMdDistribution._distribution(input, mq));
+
+        return new IgniteSender(cluster, traits, input, target);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SerializedTraits.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SerializedTraits.java
new file mode 100644
index 0000000..94efe86
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SerializedTraits.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class SerializedTraits implements Serializable {
+    private static final Byte CONVENTION = 0;
+
+    private final List<Serializable> traits;
+
+    public SerializedTraits(RelTraitSet traits) {
+        this.traits = Commons.transform(traits, this::toSerializable);
+    }
+
+    public RelTraitSet toTraitSet(RelOptCluster cluster) {
+        RelTraitSet traits = cluster.traitSet();
+
+        for (Serializable trait : this.traits)
+            traits.replace(fromSerializable(trait));
+
+        return traits.simplify();
+    }
+
+    private Serializable toSerializable(RelTrait trait) {
+        if (trait instanceof Serializable)
+            return (Serializable) trait;
+        if (trait == IgniteConvention.INSTANCE)
+            return CONVENTION;
+
+        throw new AssertionError();
+    }
+
+    private RelTrait fromSerializable(Serializable trait) {
+        if (trait instanceof RelTrait)
+            return (RelTrait) trait;
+        if (CONVENTION.equals(trait))
+            return IgniteConvention.INSTANCE;
+
+        throw new AssertionError();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java
new file mode 100644
index 0000000..097a8fa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.relation;
+
+import java.util.List;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+
+/**
+ *
+ */
+public class TableScanNode extends RelGraphNode {
+    private final List<String> tableName;
+
+    private TableScanNode(RelTraitSet traits, List<String> tableName) {
+        super(traits);
+        this.tableName = tableName;
+    }
+
+    public static TableScanNode create(IgniteTableScan rel) {
+        return new TableScanNode(rel.getTraitSet(), rel.getTable().getQualifiedName());
+    }
+
+    @Override public RelNode toRel(ConversionContext ctx, List<RelNode> children) {
+        return new IgniteTableScan(ctx.getCluster(),
+            traitSet.toTraitSet(ctx.getCluster()),
+            ctx.getSchema().getTableForMember(tableName));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java
new file mode 100644
index 0000000..317c95c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.type;
+
+import java.io.Serializable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+
+/**
+ *
+ */
+public interface DataType extends Serializable {
+    static DataType fromType(RelDataType type) {
+        return type.isStruct() ? StructType.fromType(type) : SimpleType.fromType(type);
+    }
+
+    RelDataType toRelDataType(RelDataTypeFactory factory);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/SimpleType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/SimpleType.java
new file mode 100644
index 0000000..a5bb3f4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/SimpleType.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ *
+ */
+public class SimpleType implements DataType {
+    private final Class clazz;
+    private final SqlTypeName typeName;
+    private final int precision;
+    private final int scale;
+
+    public static SimpleType fromType(RelDataType type) {
+        assert !type.isStruct();
+
+        if (type instanceof RelDataTypeFactoryImpl.JavaType)
+            return new SimpleType(((RelDataTypeFactoryImpl.JavaType) type).getJavaClass(), null, 0, 0);
+
+        return new SimpleType(null, type.getSqlTypeName(), type.getPrecision(), type.getScale());
+    }
+
+    private SimpleType(Class clazz, SqlTypeName typeName, int precision, int scale) {
+        this.clazz = clazz;
+        this.typeName = typeName;
+        this.precision = precision;
+        this.scale = scale;
+    }
+
+    @Override public RelDataType toRelDataType(RelDataTypeFactory factory) {
+        if (clazz != null)
+            return factory.createJavaType(clazz);
+        if (typeName.allowsNoPrecNoScale())
+            return factory.createSqlType(typeName);
+        if (typeName.allowsPrecNoScale())
+            return factory.createSqlType(typeName, precision);
+
+        return factory.createSqlType(typeName, precision, scale);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/StructType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/StructType.java
new file mode 100644
index 0000000..c712754
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/StructType.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.serialize.type;
+
+import java.util.LinkedHashMap;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+
+/**
+ *
+ */
+public class StructType implements DataType {
+    private final LinkedHashMap<String, DataType> fields;
+
+    static StructType fromType(RelDataType type) {
+        assert type.isStruct();
+
+        LinkedHashMap<String, DataType> fields = new LinkedHashMap<>();
+
+        for (RelDataTypeField field : type.getFieldList()) {
+            fields.put(field.getName(), DataType.fromType(field.getType()));
+        }
+
+        return new StructType(fields);
+    }
+
+    private StructType(LinkedHashMap<String, DataType> fields) {
+        this.fields = fields;
+    }
+
+    @Override public RelDataType toRelDataType(RelDataTypeFactory factory) {
+        RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(factory);
+        fields.forEach((n,f) -> builder.add(n,f.toRelDataType(factory)));
+        return builder.build();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Edge.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Edge.java
new file mode 100644
index 0000000..a065dbe
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Edge.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import org.apache.calcite.rel.RelNode;
+
+/**
+ *
+ */
+public class Edge {
+    private final RelNode parent;
+    private final RelNode child;
+    private final int childIdx;
+
+    public Edge(RelNode parent, RelNode child, int childIdx) {
+        this.parent = parent;
+        this.child = child;
+        this.childIdx = childIdx;
+    }
+
+    public RelNode parent() {
+        return parent;
+    }
+
+    public RelNode child() {
+        return child;
+    }
+
+    public int childIdx() {
+        return childIdx;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Fragment.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Fragment.java
new file mode 100644
index 0000000..c812f66
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Fragment.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import com.google.common.collect.ImmutableList;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentInfo;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMdFragmentInfo;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+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.trait.IgniteDistribution;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class Fragment implements RelSource {
+    private static final AtomicLong ID_GEN = new AtomicLong();
+
+    private final long exchangeId = ID_GEN.getAndIncrement();
+
+    private final RelNode root;
+
+    private NodesMapping mapping;
+
+    public Fragment(RelNode root) {
+        this.root = root;
+    }
+
+    public void init(PlannerContext ctx, RelMetadataQuery mq) {
+        FragmentInfo info = IgniteMdFragmentInfo.fragmentInfo(root, mq);
+
+        if (info.mapping() == null)
+            mapping = remote() ? ctx.mapForRandom(ctx.topologyVersion()) : ctx.mapForLocal();
+        else
+            mapping = info.mapping().deduplicate();
+
+        ImmutableList<Pair<IgniteReceiver, RelSource>> sources = info.sources();
+
+        if (!F.isEmpty(sources)) {
+            for (Pair<IgniteReceiver, RelSource> input : sources) {
+                IgniteReceiver receiver = input.left;
+                RelSource source = input.right;
+
+                source.init(mapping, receiver.distribution(), ctx, mq);
+            }
+        }
+    }
+
+    @Override public long exchangeId() {
+        return exchangeId;
+    }
+
+    @Override public void init(NodesMapping mapping, IgniteDistribution distribution, PlannerContext ctx, RelMetadataQuery mq) {
+        assert remote();
+
+        ((IgniteSender) root).target(new RelTargetImpl(exchangeId, mapping, distribution));
+
+        init(ctx, mq);
+    }
+
+    public RelNode root() {
+        return root;
+    }
+
+    @Override public NodesMapping mapping() {
+        return mapping;
+    }
+
+    private boolean remote() {
+        return root instanceof IgniteSender;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/QueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/QueryPlan.java
new file mode 100644
index 0000000..4a8ac5d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/QueryPlan.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.util.List;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.metadata.OptimisticPlanningException;
+import org.apache.ignite.internal.processors.query.calcite.metadata.RelMetadataQueryEx;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+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.util.typedef.F;
+
+/**
+ *
+ */
+public class QueryPlan {
+    private final List<Fragment> fragments;
+
+    public QueryPlan(List<Fragment> fragments) {
+        this.fragments = fragments;
+    }
+
+    public void init(PlannerContext ctx) {
+        int i = 0;
+
+        RelMetadataQueryEx mq = RelMetadataQueryEx.instance();
+
+        while (true) {
+            try {
+                F.first(fragments).init(ctx, mq);
+
+                break;
+            }
+            catch (OptimisticPlanningException e) {
+                if (++i > 3)
+                    throw new IgniteSQLException("Failed to map query.", e);
+
+                Edge edge = e.edge();
+
+                RelNode parent = edge.parent();
+                RelNode child = edge.child();
+
+                RelOptCluster cluster = child.getCluster();
+                RelTraitSet traitSet = child.getTraitSet();
+
+                Fragment fragment = new Fragment(new IgniteSender(cluster, traitSet, child));
+
+                fragments.add(fragment);
+
+                parent.replaceInput(edge.childIdx(), new IgniteReceiver(cluster, traitSet, child.getRowType(), fragment));
+            }
+        }
+    }
+
+    public List<Fragment> fragments() {
+        return fragments;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSource.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSource.java
new file mode 100644
index 0000000..b4eb49e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSource.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public interface RelSource {
+    /**
+     * @return Exchange id, has to be unique in scope of query.
+     */
+    long exchangeId();
+
+    /**
+     * @return Source mapping.
+     */
+    NodesMapping mapping();
+
+    /**
+     * @param mapping Target mapping.
+     * @param distribution Target distribution.
+     * @param ctx Context.
+     * @param mq Metadata query instance.
+     */
+    default void init(NodesMapping mapping, IgniteDistribution distribution, PlannerContext ctx, RelMetadataQuery mq) {
+        // No-op
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSourceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSourceImpl.java
new file mode 100644
index 0000000..b0ca757
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelSourceImpl.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+
+/**
+ *
+ */
+public class RelSourceImpl implements RelSource, Serializable {
+    private final long exchangeId;
+    private final NodesMapping mapping;
+
+    public RelSourceImpl(long exchangeId, NodesMapping mapping) {
+        this.exchangeId = exchangeId;
+        this.mapping = mapping;
+    }
+
+    @Override public long exchangeId() {
+        return exchangeId;
+    }
+
+    @Override public NodesMapping mapping() {
+        return mapping;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
new file mode 100644
index 0000000..b031b8e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public interface RelTarget {
+    long exchangeId();
+    NodesMapping mapping();
+    IgniteDistribution distribution();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTargetImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTargetImpl.java
new file mode 100644
index 0000000..a3c9b29
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTargetImpl.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public class RelTargetImpl implements RelTarget, Serializable {
+    private final long exchangeId;
+    private final NodesMapping mapping;
+    private final IgniteDistribution distribution;
+
+    public RelTargetImpl(long exchangeId, NodesMapping mapping, IgniteDistribution distribution) {
+        this.exchangeId = exchangeId;
+        this.mapping = mapping;
+        this.distribution = distribution;
+    }
+
+    @Override public long exchangeId() {
+        return exchangeId;
+    }
+
+    @Override public NodesMapping mapping() {
+        return mapping;
+    }
+
+    @Override public IgniteDistribution distribution() {
+        return distribution;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Splitter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Splitter.java
new file mode 100644
index 0000000..8dd3678
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/Splitter.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.splitter;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+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.IgniteRel;
+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.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.igniteRel;
+
+/**
+ *
+ */
+public class Splitter implements IgniteRelVisitor<IgniteRel>, RelOp<IgniteRel, QueryPlan> {
+    private List<Fragment> fragments;
+
+    @Override public QueryPlan go(IgniteRel root) {
+        fragments = new ArrayList<>();
+
+        fragments.add(new Fragment(visit(root)));
+
+        Collections.reverse(fragments);
+
+        return new QueryPlan(fragments);
+    }
+
+    @Override public IgniteRel visit(IgniteExchange rel) {
+        RelOptCluster cluster = rel.getCluster();
+        RelTraitSet outTraits = rel.getTraitSet();
+
+        IgniteRel input = visit(igniteRel(rel.getInput()));
+        RelTraitSet inTraits = input.getTraitSet();
+
+        Fragment fragment = new Fragment(new IgniteSender(cluster, inTraits, input));
+
+        fragments.add(fragment);
+
+        return new IgniteReceiver(cluster, outTraits, input.getRowType(), fragment);
+    }
+
+    @Override public IgniteRel visit(IgniteFilter rel) {
+        return visitChild(rel);
+    }
+
+    @Override public IgniteRel visit(IgniteProject rel) {
+        return visitChild(rel);
+    }
+
+    @Override public IgniteRel visit(IgniteJoin rel) {
+        return visitChildren(rel);
+    }
+
+    @Override public IgniteRel visit(IgniteTableScan rel) {
+        return rel;
+    }
+
+    @Override public IgniteRel visit(IgniteRel rel) {
+        return rel.accept(this);
+    }
+
+    @Override public IgniteRel visit(IgniteReceiver rel) {
+        throw new AssertionError("An attempt to split an already split task.");
+    }
+
+    @Override public IgniteRel visit(IgniteSender rel) {
+        throw new AssertionError("An attempt to split an already split task.");
+    }
+
+    private IgniteRel visitChildren(IgniteRel rel) {
+        for (Ord<RelNode> input : Ord.zip(rel.getInputs()))
+            visitChild(rel, input.i, igniteRel(input.e));
+
+        return rel;
+    }
+
+    /**
+     * Visits a single child of a parent.
+     */
+    private <T extends SingleRel & IgniteRel> IgniteRel visitChild(T rel) {
+        visitChild(rel, 0, igniteRel(rel.getInput()));
+
+        return rel;
+    }
+
+    /**
+     * Visits a particular child of a parent.
+     */
+    private void visitChild(IgniteRel parent, int i, IgniteRel child) {
+        IgniteRel child2 = visit(child);
+        if (child2 != child)
+            parent.replaceInput(i, child2);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AbstractDestinationFunctionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AbstractDestinationFunctionFactory.java
new file mode 100644
index 0000000..e6b6bb5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AbstractDestinationFunctionFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.Objects;
+
+/**
+ *
+ */
+public abstract class AbstractDestinationFunctionFactory implements DestinationFunctionFactory {
+    @Override public int hashCode() {
+        return Objects.hashCode(key());
+    }
+
+    @Override public boolean equals(Object obj) {
+        if (obj instanceof DestinationFunctionFactory)
+            return Objects.equals(key(), ((DestinationFunctionFactory) obj).key());
+
+        return false;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityFactory.java
new file mode 100644
index 0000000..f1dbbac
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.UUID;
+import java.util.function.ToIntFunction;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+public final class AffinityFactory extends AbstractDestinationFunctionFactory {
+    private final int cacheId;
+    private final Object key;
+
+    public AffinityFactory(int cacheId, Object key) {
+        this.cacheId = cacheId;
+        this.key = key;
+    }
+
+    @Override public DestinationFunction create(PlannerContext ctx, NodesMapping mapping, ImmutableIntList keys) {
+        assert keys.size() == 1 && mapping != null && !F.isEmpty(mapping.assignments());
+
+        List<List<UUID>> assignments = mapping.assignments();
+
+        if (U.assertionsEnabled()) {
+            for (List<UUID> assignment : assignments) {
+                assert F.isEmpty(assignment) || assignment.size() == 1;
+            }
+        }
+
+        ToIntFunction<Object> rowToPart = ctx.kernalContext()
+            .cache().context().cacheContext(cacheId).affinity()::partition;
+
+        return row -> assignments.get(rowToPart.applyAsInt(((Object[]) row)[keys.getInt(0)]));
+    }
+
+    @Override public Object key() {
+        return key;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllTargetsFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllTargetsFactory.java
new file mode 100644
index 0000000..30ce8b8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllTargetsFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.io.ObjectStreamException;
+import java.util.List;
+import java.util.UUID;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public final class AllTargetsFactory extends AbstractDestinationFunctionFactory {
+    public static final DestinationFunctionFactory INSTANCE = new AllTargetsFactory();
+
+    @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+        List<UUID> nodes = m.nodes();
+
+        return r -> nodes;
+    }
+
+    @Override public Object key() {
+        return "AllTargetsFactory";
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+        return INSTANCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
new file mode 100644
index 0000000..3d4dc41
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.UUID;
+
+/**
+ *
+ */
+public interface DestinationFunction {
+    List<UUID> destination(Object row);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunctionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunctionFactory.java
new file mode 100644
index 0000000..d12ec15
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunctionFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.io.Serializable;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public interface DestinationFunctionFactory extends Serializable {
+    DestinationFunction create(PlannerContext ctx, NodesMapping mapping, ImmutableIntList keys);
+
+    Object key();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
new file mode 100644
index 0000000..5972a11
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
@@ -0,0 +1,174 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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 com.google.common.collect.Ordering;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Objects;
+import org.apache.calcite.plan.RelMultipleTrait;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.mapping.Mappings;
+
+import static org.apache.calcite.rel.RelDistribution.Type.ANY;
+import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.RANDOM_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.RANGE_DISTRIBUTED;
+import static org.apache.calcite.rel.RelDistribution.Type.ROUND_ROBIN_DISTRIBUTED;
+
+/**
+ *
+ */
+public final class DistributionTrait implements IgniteDistribution, Serializable {
+    private static final Ordering<Iterable<Integer>> ORDERING =
+        Ordering.<Integer>natural().lexicographical();
+
+    private RelDistribution.Type type;
+    private ImmutableIntList keys;
+    private DestinationFunctionFactory functionFactory;
+
+    public DistributionTrait() {
+    }
+
+    public DistributionTrait(RelDistribution.Type type, ImmutableIntList keys, DestinationFunctionFactory functionFactory) {
+        if (type == RANGE_DISTRIBUTED || type == ROUND_ROBIN_DISTRIBUTED)
+            throw new IllegalArgumentException("Distribution type " + type + " is unsupported.");
+
+        this.type = type;
+        this.keys = keys;
+        this.functionFactory = functionFactory;
+    }
+
+    @Override public RelDistribution.Type getType() {
+        return type;
+    }
+
+    @Override public DestinationFunctionFactory destinationFunctionFactory() {
+        return functionFactory;
+    }
+
+    @Override public ImmutableIntList getKeys() {
+        return keys;
+    }
+
+    @Override public void register(RelOptPlanner planner) {}
+
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o instanceof DistributionTrait) {
+            DistributionTrait that = (DistributionTrait) o;
+
+            return type == that.type
+                && Objects.equals(keys, that.keys)
+                && Objects.equals(functionFactory.key(), that.functionFactory.key());
+        }
+
+        return false;
+    }
+
+    @Override public int hashCode() {
+        return Objects.hash(type, keys);
+    }
+
+    @Override public String toString() {
+        return type + (type == Type.HASH_DISTRIBUTED ? "[" + functionFactory.key() + "]" + keys : "");
+    }
+
+    @Override public DistributionTraitDef getTraitDef() {
+        return DistributionTraitDef.INSTANCE;
+    }
+
+    @Override public boolean satisfies(RelTrait trait) {
+        if (trait == this)
+            return true;
+
+        if (!(trait instanceof DistributionTrait))
+            return false;
+
+        DistributionTrait other = (DistributionTrait) trait;
+
+        if (other.getType() == ANY)
+            return true;
+
+        if (getType() == other.getType())
+            return getType() != HASH_DISTRIBUTED
+                || (Objects.equals(keys, other.keys)
+                    && Objects.equals(functionFactory, other.functionFactory));
+
+        return other.getType() == RANDOM_DISTRIBUTED && getType() == HASH_DISTRIBUTED;
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException {
+        out.writeObject(type);
+        out.writeObject(keys.toIntArray());
+        out.writeObject(functionFactory);
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+        type = (Type) in.readObject();
+        keys = ImmutableIntList.of((int[])in.readObject());
+        functionFactory = (DestinationFunctionFactory) in.readObject();
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+        return DistributionTraitDef.INSTANCE.canonize(this);
+    }
+
+    @Override public IgniteDistribution apply(Mappings.TargetMapping mapping) {
+        if (keys.isEmpty())
+            return this;
+
+        assert type == HASH_DISTRIBUTED;
+
+        List<Integer> newKeys = IgniteDistributions.projectDistributionKeys(mapping, keys);
+
+        return newKeys.size() == keys.size() ? IgniteDistributions.hash(newKeys, functionFactory) :
+            IgniteDistributions.random();
+    }
+
+    @Override public boolean isTop() {
+        return type == Type.ANY;
+    }
+
+    @Override public int compareTo(RelMultipleTrait o) {
+        // TODO is this method really needed??
+
+        final IgniteDistribution distribution = (IgniteDistribution) o;
+
+        if (type == distribution.getType()
+            && (type == Type.HASH_DISTRIBUTED
+            || type == Type.RANGE_DISTRIBUTED)) {
+            int cmp = ORDERING.compare(getKeys(), distribution.getKeys());
+
+            if (cmp == 0)
+                cmp = Integer.compare(functionFactory.key().hashCode(), distribution.destinationFunctionFactory().key().hashCode());
+
+            return cmp;
+        }
+
+        return type.compareTo(distribution.getType());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTraitDef.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTraitDef.java
new file mode 100644
index 0000000..306b7aa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTraitDef.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.Convention;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Exchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+
+/**
+ *
+ */
+public class DistributionTraitDef extends RelTraitDef<IgniteDistribution> {
+    /** */
+    public static final DistributionTraitDef INSTANCE = new DistributionTraitDef();
+
+    @Override public Class<IgniteDistribution> getTraitClass() {
+        return IgniteDistribution.class;
+    }
+
+    @Override public String getSimpleName() {
+        return "distr";
+    }
+
+    @Override public RelNode convert(RelOptPlanner planner, RelNode rel, IgniteDistribution targetDist, boolean allowInfiniteCostConverters) {
+        if (rel.getConvention() == Convention.NONE)
+            return null;
+
+        RelDistribution srcDist = rel.getTraitSet().getTrait(INSTANCE);
+
+        if (srcDist == targetDist) // has to be interned
+            return rel;
+
+        switch(targetDist.getType()){
+            case HASH_DISTRIBUTED:
+            case BROADCAST_DISTRIBUTED:
+            case SINGLETON:
+                Exchange exchange = new IgniteExchange(rel.getCluster(), rel.getTraitSet().replace(targetDist), rel, targetDist);
+                RelNode newRel = planner.register(exchange, rel);
+                RelTraitSet newTraits = rel.getTraitSet().replace(targetDist);
+
+                if (!newRel.getTraitSet().equals(newTraits))
+                    newRel = planner.changeTraits(newRel, newTraits);
+
+                return newRel;
+            case ANY:
+                return rel;
+            default:
+                return null;
+        }
+    }
+
+    @Override public boolean canConvert(RelOptPlanner planner, IgniteDistribution fromTrait, IgniteDistribution toTrait) {
+        return true;
+    }
+
+    @Override public IgniteDistribution getDefault() {
+        return IgniteDistributions.any();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/HashFunctionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/HashFunctionFactory.java
new file mode 100644
index 0000000..82c7325
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/HashFunctionFactory.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.io.ObjectStreamException;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.ToIntFunction;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ *
+ */
+public final class HashFunctionFactory extends AbstractDestinationFunctionFactory {
+    public static final DestinationFunctionFactory INSTANCE = new HashFunctionFactory();
+
+    @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+        assert m != null && !F.isEmpty(m.assignments());
+
+        int[] fields = k.toIntArray();
+
+        ToIntFunction<Object> hashFun = r -> {
+            Object[] row = (Object[]) r;
+
+            if (row == null)
+                return 0;
+
+            int hash = 1;
+
+            for (int i : fields)
+                hash = 31 * hash + (row[i] == null ? 0 : row[i].hashCode());
+
+            return hash;
+        };
+
+        List<List<UUID>> assignments = m.assignments();
+
+        if (U.assertionsEnabled()) {
+            for (List<UUID> assignment : assignments) {
+                assert F.isEmpty(assignment) || assignment.size() == 1;
+            }
+        }
+
+        return r -> assignments.get(hashFun.applyAsInt(r) % assignments.size());
+    }
+
+    @Override public Object key() {
+        return "DefaultHashFunction";
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+        return INSTANCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
new file mode 100644
index 0000000..5f3b175
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.rel.RelDistribution;
+import org.apache.calcite.util.mapping.Mappings;
+
+/**
+ *
+ */
+public interface IgniteDistribution extends RelDistribution {
+    DestinationFunctionFactory destinationFunctionFactory();
+    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistributions.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistributions.java
new file mode 100644
index 0000000..3e3aa55
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistributions.java
@@ -0,0 +1,272 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.mapping.Mappings;
+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.rel.RelDistribution.Type.ANY;
+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.RANDOM_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;
+
+/**
+ *
+ */
+public class IgniteDistributions {
+    private static final int BEST_CNT = 3;
+
+    private static final IgniteDistribution BROADCAST_DISTR = new DistributionTrait(BROADCAST_DISTRIBUTED, ImmutableIntList.of(), AllTargetsFactory.INSTANCE);
+    private static final IgniteDistribution SINGLETON_DISTR = new DistributionTrait(SINGLETON, ImmutableIntList.of(), SingleTargetFactory.INSTANCE);
+    private static final IgniteDistribution RANDOM_DISTR = new DistributionTrait(RANDOM_DISTRIBUTED, ImmutableIntList.of(), RandomTargetFactory.INSTANCE);
+    private static final IgniteDistribution ANY_DISTR = new DistributionTrait(ANY, ImmutableIntList.of(), NoOpFactory.INSTANCE);
+
+    public static IgniteDistribution any() {
+        return ANY_DISTR;
+    }
+
+    public static IgniteDistribution random() {
+        return RANDOM_DISTR;
+    }
+
+    public static IgniteDistribution single() {
+        return SINGLETON_DISTR;
+    }
+
+    public static IgniteDistribution broadcast() {
+        return BROADCAST_DISTR;
+    }
+
+    public static IgniteDistribution hash(List<Integer> keys) {
+        return DistributionTraitDef.INSTANCE.canonize(
+            new DistributionTrait(HASH_DISTRIBUTED, ImmutableIntList.copyOf(keys), HashFunctionFactory.INSTANCE));
+    }
+
+    public static IgniteDistribution hash(List<Integer> keys, DestinationFunctionFactory factory) {
+        return DistributionTraitDef.INSTANCE.canonize(
+            new DistributionTrait(HASH_DISTRIBUTED, ImmutableIntList.copyOf(keys), factory));
+    }
+
+    public static List<BiSuggestion> suggestJoin(IgniteDistribution leftIn, IgniteDistribution rightIn,
+        JoinInfo joinInfo, JoinRelType joinType) {
+        return topN(suggestJoin0(leftIn, rightIn, joinInfo, joinType), BEST_CNT);
+    }
+
+    public static List<BiSuggestion> suggestJoin(List<IgniteDistribution> leftIn, List<IgniteDistribution> rightIn,
+        JoinInfo joinInfo, JoinRelType joinType) {
+        HashSet<BiSuggestion> suggestions = new HashSet<>();
+
+        int bestCnt = 0;
+
+        for (IgniteDistribution leftIn0 : leftIn) {
+            for (IgniteDistribution rightIn0 : rightIn) {
+                for (BiSuggestion suggest : suggestJoin0(leftIn0, rightIn0, joinInfo, joinType)) {
+                    if (suggestions.add(suggest) && suggest.needExchange == 0 && (++bestCnt) == BEST_CNT)
+                        topN(new ArrayList<>(suggestions), BEST_CNT);
+                }
+            }
+        }
+
+        return topN(new ArrayList<>(suggestions), BEST_CNT);
+    }
+
+    private static ArrayList<BiSuggestion> suggestJoin0(IgniteDistribution leftIn, IgniteDistribution rightIn,
+        JoinInfo joinInfo, JoinRelType joinType) {
+        /*
+         * Distributions table:
+         *
+         * ===============INNER JOIN==============
+         * hash + hash = hash
+         * broadcast + hash = hash
+         * hash + broadcast = hash
+         * broadcast + broadcast = broadcast
+         * single + single = single
+         *
+         * ===============LEFT JOIN===============
+         * hash + hash = hash
+         * hash + broadcast = hash
+         * broadcast + broadcast = broadcast
+         * single + single = single
+         *
+         * ===============RIGHT JOIN==============
+         * hash + hash = hash
+         * broadcast + hash = hash
+         * broadcast + broadcast = broadcast
+         * single + single = single
+         *
+         * ===========FULL JOIN/CROSS JOIN========
+         * broadcast + broadcast = broadcast
+         * single + single = single
+         *
+         *
+         * others require redistribution
+         */
+
+        ArrayList<BiSuggestion> res = new ArrayList<>();
+
+        IgniteDistribution out, left, right;
+
+        if (joinType == LEFT || joinType == RIGHT || (joinType == INNER && !F.isEmpty(joinInfo.keys()))) {
+            HashSet<DestinationFunctionFactory> factories = U.newHashSet(3);
+
+            if (leftIn.getKeys().equals(joinInfo.leftKeys))
+                factories.add(leftIn.destinationFunctionFactory());
+
+            if (rightIn.getKeys().equals(joinInfo.rightKeys))
+                factories.add(rightIn.destinationFunctionFactory());
+
+            factories.add(HashFunctionFactory.INSTANCE);
+
+            for (DestinationFunctionFactory factory : factories) {
+                out = hash(joinInfo.leftKeys, factory);
+
+                left = hash(joinInfo.leftKeys, factory); right = hash(joinInfo.rightKeys, factory);
+                add(res, out, leftIn, rightIn, left, right);
+
+                if (joinType == INNER || joinType == LEFT) {
+                    left = hash(joinInfo.leftKeys, factory); right = broadcast();
+                    add(res, out, leftIn, rightIn, left, right);
+                }
+
+                if (joinType == INNER || joinType == RIGHT) {
+                    left = broadcast(); right = hash(joinInfo.rightKeys, factory);
+                    add(res, out, leftIn, rightIn, left, right);
+                }
+            }
+        }
+
+        out = left = right = broadcast();
+        add(res, out, leftIn, rightIn, left, right);
+
+        out = left = right = single();
+        add(res, out, leftIn, rightIn, left, right);
+
+        return res;
+    }
+
+    private static int add(ArrayList<BiSuggestion> dst, IgniteDistribution out, IgniteDistribution left, IgniteDistribution right,
+        IgniteDistribution newLeft, IgniteDistribution newRight) {
+        int exch = 0;
+
+        if (!left.satisfies(newLeft))
+            exch++;
+
+        if (!right.satisfies(newRight))
+            exch++;
+
+        dst.add(new BiSuggestion(out, newLeft, newRight, exch));
+
+        return exch;
+    }
+
+    private static List<BiSuggestion> topN(ArrayList<BiSuggestion> src, int n) {
+        Collections.sort(src);
+
+        return src.size() <= n ? src : src.subList(0, n);
+    }
+
+    public static List<Integer> projectDistributionKeys(Mappings.TargetMapping mapping, ImmutableIntList keys) {
+        if (mapping.getTargetCount() < keys.size())
+            return Collections.emptyList();
+
+        List<Integer> resKeys = new ArrayList<>(mapping.getTargetCount());
+
+        parent:
+        for (int i = 0; i < keys.size(); i++) {
+            int key = keys.getInt(i);
+
+            for (int j = 0; j < mapping.getTargetCount(); j++) {
+                if (mapping.getSourceOpt(j) == key) {
+                    resKeys.add(j);
+
+                    continue parent;
+                }
+            }
+
+            return Collections.emptyList();
+        }
+
+        return resKeys;
+    }
+
+    public static class BiSuggestion implements Comparable<BiSuggestion> {
+        private final IgniteDistribution out;
+        private final IgniteDistribution left;
+        private final IgniteDistribution right;
+        private final int needExchange;
+
+        public BiSuggestion(IgniteDistribution out, IgniteDistribution left, IgniteDistribution right, int needExchange) {
+            this.out = out;
+            this.left = left;
+            this.right = right;
+            this.needExchange = needExchange;
+        }
+
+        public IgniteDistribution out() {
+            return out;
+        }
+
+        public IgniteDistribution left() {
+            return left;
+        }
+
+        public IgniteDistribution right() {
+            return right;
+        }
+
+        public int needExchange() {
+            return needExchange;
+        }
+
+        @Override public int compareTo(@NotNull IgniteDistributions.BiSuggestion o) {
+            return Integer.compare(needExchange, o.needExchange);
+        }
+
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            BiSuggestion that = (BiSuggestion) o;
+
+            if (needExchange != that.needExchange) return false;
+            if (out != that.out) return false;
+            if (left != that.left) return false;
+            return right == that.right;
+        }
+
+        @Override public int hashCode() {
+            int result = out.hashCode();
+            result = 31 * result + left.hashCode();
+            result = 31 * result + right.hashCode();
+            result = 31 * result + needExchange;
+            return result;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/NoOpFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/NoOpFactory.java
new file mode 100644
index 0000000..7086277
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/NoOpFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.io.ObjectStreamException;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public final class NoOpFactory extends AbstractDestinationFunctionFactory {
+    public static final DestinationFunctionFactory INSTANCE = new NoOpFactory();
+
+    @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+        return null;
+    }
+
+    @Override public Object key() {
+        return "NoOpFactory";
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+        return INSTANCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomTargetFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomTargetFactory.java
new file mode 100644
index 0000000..8a643c3
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomTargetFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.io.ObjectStreamException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+
+/**
+ *
+ */
+public final class RandomTargetFactory extends AbstractDestinationFunctionFactory {
+    public static final DestinationFunctionFactory INSTANCE = new RandomTargetFactory();
+
+    @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+        List<UUID> nodes = m.nodes();
+
+        return r -> Collections.singletonList(nodes.get(ThreadLocalRandom.current().nextInt(nodes.size())));
+    }
+
+    @Override public Object key() {
+        return "RandomTargetFactory";
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+        return INSTANCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/SingleTargetFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/SingleTargetFactory.java
new file mode 100644
index 0000000..8a8b27c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/SingleTargetFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.io.ObjectStreamException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public final class SingleTargetFactory extends AbstractDestinationFunctionFactory {
+    public static final DestinationFunctionFactory INSTANCE = new SingleTargetFactory();
+
+    @Override public DestinationFunction create(PlannerContext ctx, NodesMapping m, ImmutableIntList k) {
+        List<UUID> nodes = Collections.singletonList(Objects.requireNonNull(F.first(m.nodes())));
+
+        return r -> nodes;
+    }
+
+    @Override public Object key() {
+        return "SingleTargetFactory";
+    }
+
+    private Object readResolve() throws ObjectStreamException {
+        return INSTANCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
new file mode 100644
index 0000000..83a2a19
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.type;
+
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+
+/**
+ *
+ */
+public class IgniteTypeFactory extends JavaTypeFactoryImpl {
+    public IgniteTypeFactory() {
+        super(IgniteTypeSystem.DEFAULT);
+    }
+
+    public IgniteTypeFactory(RelDataTypeSystem typeSystem) {
+        super(typeSystem);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java
new file mode 100644
index 0000000..6dbfd02
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.type;
+
+import java.io.Serializable;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+/**
+ *
+ */
+public class IgniteTypeSystem extends RelDataTypeSystemImpl implements Serializable {
+    public static final RelDataTypeSystem DEFAULT = new IgniteTypeSystem();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/RowType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/RowType.java
new file mode 100644
index 0000000..2133c56
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/RowType.java
@@ -0,0 +1,140 @@
+/*
+ * Copyright 2019 GridGain Systems, Inc. and Contributors.
+ *
+ * Licensed under the GridGain Community Edition License (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     https://www.gridgain.com/products/software/community-edition/gridgain-community-edition-license
+ *
+ * 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.type;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.util.ImmutableIntList;
+
+/**
+ *
+ */
+public class RowType {
+    private final String[] fields;
+    private final Class[] types;
+    private final BitSet keyFields;
+    private final int affinityKey;
+
+    public RowType(String[] fields, Class[] types, BitSet keyFields, int affinityKey) {
+        this.fields = fields;
+        this.types = types;
+        this.keyFields = keyFields;
+        this.affinityKey = affinityKey;
+    }
+
+    public RelDataType asRelDataType(RelDataTypeFactory factory) {
+        RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(factory);
+
+        int len = fields.length;
+
+        for (int i = 0; i < len; i++)
+            builder.add(fields[i], factory.createJavaType(types[i]));
+
+        return builder.build();
+    }
+
+    public List<Integer> distributionKeys() {
+        return ImmutableIntList.of(affinityKey);
+    }
+
+    public boolean isKeyField(int idx) {
+        return keyFields.get(idx);
+    }
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    public static class Builder {
+        private int affinityKey;
+        private final LinkedHashSet<String> fields;
+        private final BitSet keyFields;
+        private final ArrayList<Class> types;
+
+        private Builder() {
+            fields = new LinkedHashSet<>();
+            types = new ArrayList<>();
+            keyFields = new BitSet();
+
+            fields.add("_key"); types.add(Object.class);
+            fields.add("_val"); types.add(Object.class);
+        }
+
+        public Builder key(Class type) {
+            if (types.get(0) != Object.class && types.get(0) != type)
+                throw new IllegalStateException("Key type is already set.");
+
+            types.set(0, type);
+
+            return this;
+        }
+
+        public Builder val(Class type) {
+            if (types.get(1) != Object.class && types.get(1) != type)
+                throw new IllegalStateException("Value type is already set.");
+
+            types.set(1, type);
+
+            return this;
+        }
+
+        public Builder field(String name, Class type) {
+            if (!fields.add(name))
+                throw new IllegalStateException("Field name must be unique.");
+
+            types.add(type);
+
+            return this;
+        }
+
+        public Builder keyField(String name, Class type) {
+            if (!fields.add(name))
+                throw new IllegalStateException("Field name must be unique.");
+
+            types.add(type);
+
+            keyFields.set(types.size() - 1);
+
+            return this;
+        }
+
+        public Builder keyField(String name, Class type, boolean affinityKey) {
+            if (affinityKey && this.affinityKey > 0)
+                throw new IllegalStateException("Affinity key field must be unique.");
+
+            if (!fields.add(name))
+                throw new IllegalStateException("Field name must be unique.");
+
+            types.add(type);
+
+            keyFields.set(types.size() - 1);
+
+            if (affinityKey)
+                this.affinityKey = types.size() - 1;
+
+            return this;
+        }
+
+        public RowType build() {
+            return new RowType(fields.toArray(new String[0]), types.toArray(new Class[0]), keyFields, affinityKey);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
new file mode 100644
index 0000000..1eb5c76
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
@@ -0,0 +1,145 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.type.RowType;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public final class Commons {
+    private Commons(){}
+
+    public static Context convert(QueryContext ctx) {
+        return ctx == null ? Contexts.empty() : Contexts.of(ctx.unwrap(Object[].class));
+    }
+
+    /** */
+    public static RowType rowType(GridQueryTypeDescriptor desc) {
+        RowType.Builder b = RowType.builder();
+
+        Map<String, Class<?>> fields = desc.fields();
+
+        b.key(desc.keyClass()).val(desc.valueClass());
+
+        for (Map.Entry<String, Class<?>> entry : fields.entrySet()) {
+            GridQueryProperty prop = desc.property(entry.getKey());
+
+            if (prop.key())
+                b.keyField(prop.name(), prop.type(), Objects.equals(desc.affinityKey(), prop.name()));
+            else
+                b.field(prop.name(), prop.type());
+        }
+
+        return b.build();
+    }
+
+    public static RelOptRuleOperand any(Class<? extends RelNode> first, Class<? extends RelNode> second) {
+        return RelOptRule.operand(first, RelOptRule.operand(second, RelOptRule.any()));
+    }
+
+    public static <T> List<T> intersect(List<T> left, List<T> right) {
+        if (F.isEmpty(left) || F.isEmpty(right))
+            return Collections.emptyList();
+        else if (left.size() > right.size())
+            return intersect0(right, left);
+        else
+            return intersect0(left, right);
+    }
+
+    public static <T> List<T> intersect0(List<T> left, List<T> right) {
+        List<T> res = new ArrayList<>(Math.min(left.size(), right.size()));
+        HashSet<T> set = new HashSet<>(left);
+
+        for (T t : right) {
+            if (set.contains(t))
+                res.add(t);
+        }
+
+        return res;
+    }
+
+    public static <T> List<T> concat(List<T> col, T... elements) {
+        ArrayList<T> res = new ArrayList<>(col.size() + elements.length);
+
+        res.addAll(col);
+        res.addAll(Arrays.asList(elements));
+
+        return res;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> List<T> cast(List<?> src) {
+        return (List)src;
+    }
+
+    public static <T,R> List<R> transform(@NotNull List<T> src, @NotNull Function<T,R> mapFun) {
+        List<R> list = new ArrayList<>(src.size());
+
+        for (T t : src)
+            list.add(mapFun.apply(t));
+
+        return list;
+    }
+
+    public static <T,R> Set<R> transform(@NotNull Set<T> src, @NotNull Function<T,R> mapFun) {
+        Set<R> set = new HashSet<>(src.size());
+
+        for (T t : src)
+            set.add(mapFun.apply(t));
+
+        return set;
+    }
+
+    public static PlannerContext plannerContext(RelNode rel) {
+        return plannerContext(rel.getCluster().getPlanner().getContext());
+    }
+
+    public static PlannerContext plannerContext(Context ctx) {
+        return Objects.requireNonNull(ctx.unwrap(PlannerContext.class));
+    }
+
+    public static IgniteRel igniteRel(RelNode rel) {
+        if (rel.getConvention() != IgniteConvention.INSTANCE)
+            throw new AssertionError("Unexpected node: " + rel);
+
+        return (IgniteRel) rel;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java
new file mode 100644
index 0000000..5104816
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java
@@ -0,0 +1,42 @@
+/*
+ * 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.util;
+
+import java.lang.reflect.Method;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.DerivedDistribution;
+import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.FragmentMetadata;
+
+/**
+ *
+ */
+public enum IgniteMethod {
+    DERIVED_DISTRIBUTIONS(DerivedDistribution.class, "deriveDistributions"),
+    FRAGMENT_INFO(FragmentMetadata.class, "getFragmentInfo");
+
+    private final Method method;
+
+    IgniteMethod(Class clazz, String methodName, Class... argumentTypes) {
+        method = Types.lookupMethod(clazz, methodName, argumentTypes);
+    }
+
+    /** */
+    public Method method() {
+        return method;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/ListFieldsQueryCursor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/ListFieldsQueryCursor.java
new file mode 100644
index 0000000..c46e762
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/ListFieldsQueryCursor.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ */
+public class ListFieldsQueryCursor<T> implements FieldsQueryCursor<List<?>> {
+    /** */
+    private final RelDataType rowType;
+
+    /** */
+    private final Enumerable<T> enumerable;
+
+    /** */
+    private final Function<T, List<?>> converter;
+
+    /** */
+    private Iterator<T> it;
+
+    /**
+     * @param rowType Row data type description.
+     * @param enumerable Rows source.
+     * @param converter Row converter.
+     */
+    public ListFieldsQueryCursor(RelDataType rowType, Enumerable<T> enumerable, Function<T, List<?>> converter) {
+        this.rowType = rowType;
+        this.enumerable = enumerable;
+        this.converter = converter;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getFieldName(int idx) {
+        return rowType.getFieldList().get(idx).getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getColumnsCount() {
+        return rowType.getFieldCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<List<?>> getAll() {
+        return StreamSupport.stream(enumerable.spliterator(), false)
+            .map(converter)
+            .collect(Collectors.toList());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        closeIterator();
+    }
+
+    /** {@inheritDoc} */
+    @NotNull @Override public Iterator<List<?>> iterator() {
+        closeIterator();
+
+        return F.iterator(it = enumerable.iterator(), converter::apply, true);
+    }
+
+    private void closeIterator() {
+        if (it instanceof AutoCloseable)
+            U.closeQuiet((AutoCloseable)it);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/TableScanIterator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/TableScanIterator.java
new file mode 100644
index 0000000..64f05dc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/TableScanIterator.java
@@ -0,0 +1,149 @@
+/*
+ * 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.util;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UNDEFINED_CACHE_ID;
+
+/**
+ *
+ */
+public class TableScanIterator<T> extends GridCloseableIteratorAdapter<T> {
+    private final int cacheId;
+    private final Iterator<GridDhtLocalPartition> parts;
+    private final Function<CacheDataRow, T> typeWrapper;
+    private final Predicate<CacheDataRow> typeFilter;
+
+    /**
+     *
+     */
+    private GridCursor<? extends CacheDataRow> cur;
+    /**
+     *
+     */
+    private GridDhtLocalPartition curPart;
+
+    /**
+     *
+     */
+    private T next;
+
+    public TableScanIterator(int cacheId, Iterator<GridDhtLocalPartition> parts, Function<CacheDataRow, T> typeWrapper,
+        Predicate<CacheDataRow> typeFilter) {
+        this.cacheId = cacheId;
+        this.parts = parts;
+        this.typeWrapper = typeWrapper;
+        this.typeFilter = typeFilter;
+    }
+
+    @Override
+    protected T onNext() {
+        if (next == null)
+            throw new NoSuchElementException();
+
+        T next = this.next;
+
+        this.next = null;
+
+        return next;
+    }
+
+    @Override
+    protected boolean onHasNext() throws IgniteCheckedException {
+        if (next != null)
+            return true;
+
+        while (true) {
+            if (cur == null) {
+                if (parts.hasNext()) {
+                    GridDhtLocalPartition part = parts.next();
+
+                    if (!reservePartition(part))
+                        throw new IgniteSQLException("Failed to reserve partition, please retry on stable topology.");
+
+                    IgniteCacheOffheapManager.CacheDataStore ds = part.dataStore();
+
+                    cur = cacheId == UNDEFINED_CACHE_ID ? ds.cursor() : ds.cursor(cacheId);
+                } else
+                    break;
+            }
+
+            if (cur.next()) {
+                CacheDataRow row = cur.get();
+
+                if (!typeFilter.test(row))
+                    continue;
+
+                next = typeWrapper.apply(row);
+
+                break;
+            } else {
+                cur = null;
+
+                releaseCurrentPartition();
+            }
+        }
+
+        return next != null;
+    }
+
+    /**
+     *
+     */
+    private void releaseCurrentPartition() {
+        GridDhtLocalPartition p = curPart;
+
+        assert p != null;
+
+        curPart = null;
+
+        p.release();
+    }
+
+    /**
+     *
+     */
+    private boolean reservePartition(GridDhtLocalPartition p) {
+        if (p != null && p.reserve()) {
+            curPart = p;
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override protected void onClose() {
+        if (curPart != null)
+            releaseCurrentPartition();
+    }
+}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
new file mode 100644
index 0000000..f0afa1a
--- /dev/null
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
@@ -0,0 +1,1203 @@
+/*
+ * 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;
+
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.exec.ConsumerNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.Implementor;
+import org.apache.ignite.internal.processors.query.calcite.exec.Node;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue;
+import org.apache.ignite.internal.processors.query.calcite.prepare.DataContextImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerPhase;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerType;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Query;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
+import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Splitter;
+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.type.RowType;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.igniteRel;
+
+/**
+ *
+ */
+//@WithSystemProperty(key = "calcite.debug", value = "true")
+public class CalciteQueryProcessorTest extends GridCommonAbstractTest {
+
+    private GridTestKernalContext kernalContext;
+    private CalciteQueryProcessor proc;
+    private SchemaPlus schema;
+    private List<UUID> nodes;
+
+    private TestIgniteTable city;
+    private TestIgniteTable country;
+    private TestIgniteTable project;
+    private TestIgniteTable developer;
+
+    @Before
+    public void setup() {
+        kernalContext = new GridTestKernalContext(log);
+        proc = new CalciteQueryProcessor();
+        proc.setLogger(log);
+        proc.start(kernalContext);
+
+        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
+
+        developer = new TestIgniteTable("Developer", "Developer",
+            RowType.builder()
+                .keyField("id", Integer.class, true)
+                .field("name", String.class)
+                .field("projectId", Integer.class)
+                .field("cityId", Integer.class)
+                .build(), Arrays.asList(
+            new Object[]{0, null, 0, "Igor", 0, 1},
+            new Object[]{1, null, 1, "Roman", 0, 0}
+        ));
+
+        project = new TestIgniteTable("Project", "Project",
+            RowType.builder()
+                .keyField("id", Integer.class, true)
+                .field("name", String.class)
+                .field("ver", Integer.class)
+                .build(), Arrays.asList(
+            new Object[]{0, null, 0, "Calcite", 1},
+            new Object[]{1, null, 1, "Ignite", 1}
+        ));
+
+        country = new TestIgniteTable("Country", "Country",
+            RowType.builder()
+                .keyField("id", Integer.class, true)
+                .field("name", String.class)
+                .field("countryCode", Integer.class)
+                .build(), Arrays.<Object[]>asList(
+            new Object[]{0, null, 0, "Russia", 7}
+        ));
+
+        city = new TestIgniteTable("City", "City",
+            RowType.builder()
+                .keyField("id", Integer.class, true)
+                .field("name", String.class)
+                .field("countryId", Integer.class)
+                .build(), Arrays.asList(
+            new Object[]{0, null, 0, "Moscow", 0},
+            new Object[]{1, null, 1, "Saint Petersburg", 0}
+        ));
+
+        publicSchema.addTable(developer);
+        publicSchema.addTable(project);
+        publicSchema.addTable(country);
+        publicSchema.addTable(city);
+
+        schema = Frameworks
+            .createRootSchema(false)
+            .add("PUBLIC", publicSchema);
+
+        nodes = new ArrayList<>(4);
+
+        for (int i = 0; i < 4; i++) {
+            nodes.add(UUID.randomUUID());
+        }
+    }
+
+    @Test
+    public void testLogicalPlan() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.projectId = p.id0 + 1" +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(query);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+        }
+
+        assertNotNull(relRoot.rel);
+    }
+
+    @Test
+    public void testLogicalPlanDefaultSchema() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM Project pp" +
+            ") p " +
+            "ON d.projectId = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(query);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+        }
+
+        assertNotNull(relRoot.rel);
+
+
+    }
+
+    @Test
+    public void testCorrelatedQuery() throws Exception {
+        String sql = "SELECT d.id, (SELECT p.name FROM Project p WHERE p.id = d.id) name, d.projectId " +
+            "FROM Developer d";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(query);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+        }
+
+        assertNotNull(relRoot.rel);
+    }
+
+    @Test
+    public void testHepPlaner() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.projectId = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(query);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot.rel);
+    }
+
+    @Test
+    public void testVolcanoPlanerDistributed() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.projectId = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(query);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot.rel);
+
+        RexToExpTranslator translator = new RexToExpTranslator();
+
+        Project proj = (Project) relRoot.rel.getInput(0);
+
+        List<Expression> expressions = translator.translate(proj.getProjects());
+
+        assertNotNull(expressions);
+    }
+
+    @Test
+    public void testPlanSerializationDeserialization() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.id = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        byte[] convertedBytes;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            RelNode rel = planner.convert(sqlNode);
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            assertNotNull(rel);
+
+            QueryPlan plan = planner.plan(rel);
+
+            assertNotNull(plan);
+
+            assertTrue(plan.fragments().size() == 2);
+
+            plan.init(ctx);
+
+            RelGraph graph = new RelToGraphConverter().go(igniteRel(plan.fragments().get(1).root()));
+
+            convertedBytes = new JdkMarshaller().marshal(graph);
+
+            assertNotNull(convertedBytes);
+        }
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)) {
+            assertNotNull(planner);
+
+            RelGraph graph = new JdkMarshaller().unmarshal(convertedBytes, getClass().getClassLoader());
+
+            assertNotNull(graph);
+
+            RelNode rel = planner.convert(graph);
+
+            assertNotNull(rel);
+        }
+    }
+
+    @Test
+    public void testSplitterCollocatedPartitionedPartitioned() throws Exception {
+        Object key = new Object();
+
+        developer.identityKey(key);
+        project.identityKey(key);
+
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.id = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot);
+
+        QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+        assertNotNull(plan);
+
+        plan.init(ctx);
+
+        assertNotNull(plan);
+
+        assertTrue(plan.fragments().size() == 2);
+    }
+
+    @Test
+    public void testPhysicalPlan() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.name0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.name as name0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.projectId = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{-10}, this::context);
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            RelRoot relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSetOf(IgniteConvention.INSTANCE);
+
+            RelNode phys = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            assertNotNull(phys);
+
+            Map<String, Object> params = ctx.query().params(F.asMap(ContextValue.QUERY_ID.valueName(), new GridCacheVersion()));
+
+            Implementor implementor = new Implementor(new DataContextImpl(params, ctx));
+
+            Node<Object[]> exec = implementor.go(igniteRel(phys));
+
+            assertNotNull(exec);
+
+            assertTrue(exec instanceof ConsumerNode);
+
+            ConsumerNode consumer = (ConsumerNode) exec;
+
+            assertTrue(consumer.hasNext());
+
+            ArrayList<Object[]> res = new ArrayList<>();
+
+            while (consumer.hasNext())
+                res.add(consumer.next());
+
+            assertFalse(res.isEmpty());
+
+            Assert.assertArrayEquals(new Object[]{0, "Igor", 0, "Calcite", 1}, res.get(0));
+            Assert.assertArrayEquals(new Object[]{1, "Roman", 0, "Calcite", 1}, res.get(1));
+        }
+    }
+
+    @Test
+    public void testSplitterCollocatedReplicatedReplicated() throws Exception {
+        String sql = "SELECT d.id, (d.id + 1) as id2, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.id = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        MappingService ms = new MappingService() {
+            @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+                return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+            }
+
+            @Override public NodesMapping local() {
+                return new NodesMapping(select(nodes, 0), null, (byte) 0);
+            }
+
+            @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+                if (cacheId == CU.cacheId("Developer"))
+                    return new NodesMapping(select(nodes, 0,1,2), null, NodesMapping.HAS_REPLICATED_CACHES);
+                if (cacheId == CU.cacheId("Project"))
+                    return new NodesMapping(select(nodes, 0,1,2), null, NodesMapping.HAS_REPLICATED_CACHES);
+
+                throw new AssertionError("Unexpected cache id:" + cacheId);
+            }
+        };
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot);
+
+        QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+        assertNotNull(plan);
+
+        plan.init(ctx);
+
+        assertNotNull(plan);
+
+        assertTrue(plan.fragments().size() == 2);
+    }
+
+    @Test
+    public void testSplitterCollocatedReplicatedAndPartitioned() throws Exception {
+        developer.identityKey(new Object());
+
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.id = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        MappingService ms = new MappingService() {
+            @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+                return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+            }
+
+            @Override public NodesMapping local() {
+                return new NodesMapping(select(nodes, 0), null, (byte) 0);
+            }
+
+            @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+                if (cacheId == CU.cacheId("Developer"))
+                    return new NodesMapping(null, Arrays.asList(
+                        select(nodes, 0,1),
+                        select(nodes, 1,2),
+                        select(nodes, 2,0),
+                        select(nodes, 0,1),
+                        select(nodes, 1,2)
+                    ), NodesMapping.HAS_PARTITIONED_CACHES);
+                if (cacheId == CU.cacheId("Project"))
+                    return new NodesMapping(select(nodes, 0,1), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+                throw new AssertionError("Unexpected cache id:" + cacheId);
+            }
+        };
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot);
+
+        QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+        assertNotNull(plan);
+
+        plan.init(ctx);
+
+        assertNotNull(plan);
+
+        assertTrue(plan.fragments().size() == 2);
+    }
+
+    @Test
+    public void testSplitterPartiallyCollocated() throws Exception {
+        developer.identityKey(new Object());
+
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.projectId = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        MappingService ms = new MappingService() {
+            @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+                return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+            }
+
+            @Override public NodesMapping local() {
+                return new NodesMapping(select(nodes, 0), null, (byte) 0);
+            }
+
+            @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+                if (cacheId == CU.cacheId("Developer"))
+                    return new NodesMapping(null, Arrays.asList(
+                        select(nodes, 1),
+                        select(nodes, 2),
+                        select(nodes, 2),
+                        select(nodes, 0),
+                        select(nodes, 1)
+                    ), NodesMapping.HAS_PARTITIONED_CACHES);
+                if (cacheId == CU.cacheId("Project"))
+                    return new NodesMapping(select(nodes, 0,1), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+                throw new AssertionError("Unexpected cache id:" + cacheId);
+            }
+        };
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot);
+
+        QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+        assertNotNull(plan);
+
+        plan.init(ctx);
+
+        assertNotNull(plan);
+
+        assertTrue(plan.fragments().size() == 3);
+    }
+
+    @Test
+    public void testSplitterNonCollocated() throws Exception {
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.projectId = p.ver0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        MappingService ms = new MappingService() {
+            @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+                return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+            }
+
+            @Override public NodesMapping local() {
+                return new NodesMapping(select(nodes, 0), null, (byte) 0);
+            }
+
+            @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+                if (cacheId == CU.cacheId("Developer"))
+                    return new NodesMapping(select(nodes, 2), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+                else if (cacheId == CU.cacheId("Project"))
+                    return new NodesMapping(select(nodes, 0,1), null, (byte)(NodesMapping.HAS_REPLICATED_CACHES | NodesMapping.PARTIALLY_REPLICATED));
+
+                throw new AssertionError("Unexpected cache id:" + cacheId);
+            }
+        };
+
+        PlannerContext ctx = proc.context(Contexts.empty(), sql, new Object[]{2}, (c, q) -> context(c, q, ms));
+
+        assertNotNull(ctx);
+
+        RelTraitDef[] traitDefs = {
+            DistributionTraitDef.INSTANCE,
+            ConventionTraitDef.INSTANCE
+        };
+
+        RelRoot relRoot;
+
+        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)){
+            assertNotNull(planner);
+
+            Query query = ctx.query();
+
+            assertNotNull(planner);
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query.sql());
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
+        }
+
+        assertNotNull(relRoot);
+
+        QueryPlan plan = new Splitter().go(igniteRel(relRoot.rel));
+
+        assertNotNull(plan);
+
+        plan.init(ctx);
+
+        assertNotNull(plan);
+
+        assertTrue(plan.fragments().size() == 3);
+    }
+
+    @Test
+    public void testSplitterPartiallyReplicated1() throws Exception {
+        developer.identityKey(new Object());
+
+        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+            "FROM PUBLIC.Developer d JOIN (" +
+            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            ") p " +
+            "ON d.id = p.id0 " +
+            "WHERE (d.projectId + 1) > ?";
+
+        MappingService ms = new MappingService() {
+            @Override public NodesMapping random(AffinityTopologyVersion topVer) {
+                return new NodesMapping(select(nodes, 0,1,2,3), null, (byte) 0);
+            }
+
+            @Override public NodesMapping local() {
+                return new NodesMapping(select(nodes, 0), null, (byte) 0);
+            }
+
+            @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
+                if (cacheId == CU.cacheId("Developer"))
... 1036 lines suppressed ...