You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tl...@apache.org on 2021/07/22 08:24:33 UTC

[ignite-3] branch main updated: IGNITE-14836 Integrate execution of SELECT queries into Ignite 3.0 (#213)

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

tledkov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 50ea1ed  IGNITE-14836 Integrate execution of SELECT queries into Ignite 3.0 (#213)
50ea1ed is described below

commit 50ea1ed9d420d2e525d9935bfb02f75ee0938f1f
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Thu Jul 22 11:24:27 2021 +0300

    IGNITE-14836 Integrate execution of SELECT queries into Ignite 3.0 (#213)
---
 modules/calcite/pom.xml                            |  11 +
 .../query/calcite/SqlQueryProcessor.java           | 160 ++++
 .../internal/processors/query/calcite/Stubs.java   |  54 ++
 .../calcite/exec/ClosableIteratorsHolder.java      | 168 ++++
 .../query/calcite/exec/ExchangeService.java        |  84 ++
 .../query/calcite/exec/ExchangeServiceImpl.java    | 262 +++++++
 .../query/calcite/exec/ExecutionContext.java       |   4 +-
 .../ExecutionService.java}                         |  33 +-
 .../query/calcite/exec/ExecutionServiceImpl.java   | 865 +++++++++++++++++++++
 .../query/calcite/exec/LogicalRelImplementor.java  | 654 ++++++++++++++++
 .../query/calcite/exec/MailboxRegistry.java        |  99 +++
 .../query/calcite/exec/MailboxRegistryImpl.java    | 169 ++++
 .../query/calcite/exec/QueryTaskExecutorImpl.java  |   4 +-
 .../processors/query/calcite/exec/rel/Inbox.java   | 519 +++++++++++++
 .../processors/query/calcite/exec/rel/Outbox.java  | 361 +++++++++
 .../query/calcite/exec/rel/RootNode.java           |  16 +-
 .../ErrorMessage.java}                             |  31 +-
 .../ExecutionContextAwareMessage.java}             |  32 +-
 .../InboxCloseMessage.java}                        |  31 +-
 .../{Stubs.java => message/MessageListener.java}   |  25 +-
 .../MessageService.java}                           |  42 +-
 .../query/calcite/message/MessageServiceImpl.java  | 149 ++++
 .../OutboxCloseMessage.java}                       |  31 +-
 .../QueryBatchAcknowledgeMessage.java}             |  30 +-
 .../QueryBatchMessage.java}                        |  34 +-
 .../QueryStartRequest.java}                        |  42 +-
 .../QueryStartResponse.java}                       |  31 +-
 .../SqlQueryMessageGroup.java}                     |  30 +-
 .../{Stubs.java => metadata/AffinityService.java}  |  27 +-
 .../query/calcite/metadata/ColocationGroup.java    |  50 +-
 .../calcite/metadata/FragmentDescription.java      |  12 +-
 .../query/calcite/metadata/FragmentMapping.java    |  12 +-
 .../query/calcite/metadata/IgniteMdRowCount.java   |   6 +-
 .../query/calcite/metadata/MappingService.java     |   3 +-
 .../query/calcite/metadata/MappingServiceImpl.java |  60 ++
 ...agmentDescription.java => RemoteException.java} |  71 +-
 .../calcite/prepare/AbstractMultiStepPlan.java     |  14 +-
 .../processors/query/calcite/prepare/CacheKey.java |  78 ++
 .../processors/query/calcite/prepare/Cloner.java   |  16 +-
 .../{MultiStepQueryPlan.java => DdlPlan.java}      |  28 +-
 ...MultiStepQueryPlan.java => DummyPlanCache.java} |  21 +-
 .../{MultiStepQueryPlan.java => ExplainPlan.java}  |  36 +-
 .../{Stubs.java => prepare/FieldsMetadata.java}    |  31 +-
 ...iStepQueryPlan.java => FieldsMetadataImpl.java} |  33 +-
 .../processors/query/calcite/prepare/Fragment.java |   5 +-
 .../{MultiStepQueryPlan.java => FragmentPlan.java} |  31 +-
 .../query/calcite/prepare/IgniteRelShuttle.java    |  22 +-
 ...ltiStepQueryPlan.java => MultiStepDmlPlan.java} |  10 +-
 .../query/calcite/prepare/MultiStepPlan.java       |   8 +-
 .../query/calcite/prepare/MultiStepQueryPlan.java  |   2 +-
 .../query/calcite/prepare/PlannerPhase.java        |   9 +-
 .../query/calcite/prepare/PlanningContext.java     |  21 +-
 ...MultiStepQueryPlan.java => QueryPlanCache.java} |  28 +-
 .../{Stubs.java => prepare/QueryPlanFactory.java}  |  25 +-
 .../query/calcite/rel/IgniteRelVisitor.java        |  16 +-
 .../query/calcite/rel/set/IgniteIntersect.java     |  68 ++
 .../query/calcite/rel/set/IgniteMapIntersect.java  |  74 ++
 .../query/calcite/rel/set/IgniteMapMinus.java      |  73 +-
 .../{IgniteMapMinus.java => IgniteMapSetOp.java}   |  87 +--
 .../query/calcite/rel/set/IgniteMinus.java         |  71 ++
 ...ReduceMinus.java => IgniteReduceIntersect.java} |  65 +-
 .../query/calcite/rel/set/IgniteReduceMinus.java   |  51 +-
 .../query/calcite/rel/set/IgniteReduceSetOp.java   |  75 ++
 .../query/calcite/rel/set/IgniteSetOp.java         |  78 ++
 .../calcite/rel/set/IgniteSingleIntersect.java     |  68 ++
 .../query/calcite/rel/set/IgniteSingleMinus.java   |  72 +-
 ...niteSingleMinus.java => IgniteSingleSetOp.java} |  63 +-
 .../query/calcite/rule/SetOpConverterRule.java     | 186 +++++
 .../query/calcite/schema/ColumnDescriptor.java     |   3 -
 .../query/calcite/schema/ColumnDescriptorImpl.java |  93 +++
 .../query/calcite/schema/IgniteTable.java          |  21 +
 .../query/calcite/schema/IgniteTableImpl.java      | 207 +++++
 .../{Stubs.java => schema/SchemaHolder.java}       |  24 +-
 .../query/calcite/schema/SchemaHolderImpl.java     | 124 +++
 .../query/calcite/schema/TableDescriptor.java      |  10 +
 .../query/calcite/schema/TableDescriptorImpl.java  | 149 ++++
 ...gniteDistribution.java => AffinityAdapter.java} |  41 +-
 .../{IgniteDistribution.java => AllNodes.java}     |  29 +-
 .../calcite/{Stubs.java => trait/Destination.java} |  30 +-
 .../query/calcite/trait/DistributionFunction.java  |  87 +++
 .../query/calcite/trait/DistributionTrait.java     |  11 +-
 .../query/calcite/trait/IgniteDistribution.java    |  14 +
 .../{IgniteDistribution.java => Partitioned.java}  |  38 +-
 .../{IgniteDistribution.java => RandomNode.java}   |  36 +-
 .../processors/query/calcite/util/Commons.java     | 101 ++-
 .../NodeLeaveHandler.java}                         |  28 +-
 .../calcite/util/StripedThreadPoolExecutor.java    |   2 -
 .../calcite/exec/rel/AbstractExecutionTest.java    |  23 +-
 .../query/calcite/planner/AbstractPlannerTest.java |  28 +-
 .../query/calcite/planner/ExceptPlannerTest.java   | 329 --------
 .../query/calcite/planner/PlannerTest.java         |   6 +-
 .../query/calcite/planner/SetOpPlannerTest.java    | 421 ++++++++++
 ...tionTest.java => TableFunctionPlannerTest.java} |   2 +-
 .../apache/ignite/internal/util/Cancellable.java}  |  27 +-
 modules/runner/pom.xml                             |   5 +
 .../org/apache/ignite/internal/app/IgniteImpl.java |  11 +-
 .../apache/ignite/internal/app/IgnitionImpl.java   |  12 +-
 modules/schema/README.md                           |   2 +-
 .../internal/table/event/TableEventParameters.java |   4 +-
 parent/pom.xml                                     |   6 +
 100 files changed, 6300 insertions(+), 1301 deletions(-)

diff --git a/modules/calcite/pom.xml b/modules/calcite/pom.xml
index 833930a..9362b1c 100644
--- a/modules/calcite/pom.xml
+++ b/modules/calcite/pom.xml
@@ -51,6 +51,17 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-table</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-network-annotation-processor</artifactId>
+            <optional>true</optional>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.calcite</groupId>
             <artifactId>calcite-core</artifactId>
         </dependency>
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlQueryProcessor.java
new file mode 100644
index 0000000..f3a8955
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/SqlQueryProcessor.java
@@ -0,0 +1,160 @@
+/*
+ * 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.List;
+
+import org.apache.ignite.internal.manager.EventListener;
+import org.apache.ignite.internal.processors.query.calcite.exec.ArrayRowHandler;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutorImpl;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.DummyPlanCache;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolderImpl;
+import org.apache.ignite.internal.processors.query.calcite.util.StripedThreadPoolExecutor;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.table.event.TableEvent;
+import org.apache.ignite.internal.table.event.TableEventParameters;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.network.ClusterService;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class SqlQueryProcessor {
+    /** Default Ignite thread keep alive time. */
+    public static final long DFLT_THREAD_KEEP_ALIVE_TIME = 60_000L;
+
+    private final ExecutionService executionSrvc;
+
+    private final MessageService msgSrvc;
+
+    private final QueryTaskExecutor taskExecutor;
+
+    public SqlQueryProcessor(
+        ClusterService clusterSrvc,
+        TableManager tableManager
+    ) {
+        taskExecutor = new QueryTaskExecutorImpl(
+            new StripedThreadPoolExecutor(
+                4,
+                "calciteQry",
+                null,
+                true,
+                DFLT_THREAD_KEEP_ALIVE_TIME
+            )
+        );
+
+        msgSrvc = new MessageServiceImpl(
+            clusterSrvc.topologyService(),
+            clusterSrvc.messagingService(),
+            taskExecutor
+        );
+
+        SchemaHolderImpl schemaHolder = new SchemaHolderImpl(clusterSrvc.topologyService());
+
+        executionSrvc = new ExecutionServiceImpl<>(
+            clusterSrvc.topologyService(),
+            msgSrvc,
+            new DummyPlanCache(),
+            schemaHolder,
+            taskExecutor,
+            ArrayRowHandler.INSTANCE
+        );
+
+        tableManager.listen(TableEvent.CREATE, new TableCreatedListener(schemaHolder));
+        tableManager.listen(TableEvent.ALTER, new TableUpdatedListener(schemaHolder));
+        tableManager.listen(TableEvent.DROP, new TableDroppedListener(schemaHolder));
+    }
+
+    public List<Cursor<List<?>>> query(String schemaName, String qry, Object... params) {
+        return executionSrvc.executeQuery(schemaName, qry, params);
+    }
+
+    private abstract static class AbstractTableEventListener implements EventListener<TableEventParameters> {
+        protected final SchemaHolderImpl schemaHolder;
+
+        private AbstractTableEventListener(
+            SchemaHolderImpl schemaHolder
+        ) {
+            this.schemaHolder = schemaHolder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove(@NotNull Throwable exception) {
+            throw new IllegalStateException();
+        }
+    }
+
+    private static class TableCreatedListener extends AbstractTableEventListener {
+        private TableCreatedListener(
+            SchemaHolderImpl schemaHolder
+        ) {
+            super(schemaHolder);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean notify(@NotNull TableEventParameters parameters, @Nullable Throwable exception) {
+            schemaHolder.onSqlTypeCreated(
+                "PUBLIC",
+                parameters.tableName(),
+                parameters.table().schemaView().schema()
+            );
+
+            return false;
+        }
+    }
+
+    private static class TableUpdatedListener extends AbstractTableEventListener {
+        private TableUpdatedListener(
+            SchemaHolderImpl schemaHolder
+        ) {
+            super(schemaHolder);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean notify(@NotNull TableEventParameters parameters, @Nullable Throwable exception) {
+            schemaHolder.onSqlTypeUpdated(
+                "PUBLIC",
+                parameters.tableName(),
+                parameters.table().schemaView().schema()
+            );
+
+            return false;
+        }
+    }
+
+    private static class TableDroppedListener extends AbstractTableEventListener {
+        private TableDroppedListener(
+            SchemaHolderImpl schemaHolder
+        ) {
+            super(schemaHolder);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean notify(@NotNull TableEventParameters parameters, @Nullable Throwable exception) {
+            schemaHolder.onSqlTypeDropped(
+                "PUBLIC",
+                parameters.tableName()
+            );
+
+            return false;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
index 093146c..6ebc28c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
@@ -16,6 +16,17 @@
  */
 package org.apache.ignite.internal.processors.query.calcite;
 
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
 /** Stubs */
 public class Stubs {
     /** */
@@ -31,4 +42,47 @@ public class Stubs {
     public static String stringFoo(Object... args) {
         return args == null ? "null" : "not null";
     }
+
+    public static Collection<Object[]> resultSetGenerator(int cnt, IgniteTypeFactory factory, RelDataType type) {
+        List<Object[]> res = new ArrayList<>(cnt);
+
+        for (int row = 0; row < cnt; row++) {
+            Object[] tmp = new Object[type.getFieldCount()];
+
+            res.add(tmp);
+
+            for (RelDataTypeField field : type.getFieldList())
+                tmp[field.getIndex()] = rndValueOfType(factory.getJavaClass(field.getType()));
+        }
+
+        return res;
+    }
+
+    private static Object rndValueOfType(Type type) {
+        if (type == byte.class || type == Byte.class)
+            return (byte)ThreadLocalRandom.current().nextInt(100);
+
+        if (type == short.class || type == Short.class)
+            return (short)ThreadLocalRandom.current().nextInt(100);
+
+        if (type == int.class || type == Integer.class)
+            return ThreadLocalRandom.current().nextInt(100);
+
+        if (type == long.class || type == Long.class)
+            return (long)ThreadLocalRandom.current().nextInt(100);
+
+        if (type == float.class || type == Float.class)
+            return ThreadLocalRandom.current().nextFloat();
+
+        if (type == double.class || type == Double.class)
+            return ThreadLocalRandom.current().nextDouble();
+
+        if (type == UUID.class)
+            return UUID.randomUUID();
+
+        if (type == String.class)
+            return UUID.randomUUID().toString();
+
+        throw new IllegalStateException("Can't generate value of type " + type.getTypeName());
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolder.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolder.java
new file mode 100644
index 0000000..56bb0e4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolder.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.lang.ref.Reference;
+import java.lang.ref.ReferenceQueue;
+import java.lang.ref.WeakReference;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.lang.IgniteLogger;
+
+/**
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class ClosableIteratorsHolder {
+    /** */
+    private final ReferenceQueue refQueue;
+
+    /** */
+    private final Map<Reference, Object> refMap;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private volatile boolean stopped;
+
+    /** */
+    private Thread cleanWorker;
+
+    /** */
+    public ClosableIteratorsHolder(IgniteLogger log) {
+        this.log = log;
+
+        refQueue = new ReferenceQueue<>();
+        refMap = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * @param src Closeable iterator.
+     * @return Weak closable iterator wrapper.
+     */
+    public <T> Iterator<T> iterator(final Iterator<T> src) {
+        cleanUp(false);
+
+        return new DelegatingIterator<>(src);
+    }
+
+    /** */
+    public void init() {
+        cleanWorker = new Thread(null, () -> cleanUp(true), "calciteIteratorsCleanWorker");
+        cleanWorker.setDaemon(true);
+        cleanWorker.start();
+    }
+
+    /** */
+    public void tearDown() {
+        stopped = true;
+        refMap.clear();
+
+        Thread t = cleanWorker;
+
+        if (t != null)
+            t.interrupt();
+    }
+
+    /** */
+    private void cleanUp(boolean blocking) {
+        for (Reference<?> ref = nextRef(blocking); !stopped && ref != null; ref = nextRef(blocking))
+            Commons.close(refMap.remove(ref), log);
+    }
+
+    /** */
+    private Reference nextRef(boolean blocking) {
+        try {
+            return !blocking ? refQueue.poll() : refQueue.remove();
+        }
+        catch (InterruptedException ignored) {
+            return null;
+        }
+    }
+
+    /** */
+    private AutoCloseable closeable(Object referent, Object resource) {
+        if (!(resource instanceof AutoCloseable))
+            return null;
+
+        return new CloseableReference(referent, resource);
+    }
+
+    /** */
+    private final class DelegatingIterator<T> implements Iterator<T>, AutoCloseable {
+        /** */
+        private final Iterator<T> delegate;
+
+        /** */
+        private final AutoCloseable closeable;
+
+        /** */
+        private DelegatingIterator(Iterator<T> delegate) {
+            closeable = closeable(this, this.delegate = delegate);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return delegate.hasNext();
+        }
+
+        /** {@inheritDoc} */
+        @Override public T next() {
+            return delegate.next();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove() {
+            delegate.remove();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void forEachRemaining(Consumer<? super T> action) {
+            delegate.forEachRemaining(action);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws Exception {
+            Commons.close(closeable);
+        }
+    }
+
+    /** */
+    private final class CloseableReference extends WeakReference implements AutoCloseable {
+        /** */
+        private CloseableReference(Object referent, Object resource) {
+            super(referent, refQueue);
+
+            refMap.put(this, resource);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws Exception {
+            try {
+                Commons.close(refMap.remove(this));
+            }
+            finally {
+                clear();
+            }
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
new file mode 100644
index 0000000..112c6c4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ *
+ */
+public interface ExchangeService {
+    /**
+     * Sends a batch of data to remote node.
+     * @param nodeId Target node ID.
+     * @param qryId Query ID.
+     * @param fragmentId Target fragment ID.
+     * @param exchangeId Exchange ID.
+     * @param batchId Batch ID.
+     * @param last Last batch flag.
+     * @param rows Data rows.
+     */
+    <Row> void sendBatch(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId, boolean last,
+        List<Row> rows) throws IgniteInternalCheckedException;
+
+    /**
+     * Acknowledges a batch with given ID is processed.
+     * @param nodeId Node ID to notify.
+     * @param qryId Query ID.
+     * @param fragmentId Target fragment ID.
+     * @param exchangeId Exchange ID.
+     * @param batchId Batch ID.
+     */
+    void acknowledge(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId) throws IgniteInternalCheckedException;
+
+    /**
+     * Sends cancel request.
+     * @param nodeId Target node ID.
+     * @param qryId Query ID.
+     * @param fragmentId Target fragment ID.
+     * @param exchangeId Exchange ID.
+     */
+    void closeInbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException;
+
+    /**
+     * Sends cancel request.
+     * @param nodeId Target node ID.
+     * @param qryId Query ID.
+     * @param fragmentId Target fragment ID.
+     * @param exchangeId Exchange ID.
+     */
+    void closeOutbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException;
+
+    /**
+     * @param nodeId Target node ID.
+     * @param qryId Query ID.
+     * @param fragmentId Source fragment ID.
+     * @param err Exception to send.
+     * @throws IgniteInternalCheckedException On error marshaling or send ErrorMessage.
+     */
+    void sendError(String nodeId, UUID qryId, long fragmentId, Throwable err) throws IgniteInternalCheckedException;
+
+    /**
+     * @param nodeId Node ID.
+     * @return {@code true} if node is alive, {@code false} otherwise.
+     */
+    boolean alive(String nodeId);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
new file mode 100644
index 0000000..c3bc745
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
+import org.apache.ignite.internal.processors.query.calcite.message.InboxCloseMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.OutboxCloseMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryBatchAcknowledgeMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryBatchMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.SqlQueryMessageGroup;
+import org.apache.ignite.internal.processors.query.calcite.message.SqlQueryMessagesFactory;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class ExchangeServiceImpl implements ExchangeService {
+    private static final IgniteLogger LOG = IgniteLogger.forClass(ExchangeServiceImpl.class);
+
+    private static final SqlQueryMessagesFactory FACTORY = new SqlQueryMessagesFactory();
+
+    /** */
+    private final QueryTaskExecutor taskExecutor;
+
+    /** */
+    private final MailboxRegistry mailboxRegistry;
+
+    /** */
+    private final MessageService msgSrvc;
+
+    public ExchangeServiceImpl(
+        QueryTaskExecutor taskExecutor,
+        MailboxRegistry mailboxRegistry,
+        MessageService msgSrvc
+    ) {
+        this.taskExecutor = taskExecutor;
+        this.mailboxRegistry = mailboxRegistry;
+        this.msgSrvc = msgSrvc;
+
+        init();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <Row> void sendBatch(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId,
+        boolean last, List<Row> rows) throws IgniteInternalCheckedException {
+        msgSrvc.send(
+            nodeId,
+            FACTORY.queryBatchMessage()
+                .queryId(qryId)
+                .fragmentId(fragmentId)
+                .exchangeId(exchangeId)
+                .batchId(batchId)
+                .last(last)
+                .rows(Commons.cast(rows))
+                .build()
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void acknowledge(String nodeId, UUID qryId, long fragmentId, long exchangeId, int batchId)
+        throws IgniteInternalCheckedException {
+        msgSrvc.send(
+            nodeId,
+            FACTORY.queryBatchAcknowledgeMessage()
+                .queryId(qryId)
+                .fragmentId(fragmentId)
+                .exchangeId(exchangeId)
+                .batchId(batchId)
+                .build()
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeOutbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException {
+        msgSrvc.send(
+            nodeId,
+            FACTORY.outboxCloseMessage()
+                .queryId(qryId)
+                .fragmentId(fragmentId)
+                .exchangeId(exchangeId)
+                .build()
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeInbox(String nodeId, UUID qryId, long fragmentId, long exchangeId) throws IgniteInternalCheckedException {
+        msgSrvc.send(
+            nodeId,
+            FACTORY.inboxCloseMessage()
+                .queryId(qryId)
+                .fragmentId(fragmentId)
+                .exchangeId(exchangeId)
+                .build()
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendError(String nodeId, UUID qryId, long fragmentId, Throwable err) throws IgniteInternalCheckedException {
+        msgSrvc.send(
+            nodeId,
+            FACTORY.errorMessage()
+                .queryId(qryId)
+                .fragmentId(fragmentId)
+                .error(err)
+                .build()
+        );
+    }
+
+    private void init() {
+        msgSrvc.register((n, m) -> onMessage(n, (InboxCloseMessage) m), SqlQueryMessageGroup.INBOX_CLOSE_MESSAGE);
+        msgSrvc.register((n, m) -> onMessage(n, (OutboxCloseMessage) m), SqlQueryMessageGroup.OUTBOX_CLOSE_MESSAGE);
+        msgSrvc.register((n, m) -> onMessage(n, (QueryBatchAcknowledgeMessage) m), SqlQueryMessageGroup.QUERY_BATCH_ACK);
+        msgSrvc.register((n, m) -> onMessage(n, (QueryBatchMessage) m), SqlQueryMessageGroup.QUERY_BATCH_MESSAGE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean alive(String nodeId) {
+        return msgSrvc.alive(nodeId);
+    }
+
+    /** */
+    protected void onMessage(String nodeId, InboxCloseMessage msg) {
+        Collection<Inbox<?>> inboxes = mailboxRegistry.inboxes(msg.queryId(), msg.fragmentId(), msg.exchangeId());
+
+        if (!nullOrEmpty(inboxes)) {
+            for (Inbox<?> inbox : inboxes)
+                inbox.context().execute(inbox::close, inbox::onError);
+        }
+        else if (LOG.isDebugEnabled()) {
+            LOG.debug("Stale inbox cancel message received: [" +
+                "nodeId=" + nodeId +
+                ", queryId=" + msg.queryId() +
+                ", fragmentId=" + msg.fragmentId() +
+                ", exchangeId=" + msg.exchangeId() + "]");
+        }
+    }
+
+    /** */
+    protected void onMessage(String nodeId, OutboxCloseMessage msg) {
+        Collection<Outbox<?>> outboxes = mailboxRegistry.outboxes(msg.queryId(), msg.fragmentId(), msg.exchangeId());
+
+        if (!nullOrEmpty(outboxes)) {
+            for (Outbox<?> outbox : outboxes)
+                outbox.context().execute(outbox::close, outbox::onError);
+
+            for (Outbox<?> outbox : outboxes)
+                outbox.context().execute(outbox.context()::cancel, outbox::onError);
+        }
+        else if (LOG.isDebugEnabled()) {
+            LOG.debug("Stale oubox cancel message received: [" +
+                "nodeId=" + nodeId +
+                ", queryId=" + msg.queryId() +
+                ", fragmentId=" + msg.fragmentId() +
+                ", exchangeId=" + msg.exchangeId() + "]");
+        }
+    }
+
+    /** */
+    protected void onMessage(String nodeId, QueryBatchAcknowledgeMessage msg) {
+        Outbox<?> outbox = mailboxRegistry.outbox(msg.queryId(), msg.exchangeId());
+
+        if (outbox != null) {
+            try {
+                outbox.onAcknowledge(nodeId, msg.batchId());
+            }
+            catch (Throwable e) {
+                outbox.onError(e);
+
+                throw new IgniteInternalException("Unexpected exception", e);
+            }
+        }
+        else if (LOG.isDebugEnabled()) {
+            LOG.debug("Stale acknowledge message received: [" +
+                "nodeId=" + nodeId + ", " +
+                "queryId=" + msg.queryId() + ", " +
+                "fragmentId=" + msg.fragmentId() + ", " +
+                "exchangeId=" + msg.exchangeId() + ", " +
+                "batchId=" + msg.batchId() + "]");
+        }
+    }
+
+    /** */
+    protected void onMessage(String nodeId, QueryBatchMessage msg) {
+        Inbox<?> inbox = mailboxRegistry.inbox(msg.queryId(), msg.exchangeId());
+
+        if (inbox == null && msg.batchId() == 0) {
+            // first message sent before a fragment is built
+            // note that an inbox source fragment id is also used as an exchange id
+            Inbox<?> newInbox = new Inbox<>(baseInboxContext(nodeId, msg.queryId(), msg.fragmentId()),
+                this, mailboxRegistry, msg.exchangeId(), msg.exchangeId());
+
+            inbox = mailboxRegistry.register(newInbox);
+        }
+
+        if (inbox != null) {
+            try {
+                inbox.onBatchReceived(nodeId, msg.batchId(), msg.last(), Commons.cast(msg.rows()));
+            }
+            catch (Throwable e) {
+                inbox.onError(e);
+
+                throw new IgniteInternalException("Unexpected exception", e);
+            }
+        }
+        else if (LOG.isDebugEnabled()) {
+            LOG.debug("Stale batch message received: [" +
+                "nodeId=" + nodeId + ", " +
+                "queryId=" + msg.queryId() + ", " +
+                "fragmentId=" + msg.fragmentId() + ", " +
+                "exchangeId=" + msg.exchangeId() + ", " +
+                "batchId=" + msg.batchId() + "]");
+        }
+    }
+
+    /**
+     * @return Minimal execution context to meet Inbox needs.
+     */
+    private ExecutionContext<?> baseInboxContext(String nodeId, UUID qryId, long fragmentId) {
+        return new ExecutionContext<>(
+            taskExecutor,
+            PlanningContext.builder()
+                .originatingNodeId(nodeId)
+                .build(),
+            qryId,
+            new FragmentDescription(
+                fragmentId,
+                null,
+                null,
+                null),
+            null,
+            ImmutableMap.of());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
index 96ad599..12570ca 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
@@ -134,7 +134,7 @@ public class ExecutionContext<Row> implements DataContext {
      * @param exchangeId ExchangeId to find remote nodes for.
      * @return Remote nodes for given exchangeId.
      */
-    public List<UUID> remotes(long exchangeId) {
+    public List<String> remotes(long exchangeId) {
         return fragmentDesc.remotes().get(exchangeId);
     }
 
@@ -170,7 +170,7 @@ public class ExecutionContext<Row> implements DataContext {
     /**
      * @return Originating node ID.
      */
-    public UUID originatingNodeId() {
+    public String originatingNodeId() {
         return planningContext().originatingNodeId();
     }
 
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/exec/ExecutionService.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
index 22da20a..d98fa19 100644
--- 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/exec/ExecutionService.java
@@ -15,24 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.ignite.internal.util.Cursor;
 
 /**
- * Ignite distribution trait.
+ *
  */
-public interface IgniteDistribution extends RelDistribution {
+public interface ExecutionService {
     /**
-     * @return Distribution function.
+     * Executes a query.
+     *
+     * @param schema Schema name.
+     * @param query Query.
+     * @param params Query parameters.
+     * @return Query cursor.
      */
-    DistributionFunction function();
-
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    List<Cursor<List<?>>> executeQuery(String schema, String query, Object[] params);
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    /**
+     * Cancels a running query.
+     *
+     * @param queryId Query ID.
+     */
+    void cancelQuery(UUID queryId);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
new file mode 100644
index 0000000..15e0b1c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -0,0 +1,865 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlDdl;
+import org.apache.calcite.sql.SqlExplain;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.RootNode;
+import org.apache.ignite.internal.processors.query.calcite.message.ErrorMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryStartRequest;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryStartResponse;
+import org.apache.ignite.internal.processors.query.calcite.message.SqlQueryMessageGroup;
+import org.apache.ignite.internal.processors.query.calcite.message.SqlQueryMessagesFactory;
+import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.RemoteException;
+import org.apache.ignite.internal.processors.query.calcite.prepare.CacheKey;
+import org.apache.ignite.internal.processors.query.calcite.prepare.DdlPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ExplainPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadata;
+import org.apache.ignite.internal.processors.query.calcite.prepare.FieldsMetadataImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.prepare.FragmentPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepDmlPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.MultiStepQueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCache;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryTemplate;
+import org.apache.ignite.internal.processors.query.calcite.prepare.Splitter;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ValidationResult;
+import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlSqlToCommandConverter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
+import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.NodeLeaveHandler;
+import org.apache.ignite.internal.processors.query.calcite.util.TransformingIterator;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.internal.util.Cancellable;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.Collections.singletonList;
+import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
+import static org.apache.ignite.internal.processors.query.calcite.exec.PlannerHelper.optimize;
+import static org.apache.ignite.internal.processors.query.calcite.externalize.RelJsonReader.fromJson;
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.FRAMEWORK_CONFIG;
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class ExecutionServiceImpl<Row> implements ExecutionService {
+    private static final IgniteLogger LOG = IgniteLogger.forClass(ExecutionServiceImpl.class);
+
+    private static final SqlQueryMessagesFactory FACTORY = new SqlQueryMessagesFactory();
+
+    /** */
+    private final MessageService msgSrvc;
+
+    /** */
+    private final String locNodeId;
+
+    /** */
+    private final QueryPlanCache qryPlanCache;
+
+    /** */
+    private final SchemaHolder schemaHolder;
+
+    /** */
+    private final QueryTaskExecutor taskExecutor;
+
+    /** */
+    private final AffinityService affSrvc;
+
+    /** */
+    private final MailboxRegistry mailboxRegistry;
+
+    /** */
+    private final MappingService mappingSrvc;
+
+    /** */
+    private final ExchangeService exchangeSrvc;
+
+    /** */
+    private final ClosableIteratorsHolder iteratorsHolder;
+
+    /** */
+    private final Map<UUID, QueryInfo> running;
+
+    /** */
+    private final RowHandler<Row> handler;
+
+    /** */
+    private final DdlSqlToCommandConverter ddlConverter;
+
+    public ExecutionServiceImpl(
+        TopologyService topSrvc,
+        MessageService msgSrvc,
+        QueryPlanCache planCache,
+        SchemaHolder schemaHolder,
+        QueryTaskExecutor taskExecutor,
+        RowHandler<Row> handler
+    ) {
+        this.handler = handler;
+        this.msgSrvc = msgSrvc;
+        this.schemaHolder = schemaHolder;
+        this.taskExecutor = taskExecutor;
+
+        locNodeId = topSrvc.localMember().id();
+        qryPlanCache = planCache;
+        running = new ConcurrentHashMap<>();
+        ddlConverter = new DdlSqlToCommandConverter();
+        iteratorsHolder = new ClosableIteratorsHolder(LOG);
+        mailboxRegistry = new MailboxRegistryImpl(topSrvc);
+        exchangeSrvc = new ExchangeServiceImpl(taskExecutor, mailboxRegistry, msgSrvc);
+        mappingSrvc = new MappingServiceImpl(topSrvc);
+        // TODO: fix this
+        affSrvc = cacheId -> Objects::hashCode;
+
+        topSrvc.addEventHandler(new NodeLeaveHandler(this::onNodeLeft));
+
+        init();
+    }
+
+    private void init() {
+        msgSrvc.register((n, m) -> onMessage(n, (QueryStartRequest) m), SqlQueryMessageGroup.QUERY_START_REQUEST);
+        msgSrvc.register((n, m) -> onMessage(n, (QueryStartResponse) m), SqlQueryMessageGroup.QUERY_START_RESPONSE);
+        msgSrvc.register((n, m) -> onMessage(n, (ErrorMessage) m), SqlQueryMessageGroup.ERROR_MESSAGE);
+
+        iteratorsHolder.init();
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Cursor<List<?>>> executeQuery(
+        String schema,
+        String qry,
+        Object[] params
+    ) {
+        PlanningContext pctx = createContext(topologyVersion(), locNodeId, schema, qry, params);
+
+        List<QueryPlan> qryPlans = qryPlanCache.queryPlan(pctx, new CacheKey(pctx.schemaName(), pctx.query()), this::prepareQuery);
+
+        return executePlans(qryPlans, pctx);
+    }
+
+    /**
+     * Executes prepared plans.
+     * @param qryPlans Query plans.
+     * @param pctx Query context.
+     * @return List of query result cursors.
+     */
+    @NotNull public List<Cursor<List<?>>> executePlans(
+        Collection<QueryPlan> qryPlans,
+        PlanningContext pctx
+    ) {
+        List<Cursor<List<?>>> cursors = new ArrayList<>(qryPlans.size());
+
+        for (QueryPlan plan : qryPlans) {
+            UUID qryId = UUID.randomUUID();
+
+            Cursor<List<?>> cur = executePlan(qryId, pctx, plan);
+
+            cursors.add(cur);
+        }
+
+        return cursors;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelQuery(UUID qryId) {
+        QueryInfo info = running.get(qryId);
+
+        if (info != null)
+            info.cancel();
+    }
+
+    /** */
+    protected long topologyVersion() {
+        return 1L;
+    }
+
+    /** */
+    private PlanningContext createContext(long topVer, String originator,
+        @Nullable String schema, String qry, Object[] params) {
+        RelTraitDef<?>[] traitDefs = {
+            ConventionTraitDef.INSTANCE,
+            RelCollationTraitDef.INSTANCE,
+            DistributionTraitDef.INSTANCE,
+            RewindabilityTraitDef.INSTANCE,
+            CorrelationTraitDef.INSTANCE,
+        };
+
+        return PlanningContext.builder()
+            .localNodeId(locNodeId)
+            .originatingNodeId(originator)
+            .parentContext(Contexts.empty())
+            .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                .defaultSchema(schema != null
+                    ? schemaHolder.schema().getSubSchema(schema)
+                    : schemaHolder.schema())
+                .traitDefs(traitDefs)
+                .build())
+            .query(qry)
+            .parameters(params)
+            .topologyVersion(topVer)
+            .build();
+    }
+
+    /** */
+    private List<QueryPlan> prepareQuery(PlanningContext ctx) {
+        try {
+            String qry = ctx.query();
+
+            assert qry != null;
+
+            // Parse query.
+            SqlNode sqlNode = ctx.planner().parse(qry);
+
+            if (single(sqlNode))
+                return singletonList(prepareSingle(sqlNode, ctx));
+
+            List<SqlNode> nodes = ((SqlNodeList) sqlNode).getList();
+            List<QueryPlan> res = new ArrayList<>(nodes.size());
+
+            for (SqlNode node : nodes)
+                res.add(prepareSingle(node, ctx));
+
+            return res;
+        }
+        catch (SqlParseException e) {
+            throw new IgniteInternalException("Failed to parse query", e);
+        }
+        catch (ValidationException e) {
+            throw new IgniteInternalException("Failed to validate query", e);
+        }
+        catch (Exception e) {
+            throw new IgniteInternalException("Failed to plan query.", e);
+        }
+    }
+
+    /** */
+    private List<QueryPlan> prepareFragment(PlanningContext ctx) {
+        return ImmutableList.of(new FragmentPlan(fromJson(ctx, ctx.query())));
+    }
+
+    /** */
+    private QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) throws ValidationException {
+        assert single(sqlNode);
+
+        ctx.planner().reset();
+
+        switch (sqlNode.getKind()) {
+            case SELECT:
+            case ORDER_BY:
+            case WITH:
+            case VALUES:
+            case UNION:
+            case EXCEPT:
+            case INTERSECT:
+                return prepareQuery(sqlNode, ctx);
+
+            case INSERT:
+            case DELETE:
+            case UPDATE:
+                return prepareDml(sqlNode, ctx);
+
+            case EXPLAIN:
+                return prepareExplain(sqlNode, ctx);
+
+            case CREATE_TABLE:
+            case DROP_TABLE:
+                return prepareDdl(sqlNode, ctx);
+
+            default:
+                throw new IgniteInternalException("Unsupported operation [" +
+                    "sqlNodeKind=" + sqlNode.getKind() + "; " +
+                    "querySql=\"" + ctx.query() + "\"]");
+        }
+    }
+
+    /** */
+    private QueryPlan prepareQuery(SqlNode sqlNode, PlanningContext ctx) {
+        IgnitePlanner planner = ctx.planner();
+
+        // Validate
+        ValidationResult validated = planner.validateAndGetTypeMetadata(sqlNode);
+
+        sqlNode = validated.sqlNode();
+
+        IgniteRel igniteRel = optimize(sqlNode, planner, LOG);
+
+        // Split query plan to query fragments.
+        List<Fragment> fragments = new Splitter().go(igniteRel);
+
+        QueryTemplate template = new QueryTemplate(mappingSrvc, fragments);
+
+        return new MultiStepQueryPlan(template, queryFieldsMetadata(ctx, validated.dataType(), validated.origins()));
+    }
+
+    /** */
+    private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws ValidationException {
+        IgnitePlanner planner = ctx.planner();
+
+        // Validate
+        sqlNode = planner.validate(sqlNode);
+
+        // Convert to Relational operators graph
+        IgniteRel igniteRel = optimize(sqlNode, planner, LOG);
+
+        // Split query plan to query fragments.
+        List<Fragment> fragments = new Splitter().go(igniteRel);
+
+        QueryTemplate template = new QueryTemplate(mappingSrvc, fragments);
+
+        return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
+    }
+
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        return new DdlPlan(ddlConverter.convert(ddlNode, ctx));
+    }
+
+    /** */
+    private QueryPlan prepareExplain(SqlNode explain, PlanningContext ctx) throws ValidationException {
+        IgnitePlanner planner = ctx.planner();
+
+        SqlNode sql = ((SqlExplain)explain).getExplicandum();
+
+        // Validate
+        sql = planner.validate(sql);
+
+        // Convert to Relational operators graph
+        IgniteRel igniteRel = optimize(sql, planner, LOG);
+
+        String plan = RelOptUtil.toString(igniteRel, SqlExplainLevel.ALL_ATTRIBUTES);
+
+        return new ExplainPlan(plan, explainFieldsMetadata(ctx));
+    }
+
+    /** */
+    private FieldsMetadata explainFieldsMetadata(PlanningContext ctx) {
+        IgniteTypeFactory factory = ctx.typeFactory();
+        RelDataType planStrDataType =
+            factory.createSqlType(SqlTypeName.VARCHAR, PRECISION_NOT_SPECIFIED);
+        Map.Entry<String, RelDataType> planField = new IgniteBiTuple<>(ExplainPlan.PLAN_COL_NAME, planStrDataType);
+        RelDataType planDataType = factory.createStructType(singletonList(planField));
+
+        return queryFieldsMetadata(ctx, planDataType, null);
+    }
+
+    /** */
+    private Cursor<List<?>> executePlan(UUID qryId, PlanningContext pctx, QueryPlan plan) {
+        switch (plan.type()) {
+            case DML:
+                // TODO a barrier between previous operation and this one
+            case QUERY:
+                return executeQuery(qryId, (MultiStepPlan) plan, pctx);
+            case EXPLAIN:
+                return executeExplain((ExplainPlan)plan);
+            case DDL:
+                return executeDdl((DdlPlan)plan, pctx);
+
+            default:
+                throw new AssertionError("Unexpected plan type: " + plan);
+        }
+    }
+
+    /** */
+    private Cursor<List<?>> executeDdl(DdlPlan plan, PlanningContext pctx) {
+        throw new UnsupportedOperationException("plan=" + plan + ", ctx=" + pctx);
+    }
+
+    /** */
+    private Cursor<List<?>> executeQuery(UUID qryId, MultiStepPlan plan, PlanningContext pctx) {
+        plan.init(pctx);
+
+        List<Fragment> fragments = plan.fragments();
+
+        // Local execution
+        Fragment fragment = first(fragments);
+
+        if (IgniteUtils.assertionsEnabled()) {
+            assert fragment != null;
+
+            FragmentMapping mapping = plan.mapping(fragment);
+
+            assert mapping != null;
+
+            List<String> nodes = mapping.nodeIds();
+
+            assert nodes != null && nodes.size() == 1 && first(nodes).equals(pctx.localNodeId());
+        }
+
+        FragmentDescription fragmentDesc = new FragmentDescription(
+            fragment.fragmentId(),
+            plan.mapping(fragment),
+            plan.target(fragment),
+            plan.remotes(fragment));
+
+        ExecutionContext<Row> ectx = new ExecutionContext<>(
+            taskExecutor,
+            pctx,
+            qryId,
+            fragmentDesc,
+            handler,
+            Commons.parametersMap(pctx.parameters()));
+
+        Node<Row> node = new LogicalRelImplementor<>(ectx, affSrvc, mailboxRegistry,
+            exchangeSrvc).go(fragment.root());
+
+        QueryInfo info = new QueryInfo(ectx, plan, node);
+
+        // register query
+        register(info);
+
+        // start remote execution
+        for (int i = 1; i < fragments.size(); i++) {
+            fragment = fragments.get(i);
+            fragmentDesc = new FragmentDescription(
+                fragment.fragmentId(),
+                plan.mapping(fragment),
+                plan.target(fragment),
+                plan.remotes(fragment));
+
+            Throwable ex = null;
+            for (String nodeId : fragmentDesc.nodeIds()) {
+                if (ex != null)
+                    info.onResponse(nodeId, fragment.fragmentId(), ex);
+                else {
+                    try {
+                        QueryStartRequest req = FACTORY.queryStartRequest()
+                            .queryId(qryId)
+                            .fragmentId(fragment.fragmentId())
+                            .schema(pctx.schemaName())
+                            .root(fragment.serialized())
+                            .topologyVersion(pctx.topologyVersion())
+                            .fragmentDescription(fragmentDesc)
+                            .parameters(pctx.parameters())
+                            .build();
+
+                        msgSrvc.send(nodeId, req);
+                    }
+                    catch (Throwable e) {
+                        info.onResponse(nodeId, fragment.fragmentId(), ex = e);
+                    }
+                }
+            }
+        }
+
+        return Commons.createCursor(new TransformingIterator<>(info.iterator(), row -> {
+            int rowSize = ectx.rowHandler().columnCount(row);
+
+            List<Object> res = new ArrayList<>(rowSize);
+
+            for (int i = 0; i < rowSize; i++)
+                res.add(ectx.rowHandler().get(i, row));
+
+            return res;
+        }));
+    }
+
+    /** */
+    private Cursor<List<?>> executeExplain(ExplainPlan plan) {
+        Cursor<List<?>> cur = Commons.createCursor(singletonList(singletonList(plan.plan())));
+        // TODO: fix this
+//        cur.fieldsMeta(plan.fieldsMeta().queryFieldsMetadata(pctx.typeFactory()));
+
+        return cur;
+    }
+
+    /** */
+    private void executeFragment(UUID qryId, FragmentPlan plan, PlanningContext pctx, FragmentDescription fragmentDesc) {
+        ExecutionContext<Row> ectx = new ExecutionContext<>(taskExecutor, pctx, qryId,
+            fragmentDesc, handler, Commons.parametersMap(pctx.parameters()));
+
+        long frId = fragmentDesc.fragmentId();
+        String origNodeId = pctx.originatingNodeId();
+
+        Outbox<Row> node = new LogicalRelImplementor<>(
+            ectx,
+            affSrvc,
+            mailboxRegistry,
+            exchangeSrvc
+        ).go(plan.root());
+
+        try {
+            msgSrvc.send(
+                origNodeId,
+                FACTORY.queryStartResponse()
+                    .queryId(qryId)
+                    .fragmentId(frId)
+                    .build()
+            );
+        }
+        catch (IgniteInternalCheckedException e) {
+            IgniteInternalException wrpEx = new IgniteInternalException("Failed to send reply. [nodeId=" + origNodeId + ']', e);
+
+            throw wrpEx;
+        }
+
+        node.init();
+    }
+
+    /** */
+    private void register(QueryInfo info) {
+        UUID qryId = info.ctx.queryId();
+
+        running.put(qryId, info);
+    }
+
+    /** */
+    private FieldsMetadata queryFieldsMetadata(PlanningContext ctx, RelDataType sqlType,
+        @Nullable List<List<String>> origins) {
+        RelDataType resultType = TypeUtils.getResultType(
+            ctx.typeFactory(), ctx.catalogReader(), sqlType, origins);
+        return new FieldsMetadataImpl(resultType, origins);
+    }
+
+    /** */
+    private boolean single(SqlNode sqlNode) {
+        return !(sqlNode instanceof SqlNodeList);
+    }
+
+    /** */
+    private void onMessage(String nodeId, QueryStartRequest msg) {
+        assert nodeId != null && msg != null;
+
+        try {
+            PlanningContext pctx = createContext(msg.topologyVersion(), nodeId, msg.schema(),
+                msg.root(), msg.parameters());
+
+            List<QueryPlan> qryPlans = qryPlanCache.queryPlan(
+                pctx,
+                new CacheKey(pctx.schemaName(), pctx.query()),
+                this::prepareFragment
+            );
+
+            assert qryPlans.size() == 1 && qryPlans.get(0).type() == QueryPlan.Type.FRAGMENT;
+
+            FragmentPlan plan = (FragmentPlan)qryPlans.get(0);
+
+            executeFragment(msg.queryId(), plan, pctx, msg.fragmentDescription());
+        }
+        catch (Throwable ex) {
+            LOG.error("Failed to start query fragment", ex);
+
+            mailboxRegistry.outboxes(msg.queryId(), msg.fragmentId(), -1)
+                .forEach(Outbox::close);
+            mailboxRegistry.inboxes(msg.queryId(), msg.fragmentId(), -1)
+                .forEach(Inbox::close);
+
+            try {
+                msgSrvc.send(
+                    nodeId,
+                    FACTORY.queryStartResponse()
+                        .queryId(msg.queryId())
+                        .fragmentId(msg.fragmentId())
+                        .error(ex)
+                        .build()
+                );
+            }
+            catch (Exception e) {
+                LOG.error("Error occurred during send error message", e);
+
+                IgniteInternalException wrpEx = new IgniteInternalException("Error occurred during send error message", e);
+
+                e.addSuppressed(ex);
+
+                throw wrpEx;
+            }
+
+            throw ex;
+        }
+    }
+
+    /** */
+    private void onMessage(String nodeId, QueryStartResponse msg) {
+        assert nodeId != null && msg != null;
+
+        QueryInfo info = running.get(msg.queryId());
+
+        if (info != null)
+            info.onResponse(nodeId, msg.fragmentId(), msg.error());
+    }
+
+    /** */
+    private void onMessage(String nodeId, ErrorMessage msg) {
+        assert nodeId != null && msg != null;
+
+        QueryInfo info = running.get(msg.queryId());
+
+        if (info != null)
+            info.onError(new RemoteException(nodeId, msg.queryId(), msg.fragmentId(), msg.error()));
+    }
+
+    /** */
+    private void onNodeLeft(ClusterNode node) {
+        running.forEach((uuid, queryInfo) -> queryInfo.onNodeLeft(node.id()));
+    }
+
+    /** */
+    private enum QueryState {
+        /** */
+        RUNNING,
+
+        /** */
+        CLOSING,
+
+        /** */
+        CLOSED
+    }
+
+    /** */
+    private static final class RemoteFragmentKey {
+        /** */
+        private final String nodeId;
+
+        /** */
+        private final long fragmentId;
+
+        /** */
+        private RemoteFragmentKey(String nodeId, long fragmentId) {
+            this.nodeId = nodeId;
+            this.fragmentId = fragmentId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            RemoteFragmentKey that = (RemoteFragmentKey) o;
+
+            if (fragmentId != that.fragmentId)
+                return false;
+            return nodeId.equals(that.nodeId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = nodeId.hashCode();
+            res = 31 * res + (int) (fragmentId ^ (fragmentId >>> 32));
+            return res;
+        }
+    }
+
+    /** */
+    private final class QueryInfo implements Cancellable {
+        /** */
+        private final ExecutionContext<Row> ctx;
+
+        /** */
+        private final RootNode<Row> root;
+
+        /** remote nodes */
+        private final Set<String> remotes;
+
+        /** node to fragment */
+        private final Set<RemoteFragmentKey> waiting;
+
+        /** */
+        private volatile QueryState state;
+
+        /** */
+        private QueryInfo(ExecutionContext<Row> ctx, MultiStepPlan plan, Node<Row> root) {
+            this.ctx = ctx;
+
+            RootNode<Row> rootNode = new RootNode<>(ctx, plan.fieldsMetadata().rowType(), this::tryClose);
+            rootNode.register(root);
+
+            this.root = rootNode;
+
+            remotes = new HashSet<>();
+            waiting = new HashSet<>();
+
+            for (int i = 1; i < plan.fragments().size(); i++) {
+                Fragment fragment = plan.fragments().get(i);
+                List<String> nodes = plan.mapping(fragment).nodeIds();
+
+                remotes.addAll(nodes);
+
+                for (String node : nodes)
+                    waiting.add(new RemoteFragmentKey(node, fragment.fragmentId()));
+            }
+
+            state = QueryState.RUNNING;
+        }
+
+        /** */
+        public Iterator<Row> iterator() {
+            return iteratorsHolder.iterator(root);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            root.close();
+        }
+
+        /**
+         * Can be called multiple times after receive each error at {@link #onResponse(RemoteFragmentKey, Throwable)}.
+         */
+        private void tryClose() {
+            QueryState state0 = null;
+
+            synchronized (this) {
+                if (state == QueryState.CLOSED)
+                    return;
+
+                if (state == QueryState.RUNNING)
+                    state0 = state = QueryState.CLOSING;
+
+                // 1) close local fragment
+                root.closeInternal();
+
+                if (state == QueryState.CLOSING && waiting.isEmpty())
+                    state0 = state = QueryState.CLOSED;
+            }
+
+            if (state0 == QueryState.CLOSED) {
+                // 2) unregister runing query
+                running.remove(ctx.queryId());
+
+                IgniteInternalException wrpEx = null;
+
+                // 3) close remote fragments
+                for (String nodeId : remotes) {
+                    try {
+                        exchangeSrvc.closeOutbox(nodeId, ctx.queryId(), -1, -1);
+                    }
+                    catch (IgniteInternalCheckedException e) {
+                        if (wrpEx == null)
+                            wrpEx = new IgniteInternalException("Failed to send cancel message. [nodeId=" + nodeId + ']', e);
+                        else
+                            wrpEx.addSuppressed(e);
+                    }
+                }
+
+                // 4) Cancel local fragment
+                root.context().execute(ctx::cancel, root::onError);
+
+                if (wrpEx != null)
+                    throw wrpEx;
+            }
+        }
+
+        /** */
+        private void onNodeLeft(String nodeId) {
+            List<RemoteFragmentKey> fragments = null;
+
+            synchronized (this) {
+                for (RemoteFragmentKey fragment : waiting) {
+                    if (!fragment.nodeId.equals(nodeId))
+                        continue;
+
+                    if (fragments == null)
+                        fragments = new ArrayList<>();
+
+                    fragments.add(fragment);
+                }
+            }
+
+            if (!nullOrEmpty(fragments)) {
+                IgniteInternalCheckedException ex = new IgniteInternalCheckedException(
+                    "Failed to start query, node left. nodeId=" + nodeId);
+
+                for (RemoteFragmentKey fragment : fragments)
+                    onResponse(fragment, ex);
+            }
+        }
+
+        /** */
+        private void onResponse(String nodeId, long fragmentId, Throwable error) {
+            onResponse(new RemoteFragmentKey(nodeId, fragmentId), error);
+        }
+
+        /** */
+        private void onResponse(RemoteFragmentKey fragment, Throwable error) {
+            QueryState state;
+            synchronized (this) {
+                waiting.remove(fragment);
+                state = this.state;
+            }
+
+            if (error != null)
+                onError(error);
+            else if (state == QueryState.CLOSING)
+                tryClose();
+        }
+
+        /** */
+        private void onError(Throwable error) {
+            root.onError(error);
+
+            tryClose();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
new file mode 100644
index 0000000..eed5440
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
@@ -0,0 +1,654 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.core.Spool;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractSetOpNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.CorrelatedNestedLoopJoinNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.FilterNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.HashAggregateNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.IndexSpoolNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.IntersectNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.LimitNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.MergeJoinNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.MinusNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.NestedLoopJoinNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Node;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.ProjectNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.ScanNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.SortAggregateNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.SortNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.TableSpoolNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.UnionAllNode;
+import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteCorrelatedNestedLoopJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteHashIndexSpool;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteLimit;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteMergeJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteNestedLoopJoin;
+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.IgniteSort;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSortedIndexSpool;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableFunctionScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableSpool;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTrimExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteUnionAll;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
+import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteMapHashAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteMapSortAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceHashAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+import static org.apache.calcite.rel.RelDistribution.Type.HASH_DISTRIBUTED;
+import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.combinedRowType;
+import static org.apache.ignite.internal.util.ArrayUtils.asList;
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+
+/**
+ * Implements a query plan.
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class LogicalRelImplementor<Row> implements IgniteRelVisitor<Node<Row>> {
+    /** */
+    public static final String CNLJ_NOT_SUPPORTED_JOIN_ASSERTION_MSG = "only INNER and LEFT join supported by IgniteCorrelatedNestedLoop";
+
+    /** */
+    private final ExecutionContext<Row> ctx;
+
+    /** */
+    private final AffinityService affSrvc;
+
+    /** */
+    private final ExchangeService exchangeSvc;
+
+    /** */
+    private final MailboxRegistry mailboxRegistry;
+
+    /** */
+    private final ExpressionFactory<Row> expressionFactory;
+
+    /**
+     * @param ctx Root context.
+     * @param affSrvc Affinity service.
+     * @param mailboxRegistry Mailbox registry.
+     * @param exchangeSvc Exchange service.
+     */
+    public LogicalRelImplementor(
+        ExecutionContext<Row> ctx,
+        AffinityService affSrvc,
+        MailboxRegistry mailboxRegistry,
+        ExchangeService exchangeSvc
+    ) {
+        this.affSrvc = affSrvc;
+        this.mailboxRegistry = mailboxRegistry;
+        this.exchangeSvc = exchangeSvc;
+        this.ctx = ctx;
+
+        expressionFactory = ctx.expressionFactory();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteSender rel) {
+        IgniteDistribution distribution = rel.distribution();
+
+        Destination<Row> dest = distribution.destination(ctx, affSrvc, ctx.target());
+
+        // Outbox fragment ID is used as exchange ID as well.
+        Outbox<Row> outbox =
+            new Outbox<>(ctx, rel.getRowType(), exchangeSvc, mailboxRegistry, rel.exchangeId(), rel.targetFragmentId(), dest);
+
+        Node<Row> input = visit(rel.getInput());
+
+        outbox.register(input);
+
+        mailboxRegistry.register(outbox);
+
+        return outbox;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteFilter rel) {
+        Predicate<Row> pred = expressionFactory.predicate(rel.getCondition(), rel.getRowType());
+
+        FilterNode<Row> node = new FilterNode<>(ctx, rel.getRowType(), pred);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteTrimExchange rel) {
+        assert TraitUtils.distribution(rel).getType() == HASH_DISTRIBUTED;
+
+        IgniteDistribution distr = rel.distribution();
+        Destination<Row> dest = distr.destination(ctx, affSrvc, ctx.group(rel.sourceId()));
+        String localNodeId = ctx.planningContext().localNodeId();
+
+        FilterNode<Row> node = new FilterNode<>(ctx, rel.getRowType(), r -> Objects.equals(localNodeId, first(dest.targets(r))));
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteProject rel) {
+        Function<Row, Row> prj = expressionFactory.project(rel.getProjects(), rel.getInput().getRowType());
+
+        ProjectNode<Row> node = new ProjectNode<>(ctx, rel.getRowType(), prj);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteNestedLoopJoin rel) {
+        RelDataType outType = rel.getRowType();
+        RelDataType leftType = rel.getLeft().getRowType();
+        RelDataType rightType = rel.getRight().getRowType();
+        JoinRelType joinType = rel.getJoinType();
+
+        RelDataType rowType = combinedRowType(ctx.getTypeFactory(), leftType, rightType);
+        Predicate<Row> cond = expressionFactory.predicate(rel.getCondition(), rowType);
+
+        Node<Row> node = NestedLoopJoinNode.create(ctx, outType, leftType, rightType, joinType, cond);
+
+        Node<Row> leftInput = visit(rel.getLeft());
+        Node<Row> rightInput = visit(rel.getRight());
+
+        node.register(asList(leftInput, rightInput));
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteCorrelatedNestedLoopJoin rel) {
+        RelDataType outType = rel.getRowType();
+        RelDataType leftType = rel.getLeft().getRowType();
+        RelDataType rightType = rel.getRight().getRowType();
+
+        RelDataType rowType = combinedRowType(ctx.getTypeFactory(), leftType, rightType);
+        Predicate<Row> cond = expressionFactory.predicate(rel.getCondition(), rowType);
+
+        assert rel.getJoinType() == JoinRelType.INNER || rel.getJoinType() == JoinRelType.LEFT
+            : CNLJ_NOT_SUPPORTED_JOIN_ASSERTION_MSG;
+
+        Node<Row> node = new CorrelatedNestedLoopJoinNode<>(ctx, outType, cond, rel.getVariablesSet(),
+            rel.getJoinType());
+
+        Node<Row> leftInput = visit(rel.getLeft());
+        Node<Row> rightInput = visit(rel.getRight());
+
+        node.register(asList(leftInput, rightInput));
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteMergeJoin rel) {
+        RelDataType outType = rel.getRowType();
+        RelDataType leftType = rel.getLeft().getRowType();
+        RelDataType rightType = rel.getRight().getRowType();
+        JoinRelType joinType = rel.getJoinType();
+
+        int pairsCnt = rel.analyzeCondition().pairs().size();
+
+        Comparator<Row> comp = expressionFactory.comparator(
+            rel.leftCollation().getFieldCollations().subList(0, pairsCnt),
+            rel.rightCollation().getFieldCollations().subList(0, pairsCnt)
+        );
+
+        Node<Row> node = MergeJoinNode.create(ctx, outType, leftType, rightType, joinType, comp);
+
+        Node<Row> leftInput = visit(rel.getLeft());
+        Node<Row> rightInput = visit(rel.getRight());
+
+        node.register(asList(leftInput, rightInput));
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteIndexScan rel) {
+        // TODO: fix this
+//        RexNode condition = rel.condition();
+//        List<RexNode> projects = rel.projects();
+//
+        IgniteTable tbl = rel.getTable().unwrap(IgniteTable.class);
+        IgniteTypeFactory typeFactory = ctx.getTypeFactory();
+
+        ImmutableBitSet requiredColumns = rel.requiredColumns();
+//        List<RexNode> lowerCond = rel.lowerBound();
+//        List<RexNode> upperCond = rel.upperBound();
+//
+        RelDataType rowType = tbl.getRowType(typeFactory, requiredColumns);
+
+//        Predicate<Row> filters = condition == null ? null : expressionFactory.predicate(condition, rowType);
+//        Supplier<Row> lower = lowerCond == null ? null : expressionFactory.rowSource(lowerCond);
+//        Supplier<Row> upper = upperCond == null ? null : expressionFactory.rowSource(upperCond);
+//        Function<Row, Row> prj = projects == null ? null : expressionFactory.project(projects, rowType);
+//
+//        IgniteIndex idx = tbl.getIndex(rel.indexName());
+//
+//        ColocationGroup group = ctx.group(rel.sourceId());
+
+        Iterable<Row> rowsIter = (Iterable<Row>) List.of(new Object[]{0, 0}, new Object[]{1, 1});//idx.scan(ctx, group, filters, lower, upper, prj, requiredColumns);
+
+        return new ScanNode<>(ctx, rowType, rowsIter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteTableScan rel) {
+        RexNode condition = rel.condition();
+        List<RexNode> projects = rel.projects();
+        ImmutableBitSet requiredColunms = rel.requiredColumns();
+
+        IgniteTable tbl = rel.getTable().unwrap(IgniteTable.class);
+        IgniteTypeFactory typeFactory = ctx.getTypeFactory();
+
+        RelDataType rowType = tbl.getRowType(typeFactory, requiredColunms);
+
+        Predicate<Row> filters = condition == null ? null : expressionFactory.predicate(condition, rowType);
+        Function<Row, Row> prj = projects == null ? null : expressionFactory.project(projects, rowType);
+
+        ColocationGroup group = ctx.group(rel.sourceId());
+
+        Iterable<Row> rowsIter = tbl.scan(ctx, group, filters, prj, requiredColunms);
+
+        return new ScanNode<>(ctx, rowType, rowsIter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteValues rel) {
+        List<RexLiteral> vals = Commons.flat(Commons.cast(rel.getTuples()));
+
+        RelDataType rowType = rel.getRowType();
+
+        return new ScanNode<>(ctx, rowType, expressionFactory.values(vals, rowType));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteUnionAll rel) {
+        UnionAllNode<Row> node = new UnionAllNode<>(ctx, rel.getRowType());
+
+        List<Node<Row>> inputs = Commons.transform(rel.getInputs(), this::visit);
+
+        node.register(inputs);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteLimit rel) {
+        Supplier<Integer> offset = (rel.offset() == null) ? null : expressionFactory.execute(rel.offset());
+        Supplier<Integer> fetch = (rel.fetch() == null) ? null : expressionFactory.execute(rel.fetch());
+
+        LimitNode<Row> node = new LimitNode<>(ctx, rel.getRowType(), offset, fetch);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteSort rel) {
+        RelCollation collation = rel.getCollation();
+
+        SortNode<Row> node = new SortNode<>(ctx, rel.getRowType(), expressionFactory.comparator(collation));
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteTableSpool rel) {
+        TableSpoolNode<Row> node = new TableSpoolNode<>(ctx, rel.getRowType(), rel.readType == Spool.Type.LAZY);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteSortedIndexSpool rel) {
+        RelCollation collation = rel.collation();
+
+        assert rel.indexCondition() != null : rel;
+
+        List<RexNode> lowerBound = rel.indexCondition().lowerBound();
+        List<RexNode> upperBound = rel.indexCondition().upperBound();
+
+        Predicate<Row> filter = expressionFactory.predicate(rel.condition(), rel.getRowType());
+        Supplier<Row> lower = lowerBound == null ? null : expressionFactory.rowSource(lowerBound);
+        Supplier<Row> upper = upperBound == null ? null : expressionFactory.rowSource(upperBound);
+
+        IndexSpoolNode<Row> node = IndexSpoolNode.createTreeSpool(
+            ctx,
+            rel.getRowType(),
+            collation,
+            expressionFactory.comparator(collation),
+            filter,
+            lower,
+            upper
+        );
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteHashIndexSpool rel) {
+        Supplier<Row> searchRow = expressionFactory.rowSource(rel.searchRow());
+
+        IndexSpoolNode<Row> node = IndexSpoolNode.createHashSpool(
+            ctx,
+            rel.getRowType(),
+            ImmutableBitSet.of(rel.keys()),
+            searchRow
+        );
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteSetOp rel) {
+        RelDataType rowType = rel.getRowType();
+
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        List<Node<Row>> inputs = Commons.transform(rel.getInputs(), this::visit);
+
+        AbstractSetOpNode<Row> node;
+
+        if (rel instanceof Minus)
+            node = new MinusNode<>(ctx, rowType, rel.aggregateType(), rel.all(), rowFactory);
+        else if (rel instanceof Intersect)
+            node = new IntersectNode<>(ctx, rowType, rel.aggregateType(), rel.all(), rowFactory, rel.getInputs().size());
+        else
+            throw new AssertionError();
+
+        node.register(inputs);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteTableFunctionScan rel) {
+        Supplier<Iterable<Object[]>> dataSupplier = expressionFactory.execute(rel.getCall());
+
+        RelDataType rowType = rel.getRowType();
+
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        return new ScanNode<>(ctx, rowType, new TableFunctionScan<>(dataSupplier, rowFactory));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteTableModify rel) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteReceiver rel) {
+        Inbox<Row> inbox = mailboxRegistry.register(
+            new Inbox<>(ctx, exchangeSvc, mailboxRegistry, rel.exchangeId(), rel.sourceFragmentId()));
+
+        // here may be an already created (to consume rows from remote nodes) inbox
+        // without proper context, we need to init it with a right one.
+        inbox.init(ctx, rel.getRowType(), ctx.remotes(rel.exchangeId()), expressionFactory.comparator(rel.collation()));
+
+        return inbox;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteSingleHashAggregate rel) {
+        AggregateType type = AggregateType.SINGLE;
+
+        RelDataType rowType = rel.getRowType();
+        RelDataType inputType = rel.getInput().getRowType();
+
+        Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
+            type, rel.getAggCallList(), inputType);
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        HashAggregateNode<Row> node = new HashAggregateNode<>(ctx, rowType, type, rel.getGroupSets(), accFactory, rowFactory);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteMapHashAggregate rel) {
+        AggregateType type = AggregateType.MAP;
+
+        RelDataType rowType = rel.getRowType();
+        RelDataType inputType = rel.getInput().getRowType();
+
+        Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
+            type, rel.getAggCallList(), inputType);
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        HashAggregateNode<Row> node = new HashAggregateNode<>(ctx, rowType, type, rel.getGroupSets(), accFactory, rowFactory);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteReduceHashAggregate rel) {
+        AggregateType type = AggregateType.REDUCE;
+
+        RelDataType rowType = rel.getRowType();
+
+        Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
+            type, rel.getAggregateCalls(), null);
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        HashAggregateNode<Row> node = new HashAggregateNode<>(ctx, rowType, type, rel.getGroupSets(), accFactory, rowFactory);
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteSingleSortAggregate rel) {
+        AggregateType type = AggregateType.SINGLE;
+
+        RelDataType rowType = rel.getRowType();
+        RelDataType inputType = rel.getInput().getRowType();
+
+        Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
+            type,
+            rel.getAggCallList(),
+            inputType
+        );
+
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        SortAggregateNode<Row> node = new SortAggregateNode<>(
+            ctx,
+            rowType,
+            type,
+            rel.getGroupSet(),
+            accFactory,
+            rowFactory,
+            expressionFactory.comparator(rel.collation())
+        );
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteMapSortAggregate rel) {
+        AggregateType type = AggregateType.MAP;
+
+        RelDataType rowType = rel.getRowType();
+        RelDataType inputType = rel.getInput().getRowType();
+
+        Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
+            type,
+            rel.getAggCallList(),
+            inputType
+        );
+
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        SortAggregateNode<Row> node = new SortAggregateNode<>(
+            ctx,
+            rowType,
+            type,
+            rel.getGroupSet(),
+            accFactory,
+            rowFactory,
+            expressionFactory.comparator(rel.collation())
+        );
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteReduceSortAggregate rel) {
+        AggregateType type = AggregateType.REDUCE;
+
+        RelDataType rowType = rel.getRowType();
+
+        Supplier<List<AccumulatorWrapper<Row>>> accFactory = expressionFactory.accumulatorsFactory(
+            type,
+            rel.getAggregateCalls(),
+            null
+        );
+
+        RowFactory<Row> rowFactory = ctx.rowHandler().factory(ctx.getTypeFactory(), rowType);
+
+        SortAggregateNode<Row> node = new SortAggregateNode<>(
+            ctx,
+            rowType,
+            type,
+            rel.getGroupSet(),
+            accFactory,
+            rowFactory,
+            expressionFactory.comparator(rel.collation())
+            );
+
+        Node<Row> input = visit(rel.getInput());
+
+        node.register(input);
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteRel rel) {
+        return rel.accept(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Row> visit(IgniteExchange rel) {
+        throw new AssertionError();
+    }
+
+    /** */
+    private Node<Row> visit(RelNode rel) {
+        return visit((IgniteRel) rel);
+    }
+
+    /** */
+    public <T extends Node<Row>> T go(IgniteRel rel) {
+        return (T)visit(rel);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java
new file mode 100644
index 0000000..cdc6b54
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Collection;
+import java.util.UUID;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public interface MailboxRegistry {
+    /**
+     * Tries to register and inbox node and returns it if success or returns previously registered inbox otherwise.
+     *
+     * @param inbox Inbox.
+     * @return Registered inbox.
+     */
+    <T> Inbox<T> register(Inbox<T> inbox);
+
+    /**
+     * Unregisters an inbox.
+     *
+     * @param inbox Inbox to unregister.
+     */
+    void unregister(Inbox<?> inbox);
+
+    /**
+     * Registers an outbox.
+     *
+     * @param outbox Outbox to register.
+     */
+    void register(Outbox<?> outbox);
+
+    /**
+     * Unregisters an outbox.
+     *
+     * @param outbox Outbox to unregister.
+     */
+    void unregister(Outbox<?> outbox);
+
+    /**
+     * Returns a registered outbox by provided query ID, exchange ID pair.
+     *
+     * @param qryId Query ID.
+     * @param exchangeId Exchange ID.
+     *
+     * @return Registered outbox. May be {@code null} if execution was cancelled.
+     */
+    Outbox<?> outbox(UUID qryId, long exchangeId);
+
+    /**
+     * Returns a registered inbox by provided query ID, exchange ID pair.
+     *
+     * @param qryId Query ID.
+     * @param exchangeId Exchange ID.
+     *
+     * @return Registered inbox. May be {@code null} if execution was cancelled.
+     */
+    Inbox<?> inbox(UUID qryId, long exchangeId);
+
+    /**
+     * Returns all registered inboxes for provided query ID.
+     *
+     * @param qryId Query ID. {@code null} means return inboxes with any query id.
+     * @param fragmentId Fragment Id. {@code -1} means return inboxes with any fragment id.
+     * @param exchangeId Exchange Id. {@code -1} means return inboxes with any exchange id.
+     * @return Registered inboxes.
+     */
+    Collection<Inbox<?>> inboxes(@Nullable UUID qryId, long fragmentId, long exchangeId);
+
+    /**
+     * Returns all registered outboxes for provided query ID.
+     *
+     * @param qryId Query ID. {@code null} means return outboxes with any query id.
+     * @param fragmentId Fragment Id. {@code -1} means return outboxes with any fragment id.
+     * @param exchangeId Exchange Id. {@code -1} means return outboxes with any exchange id.
+     * @return Registered outboxes.
+     */
+    Collection<Outbox<?>> outboxes(@Nullable UUID qryId, long fragmentId, long exchangeId);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java
new file mode 100644
index 0000000..e9c46a9
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Mailbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Outbox;
+import org.apache.ignite.internal.processors.query.calcite.util.NodeLeaveHandler;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class MailboxRegistryImpl implements MailboxRegistry {
+    /** */
+    private static final Predicate<Mailbox<?>> ALWAYS_TRUE = o -> true;
+
+    /** */
+    private final Map<MailboxKey, Outbox<?>> locals;
+
+    /** */
+    private final Map<MailboxKey, Inbox<?>> remotes;
+
+    public MailboxRegistryImpl(TopologyService topSrvc) {
+        locals = new ConcurrentHashMap<>();
+        remotes = new ConcurrentHashMap<>();
+
+        topSrvc.addEventHandler(new NodeLeaveHandler(this::onNodeLeft));
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> Inbox<T> register(Inbox<T> inbox) {
+        Inbox<T> old = (Inbox<T>)remotes.putIfAbsent(new MailboxKey(inbox.queryId(), inbox.exchangeId()), inbox);
+
+        return old != null ? old : inbox;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(Inbox<?> inbox) {
+        remotes.remove(new MailboxKey(inbox.queryId(), inbox.exchangeId()), inbox);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(Outbox<?> outbox) {
+        Outbox<?> res = locals.put(new MailboxKey(outbox.queryId(), outbox.exchangeId()), outbox);
+
+        assert res == null : res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(Outbox<?> outbox) {
+        locals.remove(new MailboxKey(outbox.queryId(), outbox.exchangeId()), outbox);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Outbox<?> outbox(UUID qryId, long exchangeId) {
+        return locals.get(new MailboxKey(qryId, exchangeId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Inbox<?> inbox(UUID qryId, long exchangeId) {
+        return remotes.get(new MailboxKey(qryId, exchangeId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Inbox<?>> inboxes(@Nullable UUID qryId, long fragmentId, long exchangeId) {
+        return remotes.values().stream()
+            .filter(makeFilter(qryId, fragmentId, exchangeId))
+            .collect(Collectors.toList());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Outbox<?>> outboxes(@Nullable UUID qryId, long fragmentId, long exchangeId) {
+        return locals.values().stream()
+            .filter(makeFilter(qryId, fragmentId, exchangeId))
+            .collect(Collectors.toList());
+    }
+
+    /** */
+    private void onNodeLeft(ClusterNode node) {
+        locals.values().forEach(n -> n.onNodeLeft(node.id()));
+        remotes.values().forEach(n -> n.onNodeLeft(node.id()));
+    }
+
+    /** */
+    private static Predicate<Mailbox<?>> makeFilter(@Nullable UUID qryId, long fragmentId, long exchangeId) {
+        Predicate<Mailbox<?>> filter = ALWAYS_TRUE;
+        if (qryId != null)
+            filter = filter.and(mailbox -> Objects.equals(mailbox.queryId(), qryId));
+        if (fragmentId != -1)
+            filter = filter.and(mailbox -> mailbox.fragmentId() == fragmentId);
+        if (exchangeId != -1)
+            filter = filter.and(mailbox -> mailbox.exchangeId() == exchangeId);
+
+        return filter;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MailboxRegistryImpl.class, this);
+    }
+
+    /** */
+    private static class MailboxKey {
+        /** */
+        private final UUID qryId;
+
+        /** */
+        private final long exchangeId;
+
+        /** */
+        private MailboxKey(UUID qryId, long exchangeId) {
+            this.qryId = qryId;
+            this.exchangeId = exchangeId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            MailboxKey that = (MailboxKey) o;
+
+            if (exchangeId != that.exchangeId)
+                return false;
+            return qryId.equals(that.qryId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = qryId.hashCode();
+            res = 31 * res + (int) (exchangeId ^ (exchangeId >>> 32));
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(MailboxKey.class, this);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
index abcad0a..92f4546 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
@@ -30,7 +30,7 @@ public class QueryTaskExecutorImpl implements QueryTaskExecutor, Thread.Uncaught
     private static final IgniteLogger LOG = IgniteLogger.forClass(QueryTaskExecutorImpl.class);
 
     /** */
-    private StripedThreadPoolExecutor stripedThreadPoolExecutor;
+    private final StripedThreadPoolExecutor stripedThreadPoolExecutor;
 
     /** */
     private Thread.UncaughtExceptionHandler eHnd;
@@ -38,7 +38,7 @@ public class QueryTaskExecutorImpl implements QueryTaskExecutor, Thread.Uncaught
     /**
      * @param stripedThreadPoolExecutor Executor.
      */
-    public void stripedThreadPoolExecutor(StripedThreadPoolExecutor stripedThreadPoolExecutor) {
+    public QueryTaskExecutorImpl(StripedThreadPoolExecutor stripedThreadPoolExecutor) {
         this.stripedThreadPoolExecutor = stripedThreadPoolExecutor;
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
new file mode 100644
index 0000000..f25161c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
@@ -0,0 +1,519 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistry;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A part of exchange.
+ */
+public class Inbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, SingleNode<Row> {
+    /** */
+    private final ExchangeService exchange;
+
+    /** */
+    private final MailboxRegistry registry;
+
+    /** */
+    private final long exchangeId;
+
+    /** */
+    private final long srcFragmentId;
+
+    /** */
+    private final Map<String, Buffer> perNodeBuffers;
+
+    /** */
+    private volatile Collection<String> srcNodeIds;
+
+    /** */
+    private Comparator<Row> comp;
+
+    /** */
+    private List<Buffer> buffers;
+
+    /** */
+    private int requested;
+
+    /** */
+    private boolean inLoop;
+
+    /**
+     * @param ctx Execution context.
+     * @param exchange Exchange service.
+     * @param registry Mailbox registry.
+     * @param exchangeId Exchange ID.
+     * @param srcFragmentId Source fragment ID.
+     */
+    public Inbox(
+        ExecutionContext<Row> ctx,
+        ExchangeService exchange,
+        MailboxRegistry registry,
+        long exchangeId,
+        long srcFragmentId
+    ) {
+        super(ctx, ctx.getTypeFactory().createUnknownType());
+        this.exchange = exchange;
+        this.registry = registry;
+
+        this.srcFragmentId = srcFragmentId;
+        this.exchangeId = exchangeId;
+
+        perNodeBuffers = new HashMap<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * Inits this Inbox.
+     *
+     * @param ctx Execution context.
+     * @param srcNodeIds Source node IDs.
+     * @param comp Optional comparator for merge exchange.
+     */
+    public void init(
+        ExecutionContext<Row> ctx, RelDataType rowType, Collection<String> srcNodeIds, @Nullable Comparator<Row> comp) {
+        assert context().fragmentId() == ctx.fragmentId() : "different fragments unsupported: previous=" + context().fragmentId() +
+            " current=" + ctx.fragmentId();
+
+        // It's important to set proper context here because
+        // the one, that is created on a first message
+        // received doesn't have all context variables in place.
+        context(ctx);
+        rowType(rowType);
+
+        this.comp = comp;
+
+        // memory barier
+        this.srcNodeIds = new HashSet<>(srcNodeIds);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) throws Exception {
+        assert srcNodeIds != null;
+        assert rowsCnt > 0 && requested == 0;
+
+        checkState();
+
+        requested = rowsCnt;
+
+        if (!inLoop)
+            context().execute(this::doPush, this::onError);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeInternal() {
+        super.closeInternal();
+
+        registry.unregister(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(List<Node<Row>> sources) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void rewindInternal() {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Pushes a batch into a buffer.
+     *
+     * @param srcNodeId Source node id.
+     * @param batchId Batch ID.
+     * @param last Last batch flag.
+     * @param rows Rows.
+     */
+    public void onBatchReceived(String srcNodeId, int batchId, boolean last, List<Row> rows) throws Exception {
+        Buffer buf = getOrCreateBuffer(srcNodeId);
+
+        boolean waitingBefore = buf.check() == State.WAITING;
+
+        buf.offer(batchId, last, rows);
+
+        if (requested > 0 && waitingBefore && buf.check() != State.WAITING)
+            push();
+    }
+
+    /** */
+    private void doPush() throws Exception {
+        checkState();
+
+        push();
+    }
+
+    /** */
+    private void push() throws Exception {
+        if (buffers == null) {
+            for (String node : srcNodeIds)
+                checkNode(node);
+
+            buffers = srcNodeIds.stream()
+                .map(this::getOrCreateBuffer)
+                .collect(Collectors.toList());
+
+            assert buffers.size() == perNodeBuffers.size();
+        }
+
+        if (comp != null)
+            pushOrdered();
+        else
+            pushUnordered();
+    }
+
+    /** */
+    private void pushOrdered() throws Exception {
+         PriorityQueue<Pair<Row, Buffer>> heap =
+            new PriorityQueue<>(Math.max(buffers.size(), 1), Map.Entry.comparingByKey(comp));
+
+        Iterator<Buffer> it = buffers.iterator();
+
+        while (it.hasNext()) {
+            Buffer buf = it.next();
+
+            switch (buf.check()) {
+                case END:
+                    it.remove();
+
+                    break;
+                case READY:
+                    heap.offer(Pair.of(buf.peek(), buf));
+
+                    break;
+                case WAITING:
+
+                    return;
+            }
+        }
+
+        inLoop = true;
+        try {
+            while (requested > 0 && !heap.isEmpty()) {
+                checkState();
+
+                Buffer buf = heap.poll().right;
+
+                requested--;
+                downstream().push(buf.remove());
+
+                switch (buf.check()) {
+                    case END:
+                        buffers.remove(buf);
+
+                        break;
+                    case READY:
+                        heap.offer(Pair.of(buf.peek(), buf));
+
+                        break;
+                    case WAITING:
+
+                        return;
+                }
+            }
+        }
+        finally {
+            inLoop = false;
+        }
+
+        if (requested > 0 && heap.isEmpty()) {
+            assert buffers.isEmpty();
+
+            requested = 0;
+            downstream().end();
+        }
+    }
+
+    /** */
+    private void pushUnordered() throws Exception {
+        int idx = 0, noProgress = 0;
+
+        inLoop = true;
+        try {
+            while (requested > 0 && !buffers.isEmpty()) {
+                checkState();
+
+                Buffer buf = buffers.get(idx);
+
+                switch (buf.check()) {
+                    case END:
+                        buffers.remove(idx--);
+
+                        break;
+                    case READY:
+                        noProgress = 0;
+                        requested--;
+                        downstream().push(buf.remove());
+
+                        break;
+                    case WAITING:
+                        if (++noProgress >= buffers.size())
+                            return;
+
+                        break;
+                }
+
+                if (++idx == buffers.size())
+                    idx = 0;
+            }
+        }
+        finally {
+            inLoop = false;
+        }
+
+        if (requested > 0 && buffers.isEmpty()) {
+            requested = 0;
+            downstream().end();
+        }
+    }
+
+    /** */
+    private void acknowledge(String nodeId, int batchId) throws IgniteInternalCheckedException {
+        exchange.acknowledge(nodeId, queryId(), srcFragmentId, exchangeId, batchId);
+    }
+
+    /** */
+    private Buffer getOrCreateBuffer(String nodeId) {
+        return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+    }
+
+    /** */
+    private Buffer createBuffer(String nodeId) {
+        return new Buffer(nodeId);
+    }
+
+    /** */
+    public void onNodeLeft(String nodeId) {
+        if (context().originatingNodeId().equals(nodeId) && srcNodeIds == null)
+            context().execute(this::close, this::onError);
+        else if (srcNodeIds != null && srcNodeIds.contains(nodeId))
+            context().execute(() -> onNodeLeft0(nodeId), this::onError);
+    }
+
+    /** */
+    private void onNodeLeft0(String nodeId) throws Exception {
+        checkState();
+
+        if (getOrCreateBuffer(nodeId).check() != State.END)
+            throw new IgniteInternalCheckedException("Failed to execute query, node left [nodeId=" + nodeId + ']');
+    }
+
+    /** */
+    private void checkNode(String nodeId) throws IgniteInternalCheckedException {
+        if (!exchange.alive(nodeId))
+            throw new IgniteInternalCheckedException("Failed to execute query, node left [nodeId=" + nodeId + ']');
+    }
+
+    /** */
+    private static final class Batch<Row> implements Comparable<Batch<Row>> {
+        /** */
+        private final int batchId;
+
+        /** */
+        private final boolean last;
+
+        /** */
+        private final List<Row> rows;
+
+        /** */
+        private int idx;
+
+        /** */
+        private Batch(int batchId, boolean last, List<Row> rows) {
+            this.batchId = batchId;
+            this.last = last;
+            this.rows = rows;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Batch<Row> batch = (Batch<Row>) o;
+
+            return batchId == batch.batchId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return batchId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(@NotNull Inbox.Batch<Row> o) {
+            return Integer.compare(batchId, o.batchId);
+        }
+    }
+
+    /** */
+    private enum State {
+        /** */
+        END,
+
+        /** */
+        READY,
+
+        /** */
+        WAITING
+    }
+
+    /** */
+    private static final Batch<?> WAITING = new Batch<>(0, false, null);
+
+    /** */
+    private static final Batch<?> END = new Batch<>(0, false, null);
+
+    /** */
+    private final class Buffer {
+        /** */
+        private final String nodeId;
+
+        /** */
+        private int lastEnqueued = -1;
+
+        /** */
+        private final PriorityQueue<Batch<Row>> batches = new PriorityQueue<>(IO_BATCH_CNT);
+
+        /** */
+        private Batch<Row> curr = waitingMark();
+
+        /** */
+        private Buffer(String nodeId) {
+            this.nodeId = nodeId;
+        }
+
+        /** */
+        private void offer(int id, boolean last, List<Row> rows) {
+            batches.offer(new Batch<>(id, last, rows));
+        }
+
+        /** */
+        private Batch<Row> pollBatch() {
+            if (batches.isEmpty() || batches.peek().batchId != lastEnqueued + 1)
+                return waitingMark();
+
+            Batch<Row> batch = batches.poll();
+
+            assert batch != null && batch.batchId == lastEnqueued + 1;
+
+            lastEnqueued = batch.batchId;
+
+            return batch;
+        }
+
+        /** */
+        private State check() {
+            if (finished())
+                return State.END;
+
+            if (waiting())
+                return State.WAITING;
+
+            if (isEnd()) {
+                curr = finishedMark();
+
+                return State.END;
+            }
+
+            return State.READY;
+        }
+
+        /** */
+        private Row peek() {
+            assert curr != null;
+            assert curr != WAITING;
+            assert curr != END;
+            assert !isEnd();
+
+            return curr.rows.get(curr.idx);
+        }
+
+        /** */
+        private Row remove() throws IgniteInternalCheckedException {
+            assert curr != null;
+            assert curr != WAITING;
+            assert curr != END;
+            assert !isEnd();
+
+            Row row = curr.rows.set(curr.idx++, null);
+
+            if (curr.idx == curr.rows.size()) {
+                acknowledge(nodeId, curr.batchId);
+
+                if (!isEnd())
+                    curr = pollBatch();
+            }
+
+            return row;
+        }
+
+        /** */
+        private boolean finished() {
+            return curr == END;
+        }
+
+        /** */
+        private boolean waiting() {
+            return curr == WAITING && (curr = pollBatch()) == WAITING;
+        }
+
+        /** */
+        private boolean isEnd() {
+            return curr.last && curr.idx == curr.rows.size();
+        }
+
+        /** */
+        private Batch<Row> finishedMark() {
+            return (Batch<Row>) END;
+        }
+
+        /** */
+        private Batch<Row> waitingMark() {
+            return (Batch<Row>) WAITING;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
new file mode 100644
index 0000000..483c8a5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistry;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.IN_BUFFER_SIZE;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * A part of exchange.
+ */
+public class Outbox<Row> extends AbstractNode<Row> implements Mailbox<Row>, SingleNode<Row>, Downstream<Row> {
+    /** */
+    private final ExchangeService exchange;
+
+    /** */
+    private final MailboxRegistry registry;
+
+    /** */
+    private final long exchangeId;
+
+    /** */
+    private final long targetFragmentId;
+
+    /** */
+    private final Destination<Row> dest;
+
+    /** */
+    private final Deque<Row> inBuf = new ArrayDeque<>(inBufSize);
+
+    /** */
+    private final Map<String, Buffer> nodeBuffers = new HashMap<>();
+
+    /** */
+    private int waiting;
+
+    /**
+     * @param ctx Execution context.
+     * @param exchange Exchange service.
+     * @param registry Mailbox registry.
+     * @param exchangeId Exchange ID.
+     * @param targetFragmentId Target fragment ID.
+     * @param dest Destination.
+     */
+    public Outbox(
+        ExecutionContext<Row> ctx,
+        RelDataType rowType,
+        ExchangeService exchange,
+        MailboxRegistry registry,
+        long exchangeId,
+        long targetFragmentId,
+        Destination<Row> dest
+    ) {
+        super(ctx, rowType);
+        this.exchange = exchange;
+        this.registry = registry;
+        this.targetFragmentId = targetFragmentId;
+        this.exchangeId = exchangeId;
+        this.dest = dest;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * callback method.
+     *
+     * @param nodeId Target ID.
+     * @param batchId Batch ID.
+     */
+    public void onAcknowledge(String nodeId, int batchId) throws Exception {
+        assert nodeBuffers.containsKey(nodeId);
+
+        checkState();
+
+        nodeBuffers.get(nodeId).acknowledge(batchId);
+    }
+
+    /** */
+    public void init() {
+        try {
+            checkState();
+
+            flush();
+        }
+        catch (Throwable t) {
+            onError(t);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowCnt) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void push(Row row) throws Exception {
+        assert waiting > 0;
+
+        checkState();
+
+        waiting--;
+
+        inBuf.add(row);
+
+        flush();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void end() throws Exception {
+        assert waiting > 0;
+
+        checkState();
+
+        waiting = -1;
+
+        flush();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onErrorInternal(Throwable e) {
+        log.error("Error occurred during execution", e);
+
+        try {
+            sendError(e);
+        }
+        catch (IgniteInternalCheckedException ex) {
+            log.error("Error occurred during send error message", e);
+        }
+        finally {
+            Commons.closeQuiet(this);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeInternal() {
+        super.closeInternal();
+
+        registry.unregister(this);
+
+        // Send cancel message for the Inbox to close Inboxes created by batch message race.
+        for (String node : dest.targets())
+            getOrCreateBuffer(node).close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onRegister(Downstream<Row> downstream) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void rewindInternal() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    /** */
+    private void sendBatch(String nodeId, int batchId, boolean last, List<Row> rows) throws IgniteInternalCheckedException {
+        exchange.sendBatch(nodeId, queryId(), targetFragmentId, exchangeId, batchId, last, rows);
+    }
+
+    /** */
+    private void sendError(Throwable err) throws IgniteInternalCheckedException {
+        exchange.sendError(context().originatingNodeId(), queryId(), fragmentId(), err);
+    }
+
+    /** */
+    private void sendInboxClose(String nodeId) {
+        try {
+            exchange.closeInbox(nodeId, queryId(), targetFragmentId, exchangeId);
+        }
+        catch (IgniteInternalCheckedException e) {
+            log.error("Failed to send cancel message", e);
+        }
+    }
+
+    /** */
+    private Buffer getOrCreateBuffer(String nodeId) {
+        return nodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+    }
+
+    /** */
+    private Buffer createBuffer(String nodeId) {
+        return new Buffer(nodeId);
+    }
+
+    /** */
+    private void flush() throws Exception {
+        while (!inBuf.isEmpty()) {
+            checkState();
+
+            Collection<Buffer> buffers = dest.targets(inBuf.peek()).stream()
+                .map(this::getOrCreateBuffer)
+                .collect(Collectors.toList());
+
+            assert !nullOrEmpty(buffers);
+
+            if (!buffers.stream().allMatch(Buffer::ready))
+                return;
+
+            Row row = inBuf.remove();
+
+            for (Buffer dest : buffers)
+                dest.add(row);
+        }
+
+        assert inBuf.isEmpty();
+
+        if (waiting == 0)
+            source().request(waiting = IN_BUFFER_SIZE);
+        else if (waiting == -1) {
+            for (String node : dest.targets())
+                getOrCreateBuffer(node).end();
+        }
+    }
+
+    /** */
+    public void onNodeLeft(String nodeId) {
+        if (nodeId.equals(context().originatingNodeId()))
+            context().execute(this::close, this::onError);
+    }
+
+    /** */
+    private final class Buffer {
+        /** */
+        private final String nodeId;
+
+        /** */
+        private int hwm = -1;
+
+        /** */
+        private int lwm = -1;
+
+        /** */
+        private List<Row> curr;
+
+        /** */
+        private Buffer(String nodeId) {
+            this.nodeId = nodeId;
+
+            curr = new ArrayList<>(IO_BATCH_SIZE);
+        }
+
+        /**
+         * Checks whether there is a place for a new row.
+         *
+         * @return {@code True} is it possible to add a row to a batch.
+         */
+        private boolean ready() {
+            if (hwm == Integer.MAX_VALUE)
+                return false;
+
+            return curr.size() < IO_BATCH_SIZE || hwm - lwm < IO_BATCH_CNT;
+        }
+
+        /**
+         * Adds a row to current batch.
+         *
+         * @param row Row.
+         */
+        public void add(Row row) throws IgniteInternalCheckedException {
+            assert ready();
+
+            if (curr.size() == IO_BATCH_SIZE) {
+                sendBatch(nodeId, ++hwm, false, curr);
+
+                curr = new ArrayList<>(IO_BATCH_SIZE);
+            }
+
+            curr.add(row);
+        }
+
+        /**
+         * Signals data is over.
+         */
+        public void end() throws IgniteInternalCheckedException {
+            if (hwm == Integer.MAX_VALUE)
+                return;
+
+            int batchId = hwm + 1;
+            hwm = Integer.MAX_VALUE;
+
+            List<Row> tmp = curr;
+            curr = null;
+
+            sendBatch(nodeId, batchId, true, tmp);
+        }
+
+        /**
+         * Callback method.
+         *
+         * @param id batch ID.
+         */
+        private void acknowledge(int id) throws Exception {
+            if (lwm > id)
+                return;
+
+            boolean readyBefore = ready();
+
+            lwm = id;
+
+            if (!readyBefore && ready())
+                flush();
+        }
+
+        /** */
+        public void close() {
+            int currBatchId = hwm;
+
+            if (hwm == Integer.MAX_VALUE)
+                return;
+
+            hwm = Integer.MAX_VALUE;
+
+            curr = null;
+
+            if (currBatchId >= 0)
+                sendInboxClose(nodeId);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
index b4417d9..2c39021 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
@@ -22,7 +22,6 @@ import java.util.Deque;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.UUID;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
@@ -121,20 +120,7 @@ public class RootNode<Row> extends AbstractNode<Row> implements SingleNode<Row>,
 
     /** {@inheritDoc} */
     @Override public void closeInternal() {
-        closeInternal0(ex.get() == null);
-    }
-
-    /** */
-    private void closeInternal0(boolean sync) {
-        try {
-            if (sync)
-                context().submit(() -> sources().forEach(Commons::closeQuiet), this::onError).get();
-            else
-                context().execute(() -> sources().forEach(Commons::closeQuiet), this::onError);
-        }
-        catch (InterruptedException | ExecutionException e) {
-            log.warn("Execution is cancelled.", e);
-        }
+        context().execute(() -> sources().forEach(Commons::closeQuiet), this::onError);
     }
 
     /** {@inheritDoc} */
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/message/ErrorMessage.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ErrorMessage.java
index 22da20a..edf4048 100644
--- 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/message/ErrorMessage.java
@@ -15,24 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.io.Serializable;
+import java.util.UUID;
+
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
 
 /**
- * Ignite distribution trait.
+ *
  */
-public interface IgniteDistribution extends RelDistribution {
+@Transferable(value = SqlQueryMessageGroup.ERROR_MESSAGE, autoSerializable = false)
+public interface ErrorMessage extends NetworkMessage, Serializable {
     /**
-     * @return Distribution function.
+     * @return Query ID.
      */
-    DistributionFunction function();
+    UUID queryId();
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    /**
+     * @return Fragment ID.
+     */
+    long fragmentId();
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    /**
+     * @return Error.
+     */
+    Throwable error();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAwareMessage.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAwareMessage.java
index 093146c..05b6cfe 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAwareMessage.java
@@ -14,21 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import java.io.Serializable;
+import java.util.UUID;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Execution context is used to determine a stripe where to process a message.
+ */
+public interface ExecutionContextAwareMessage extends NetworkMessage, Serializable {
+    /**
+     * @return Query ID.
+     */
+    UUID queryId();
+
+    /**
+     * @return Fragment ID.
+     */
+    long fragmentId();
 }
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/message/InboxCloseMessage.java
similarity index 60%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCloseMessage.java
index 22da20a..b108a7d 100644
--- 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/message/InboxCloseMessage.java
@@ -15,24 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.io.Serializable;
+import java.util.UUID;
+
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
 
 /**
- * Ignite distribution trait.
+ *
  */
-public interface IgniteDistribution extends RelDistribution {
+@Transferable(value = SqlQueryMessageGroup.INBOX_CLOSE_MESSAGE)
+public interface InboxCloseMessage extends NetworkMessage, Serializable {
     /**
-     * @return Distribution function.
+     * @return Query ID.
      */
-    DistributionFunction function();
+    UUID queryId();
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    /**
+     * @return Fragment ID.
+     */
+    long fragmentId();
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    /**
+     * @return Exchange ID.
+     */
+    long exchangeId();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
index 093146c..4c334f0 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
@@ -14,21 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import org.apache.ignite.network.NetworkMessage;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ *
+ */
+public interface MessageListener {
+    /**
+     * @param nodeId Sender node ID.
+     * @param msg Message.
+     */
+    void onMessage(String nodeId, NetworkMessage msg);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
similarity index 52%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
index 6337018..cdf24e9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
@@ -15,40 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.prepare;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.network.NetworkMessage;
 
 /**
- * Regular query or DML
+ *
  */
-public interface MultiStepPlan extends QueryPlan {
+public interface MessageService {
     /**
-     * @return Query fragments.
+     * Sends a message to given node.
+     *
+     * @param nodeId Node ID.
+     * @param msg Message.
      */
-    List<Fragment> fragments();
+    void send(String nodeId, NetworkMessage msg) throws IgniteInternalCheckedException;
 
     /**
-     * @param fragment Fragment.
-     * @return Mapping for a given fragment.
+     * Checks whether a node with given ID is alive.
+     *
+     * @param nodeId Node ID.
+     * @return {@code True} if node is alive.
      */
-    FragmentMapping mapping(Fragment fragment);
-
-    /** */
-    ColocationGroup target(Fragment fragment);
-
-    /** */
-    Map<Long, List<UUID>> remotes(Fragment fragment);
+    boolean alive(String nodeId);
 
     /**
-     * Inits query fragments.
+     * Registers a listener for messages of a given type.
      *
-     * @param ctx Planner context.
+     * @param lsnr Listener.
+     * @param msgId Message id.
      */
-    void init(PlanningContext ctx);
+    void register(MessageListener lsnr, short msgId);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java
new file mode 100644
index 0000000..8b95faa
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.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.message;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ *
+ */
+public class MessageServiceImpl implements MessageService {
+    private static final UUID QUERY_ID_STUB = UUID.randomUUID();
+
+    private static final IgniteLogger LOG = IgniteLogger.forClass(MessageServiceImpl.class);
+
+    private final TopologyService topSrvc;
+
+    private final MessagingService messagingSrvc;
+
+    /** */
+    private final String locNodeId;
+
+    /** */
+    private final QueryTaskExecutor taskExecutor;
+
+    /** */
+    private Map<Short, MessageListener> lsnrs;
+
+    /** */
+    public MessageServiceImpl(
+        TopologyService topSrvc,
+        MessagingService messagingSrvc,
+        QueryTaskExecutor taskExecutor
+    ) {
+        this.topSrvc = topSrvc;
+        this.messagingSrvc = messagingSrvc;
+        this.taskExecutor = taskExecutor;
+
+        locNodeId = topSrvc.localMember().id();
+
+        messagingSrvc.addMessageHandler(this::onMessage);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void send(String nodeId, NetworkMessage msg) throws IgniteInternalCheckedException {
+        if (locNodeId.equals(nodeId))
+            onMessage(nodeId, msg);
+        else {
+            ClusterNode node = topSrvc.allMembers().stream()
+                .filter(cn -> nodeId.equals(cn.id()))
+                .findFirst()
+                .orElseThrow(() -> new IgniteInternalException("Failed to send message to node (has node left grid?): " + nodeId));
+
+            try {
+                messagingSrvc.send(node, msg).get();
+            } catch (Exception ex) {
+                if (ex instanceof IgniteInternalCheckedException)
+                    throw (IgniteInternalCheckedException)ex;
+
+                throw new IgniteInternalCheckedException(ex);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(MessageListener lsnr, short type) {
+        if (lsnrs == null)
+            lsnrs = new HashMap<>();
+
+        MessageListener old = lsnrs.put(type, lsnr);
+
+        assert old == null : old;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean alive(String nodeId) {
+        return topSrvc.allMembers().stream()
+            .map(ClusterNode::id)
+            .anyMatch(id -> id.equals(nodeId));
+    }
+
+    /** */
+    protected void onMessage(String nodeId, NetworkMessage msg) {
+        if (msg instanceof ExecutionContextAwareMessage) {
+            ExecutionContextAwareMessage msg0 = (ExecutionContextAwareMessage) msg;
+            taskExecutor.execute(msg0.queryId(), msg0.fragmentId(), () -> onMessageInternal(nodeId, msg));
+        }
+        else {
+            taskExecutor.execute(
+                QUERY_ID_STUB,
+                ThreadLocalRandom.current().nextLong(1024),
+                () -> onMessageInternal(nodeId, msg)
+            );
+        }
+    }
+
+    /** */
+    private void onMessage(NetworkMessage msg, NetworkAddress addr, String correlationId) {
+        ClusterNode node = topSrvc.getByAddress(addr);
+
+        if (node == null) {
+            LOG.warn("Received a message from a node that has not yet" +
+                    " joined the cluster: addr={}, msg={}", addr, msg);
+
+            return;
+        }
+
+        if (msg.groupType() == 3)
+            onMessage(node.id(), msg);
+    }
+
+    /** */
+    private void onMessageInternal(String nodeId, NetworkMessage msg) {
+        MessageListener lsnr = Objects.requireNonNull(
+                lsnrs.get(msg.messageType()),
+                "there is no listener for msgType=" + msg.groupType()
+        );
+
+        lsnr.onMessage(nodeId, msg);
+    }
+}
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/message/OutboxCloseMessage.java
similarity index 60%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/OutboxCloseMessage.java
index 22da20a..debf881 100644
--- 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/message/OutboxCloseMessage.java
@@ -15,24 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.io.Serializable;
+import java.util.UUID;
+
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
 
 /**
- * Ignite distribution trait.
+ *
  */
-public interface IgniteDistribution extends RelDistribution {
+@Transferable(value = SqlQueryMessageGroup.OUTBOX_CLOSE_MESSAGE)
+public interface OutboxCloseMessage extends NetworkMessage, Serializable {
     /**
-     * @return Distribution function.
+     * @return Query ID.
      */
-    DistributionFunction function();
+    UUID queryId();
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    /**
+     * @return Fragment ID.
+     */
+    long fragmentId();
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    /**
+     * @return Exchange ID.
+     */
+    long exchangeId();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
index 093146c..54dd27c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
@@ -14,21 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import org.apache.ignite.network.annotations.Transferable;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ *
+ */
+@Transferable(value = SqlQueryMessageGroup.QUERY_BATCH_ACK)
+public interface QueryBatchAcknowledgeMessage extends ExecutionContextAwareMessage {
+    /**
+     * @return Exchange ID.
+     */
+    long exchangeId();
+
+    /**
+     * @return Batch ID.
+     */
+    int batchId();
 }
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/message/QueryBatchMessage.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java
index 22da20a..f7e70bb 100644
--- 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/message/QueryBatchMessage.java
@@ -15,24 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.util.List;
+
+import org.apache.ignite.network.annotations.Transferable;
 
 /**
- * Ignite distribution trait.
+ *
  */
-public interface IgniteDistribution extends RelDistribution {
+@Transferable(value = SqlQueryMessageGroup.QUERY_BATCH_MESSAGE, autoSerializable = false)
+public interface QueryBatchMessage extends ExecutionContextAwareMessage {
     /**
-     * @return Distribution function.
+     * @return Exchange ID.
      */
-    DistributionFunction function();
+    long exchangeId();
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    /**
+     * @return Batch ID.
+     */
+    int batchId();
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    /**
+     * @return Last batch flag.
+     */
+    boolean last();
+
+    /**
+     * @return Rows.
+     */
+    List<Object> rows();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
similarity index 55%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
index 6337018..bb6b710 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
@@ -15,40 +15,38 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.prepare;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
+import org.apache.ignite.network.annotations.Transferable;
 
 /**
- * Regular query or DML
+ *
  */
-public interface MultiStepPlan extends QueryPlan {
+@Transferable(value = SqlQueryMessageGroup.QUERY_START_REQUEST, autoSerializable = false)
+public interface QueryStartRequest extends ExecutionContextAwareMessage {
     /**
-     * @return Query fragments.
+     * @return Schema name.
      */
-    List<Fragment> fragments();
+    String schema();
 
     /**
-     * @param fragment Fragment.
-     * @return Mapping for a given fragment.
+     * @return Fragment description.
      */
-    FragmentMapping mapping(Fragment fragment);
+    FragmentDescription fragmentDescription();
 
-    /** */
-    ColocationGroup target(Fragment fragment);
+    /**
+     * @return Topology version.
+     */
+    long topologyVersion();
 
-    /** */
-    Map<Long, List<UUID>> remotes(Fragment fragment);
+    /**
+     * @return Fragment plan.
+     */
+    String root();
 
     /**
-     * Inits query fragments.
-     *
-     * @param ctx Planner context.
+     * @return Query parameters.
      */
-    void init(PlanningContext ctx);
+    Object[] parameters();
 }
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/message/QueryStartResponse.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java
index 22da20a..f6d2f9b 100644
--- 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/message/QueryStartResponse.java
@@ -15,24 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.io.Serializable;
+import java.util.UUID;
+
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.annotations.Transferable;
 
 /**
- * Ignite distribution trait.
+ *
  */
-public interface IgniteDistribution extends RelDistribution {
+@Transferable(value = SqlQueryMessageGroup.QUERY_START_RESPONSE, autoSerializable = false)
+public interface QueryStartResponse extends NetworkMessage, Serializable {
     /**
-     * @return Distribution function.
+     * @return Query ID.
      */
-    DistributionFunction function();
+    UUID queryId();
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    /**
+     * @return Fragment ID.
+     */
+    long fragmentId();
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    /**
+     * @return Error.
+     */
+    Throwable error();
 }
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/message/SqlQueryMessageGroup.java
similarity index 55%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/SqlQueryMessageGroup.java
index 22da20a..e8cbc84 100644
--- 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/message/SqlQueryMessageGroup.java
@@ -15,24 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import org.apache.ignite.network.annotations.MessageGroup;
 
 /**
- * Ignite distribution trait.
+ * Message types for the sql query processing module.
  */
-public interface IgniteDistribution extends RelDistribution {
-    /**
-     * @return Distribution function.
-     */
-    DistributionFunction function();
+@MessageGroup(groupType = 3, groupName = "SqlQueryMessages")
+public final class SqlQueryMessageGroup {
+    public static final short QUERY_START_REQUEST = 0;
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    public static final short QUERY_START_RESPONSE = 1;
 
-    /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    public static final short ERROR_MESSAGE = 2;
+
+    public static final short QUERY_BATCH_MESSAGE = 3;
+
+    public static final short QUERY_BATCH_ACK = 4;
+
+    public static final short INBOX_CLOSE_MESSAGE = 5;
+
+    public static final short OUTBOX_CLOSE_MESSAGE = 6;
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/AffinityService.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/AffinityService.java
index 093146c..dd78685 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/AffinityService.java
@@ -14,21 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.metadata;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import java.util.function.ToIntFunction;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ *
+ */
+public interface AffinityService {
+    /**
+     * Creates a partition mapping function on the basis of affinity function of cache with given ID.
+     *
+     * @param cacheId Cache ID.
+     * @return Affinity function.
+     */
+    ToIntFunction<Object> affinity(int cacheId);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java
index fa6b1f8..743bb58 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/ColocationGroup.java
@@ -17,13 +17,13 @@
 
 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.Set;
-import java.util.UUID;
 
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
@@ -36,7 +36,7 @@ import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 import static org.apache.ignite.internal.util.IgniteUtils.firstNotNull;
 
 /** */
-public class ColocationGroup {
+public class ColocationGroup implements Serializable {
     /** */
     private static final int SYNTHETIC_PARTITIONS_COUNT = 512;
         // TODO: IgniteSystemProperties.getInteger("IGNITE_CALCITE_SYNTHETIC_PARTITIONS_COUNT", 512);
@@ -45,18 +45,18 @@ public class ColocationGroup {
     private List<Long> sourceIds;
 
     /** */
-    private List<UUID> nodeIds;
+    private List<String> nodeIds;
 
     /** */
-    private List<List<UUID>> assignments;
+    private List<List<String>> assignments;
 
     /** */
-    public static ColocationGroup forNodes(List<UUID> nodeIds) {
+    public static ColocationGroup forNodes(List<String> nodeIds) {
         return new ColocationGroup(null, nodeIds, null);
     }
 
     /** */
-    public static ColocationGroup forAssignments(List<List<UUID>> assignments) {
+    public static ColocationGroup forAssignments(List<List<String>> assignments) {
         return new ColocationGroup(null, null, assignments);
     }
 
@@ -66,7 +66,7 @@ public class ColocationGroup {
     }
 
     /** */
-    private ColocationGroup(List<Long> sourceIds, List<UUID> nodeIds, List<List<UUID>> assignments) {
+    private ColocationGroup(List<Long> sourceIds, List<String> nodeIds, List<List<String>> assignments) {
         this.sourceIds = sourceIds;
         this.nodeIds = nodeIds;
         this.assignments = assignments;
@@ -82,7 +82,7 @@ public class ColocationGroup {
     /**
      * @return Lists of nodes capable to execute a query fragment for what the mapping is calculated.
      */
-    public List<UUID> nodeIds() {
+    public List<String> nodeIds() {
         return nodeIds == null ? Collections.emptyList() : nodeIds;
     }
 
@@ -90,7 +90,7 @@ public class ColocationGroup {
      * @return List of partitions (index) and nodes (items) having an appropriate partition in
      * OWNING state, calculated for distributed tables, involved in query execution.
      */
-    public List<List<UUID>> assignments() {
+    public List<List<String>> assignments() {
         return assignments == null ? Collections.emptyList() : assignments;
     }
 
@@ -123,7 +123,7 @@ public class ColocationGroup {
         else
             sourceIds = Commons.combine(this.sourceIds, other.sourceIds);
 
-        List<UUID> nodeIds;
+        List<String> nodeIds;
         if (this.nodeIds == null || other.nodeIds == null)
             nodeIds = firstNotNull(this.nodeIds, other.nodeIds);
         else
@@ -134,16 +134,16 @@ public class ColocationGroup {
                 "Replicated query parts are not co-located on all nodes");
         }
 
-        List<List<UUID>> assignments;
+        List<List<String>> assignments;
         if (this.assignments == null || other.assignments == null) {
             assignments = firstNotNull(this.assignments, other.assignments);
 
             if (assignments != null && nodeIds != null) {
-                Set<UUID> filter = new HashSet<>(nodeIds);
-                List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
+                Set<String> filter = new HashSet<>(nodeIds);
+                List<List<String>> assignments0 = new ArrayList<>(assignments.size());
 
                 for (int i = 0; i < assignments.size(); i++) {
-                    List<UUID> assignment = Commons.intersect(filter, assignments.get(i));
+                    List<String> assignment = Commons.intersect(filter, assignments.get(i));
 
                     if (assignment.isEmpty()) { // TODO check with partition filters
                         throw new ColocationMappingException("Failed to map fragment to location. " +
@@ -159,9 +159,9 @@ public class ColocationGroup {
         else {
             assert this.assignments.size() == other.assignments.size();
             assignments = new ArrayList<>(this.assignments.size());
-            Set<UUID> filter = nodeIds == null ? null : new HashSet<>(nodeIds);
+            Set<String> filter = nodeIds == null ? null : new HashSet<>(nodeIds);
             for (int i = 0; i < this.assignments.size(); i++) {
-                List<UUID> assignment = Commons.intersect(this.assignments.get(i), other.assignments.get(i));
+                List<String> assignment = Commons.intersect(this.assignments.get(i), other.assignments.get(i));
 
                 if (filter != null)
                     assignment.retainAll(filter);
@@ -182,10 +182,10 @@ public class ColocationGroup {
             return this;
 
         if (assignments != null) {
-            List<List<UUID>> assignments = new ArrayList<>(this.assignments.size());
-            Set<UUID> nodes = new HashSet<>();
-            for (List<UUID> assignment : this.assignments) {
-                UUID first = first(assignment);
+            List<List<String>> assignments = new ArrayList<>(this.assignments.size());
+            Set<String> nodes = new HashSet<>();
+            for (List<String> assignment : this.assignments) {
+                String first = first(assignment);
                 if (first != null)
                     nodes.add(first);
                 assignments.add(first != null ? Collections.singletonList(first) : Collections.emptyList());
@@ -198,13 +198,13 @@ public class ColocationGroup {
     }
 
     /** */
-    public ColocationGroup mapToNodes(List<UUID> nodeIds) {
+    public ColocationGroup mapToNodes(List<String> nodeIds) {
         return !nullOrEmpty(this.nodeIds) ? this : forNodes0(nodeIds);
     }
 
     /** */
-    @NotNull private ColocationGroup forNodes0(List<UUID> nodeIds) {
-        List<List<UUID>> assignments = new ArrayList<>(SYNTHETIC_PARTITIONS_COUNT);
+    @NotNull private ColocationGroup forNodes0(List<String> nodeIds) {
+        List<List<String>> assignments = new ArrayList<>(SYNTHETIC_PARTITIONS_COUNT);
         for (int i = 0; i < SYNTHETIC_PARTITIONS_COUNT; i++)
             assignments.add(asList(nodeIds.get(i % nodeIds.size())));
         return new ColocationGroup(sourceIds, nodeIds, assignments);
@@ -216,11 +216,11 @@ public class ColocationGroup {
      * @param nodeId Cluster node ID.
      * @return List of partitions to scan on the given node.
      */
-    public int[] partitions(UUID nodeId) {
+    public int[] partitions(String nodeId) {
         IgniteIntList parts = new IgniteIntList(assignments.size());
 
         for (int i = 0; i < assignments.size(); i++) {
-            List<UUID> assignment = assignments.get(i);
+            List<String> assignment = assignments.get(i);
             if (Objects.equals(nodeId, first(assignment)))
                 parts.add(i);
         }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
index f5ede72..cc1dc43 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
@@ -17,12 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
+import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 
 /** */
-public class FragmentDescription {
+public class FragmentDescription implements Serializable {
     /** */
     private long fragmentId;
 
@@ -33,7 +33,7 @@ public class FragmentDescription {
     private ColocationGroup target;
 
     /** */
-    private Map<Long, List<UUID>> remoteSources;
+    private Map<Long, List<String>> remoteSources;
 
     /** */
     public FragmentDescription() {
@@ -41,7 +41,7 @@ public class FragmentDescription {
 
     /** */
     public FragmentDescription(long fragmentId, FragmentMapping mapping, ColocationGroup target,
-        Map<Long, List<UUID>> remoteSources) {
+        Map<Long, List<String>> remoteSources) {
         this.fragmentId = fragmentId;
         this.mapping = mapping;
         this.target = target;
@@ -54,7 +54,7 @@ public class FragmentDescription {
     }
 
     /** */
-    public List<UUID> nodeIds() {
+    public List<String> nodeIds() {
         return mapping.nodeIds();
     }
 
@@ -64,7 +64,7 @@ public class FragmentDescription {
     }
 
     /** */
-    public Map<Long, List<UUID>> remotes() {
+    public Map<Long, List<String>> remotes() {
         return remoteSources;
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java
index 5d8f2ad..0d3ae61 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentMapping.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
+import java.io.Serializable;
 import java.util.Collections;
 import java.util.List;
-import java.util.UUID;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
@@ -34,7 +34,7 @@ import static org.apache.ignite.internal.util.IgniteUtils.firstNotNull;
 /**
  *
  */
-public class FragmentMapping {
+public class FragmentMapping implements Serializable {
     /** */
     private List<ColocationGroup> colocationGroups;
 
@@ -58,7 +58,7 @@ public class FragmentMapping {
     }
 
     /** */
-    public static FragmentMapping create(UUID nodeId) {
+    public static FragmentMapping create(String nodeId) {
         return new FragmentMapping(ColocationGroup.forNodes(Collections.singletonList(nodeId)));
     }
 
@@ -111,21 +111,21 @@ public class FragmentMapping {
     }
 
     /** */
-    public List<UUID> nodeIds() {
+    public List<String> nodeIds() {
         return colocationGroups.stream()
             .flatMap(g -> g.nodeIds().stream())
             .distinct().collect(Collectors.toList());
     }
 
     /** */
-    public FragmentMapping finalize(Supplier<List<UUID>> nodesSource) {
+    public FragmentMapping finalize(Supplier<List<String>> nodesSource) {
         if (colocationGroups.isEmpty())
             return this;
 
         List<ColocationGroup> colocationGroups = this.colocationGroups;
 
         colocationGroups = Commons.transform(colocationGroups, ColocationGroup::finalaze);
-        List<UUID> nodes = nodeIds(), nodes0 = nodes.isEmpty() ? nodesSource.get() : nodes;
+        List<String> nodes = nodeIds(), nodes0 = nodes.isEmpty() ? nodesSource.get() : nodes;
         colocationGroups = Commons.transform(colocationGroups, g -> g.mapToNodes(nodes0));
 
         return new FragmentMapping(colocationGroups);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
index 06d60a4..9882aa1 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
@@ -32,7 +32,7 @@ import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSortedIndexSpool;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMinusBase;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.calcite.util.NumberUtil.multiply;
@@ -118,9 +118,9 @@ public class IgniteMdRowCount extends RelMdRowCount {
     }
 
     /**
-     * Estimation of row count for MINUS (EXCEPT) operator.
+     * Estimation of row count for set op (MINUS, INTERSECT).
      */
-    public double getRowCount(IgniteMinusBase rel, RelMetadataQuery mq) {
+    public double getRowCount(IgniteSetOp rel, RelMetadataQuery mq) {
         return rel.estimateRowCount(mq);
     }
 
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
index e59dbfe..85fb1d5 100644
--- 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.function.Predicate;
 
 import org.apache.ignite.network.ClusterNode;
@@ -37,5 +36,5 @@ public interface MappingService {
      * @param nodeFilter Node filter.
      * @return Nodes mapping for intermediate fragments.
      */
-    List<UUID> executionNodes(long topVer, boolean single, @Nullable Predicate<ClusterNode> nodeFilter);
+    List<String> executionNodes(long topVer, boolean single, @Nullable Predicate<ClusterNode> nodeFilter);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java
new file mode 100644
index 0000000..eb9344d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.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.metadata;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.Collections.singletonList;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class MappingServiceImpl implements MappingService {
+    /** */
+    private final TopologyService topSrvc;
+
+    public MappingServiceImpl(TopologyService topSrvc) {
+        this.topSrvc = topSrvc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<String> executionNodes(long topVer, boolean single, @Nullable Predicate<ClusterNode> nodeFilter) {
+        List<ClusterNode> nodes = new ArrayList<>(topSrvc.allMembers());
+
+        if (nodeFilter != null)
+            nodes = nodes.stream().filter(nodeFilter).collect(Collectors.toList());
+
+        if (single && nodes.size() > 1)
+            nodes = singletonList(nodes.get(ThreadLocalRandom.current().nextInt(nodes.size())));
+
+        if (nullOrEmpty(nodes))
+            throw new IllegalStateException("failed to map query to execution nodes. Nodes list is empty.");
+
+        return Commons.transform(nodes, ClusterNode::id);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RemoteException.java
similarity index 54%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RemoteException.java
index f5ede72..a8810bb 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentDescription.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RemoteException.java
@@ -17,59 +17,52 @@
 
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
-import java.util.List;
-import java.util.Map;
 import java.util.UUID;
 
-/** */
-public class FragmentDescription {
-    /** */
-    private long fragmentId;
-
+/**
+ *
+ */
+public class RemoteException extends RuntimeException {
     /** */
-    private FragmentMapping mapping;
+    private final String nodeId;
 
     /** */
-    private ColocationGroup target;
+    private final UUID queryId;
 
     /** */
-    private Map<Long, List<UUID>> remoteSources;
+    private final long fragmentId;
 
-    /** */
-    public FragmentDescription() {
-    }
-
-    /** */
-    public FragmentDescription(long fragmentId, FragmentMapping mapping, ColocationGroup target,
-        Map<Long, List<UUID>> remoteSources) {
+    /**
+     * @param cause Cause.
+     * @param nodeId Node ID.
+     * @param queryId Query ID.
+     * @param fragmentId Fragment ID.
+     */
+    public RemoteException(String nodeId, UUID queryId, long fragmentId, Throwable cause) {
+        super("Remote query execution", cause);
+        this.nodeId = nodeId;
+        this.queryId = queryId;
         this.fragmentId = fragmentId;
-        this.mapping = mapping;
-        this.target = target;
-        this.remoteSources = remoteSources;
     }
 
-    /** */
-    public long fragmentId() {
-        return fragmentId;
+    /**
+     * @return Node ID.
+     */
+    public String nodeId() {
+        return nodeId;
     }
 
-    /** */
-    public List<UUID> nodeIds() {
-        return mapping.nodeIds();
-    }
-
-    /** */
-    public ColocationGroup target() {
-        return target;
-    }
-
-    /** */
-    public Map<Long, List<UUID>> remotes() {
-        return remoteSources;
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return queryId;
     }
 
-    /** */
-    public FragmentMapping mapping() {
-        return mapping;
+    /**
+     * @return Fragment ID.
+     */
+    public long fragmentId() {
+        return fragmentId;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java
index 2bfba75..6f9f1b2 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/AbstractMultiStepPlan.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.UUID;
 
 import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
@@ -36,7 +35,7 @@ import static org.apache.ignite.internal.util.IgniteUtils.newHashMap;
  */
 public abstract class AbstractMultiStepPlan implements MultiStepPlan {
     /** */
-    protected final Object fieldsMetadata;
+    protected final FieldsMetadata fieldsMetadata;
 
     /** */
     protected final QueryTemplate queryTemplate;
@@ -45,7 +44,7 @@ public abstract class AbstractMultiStepPlan implements MultiStepPlan {
     protected ExecutionPlan executionPlan;
 
     /** */
-    protected AbstractMultiStepPlan(QueryTemplate queryTemplate, Object fieldsMetadata) {
+    protected AbstractMultiStepPlan(QueryTemplate queryTemplate, FieldsMetadata fieldsMetadata) {
         this.queryTemplate = queryTemplate;
         this.fieldsMetadata = fieldsMetadata;
     }
@@ -56,6 +55,11 @@ public abstract class AbstractMultiStepPlan implements MultiStepPlan {
     }
 
     /** {@inheritDoc} */
+    @Override public FieldsMetadata fieldsMetadata() {
+        return fieldsMetadata;
+    }
+
+    /** {@inheritDoc} */
     @Override public FragmentMapping mapping(Fragment fragment) {
         return mapping(fragment.fragmentId());
     }
@@ -70,13 +74,13 @@ public abstract class AbstractMultiStepPlan implements MultiStepPlan {
     }
 
     /** {@inheritDoc} */
-    @Override public Map<Long, List<UUID>> remotes(Fragment fragment) {
+    @Override public Map<Long, List<String>> remotes(Fragment fragment) {
         List<IgniteReceiver> remotes = fragment.remotes();
 
         if (nullOrEmpty(remotes))
             return null;
 
-        HashMap<Long, List<UUID>> res = newHashMap(remotes.size());
+        HashMap<Long, List<String>> res = newHashMap(remotes.size());
 
         for (IgniteReceiver remote : remotes)
             res.put(remote.exchangeId(), mapping(remote.sourceFragmentId()).nodeIds());
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
new file mode 100644
index 0000000..a1c5bf9
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
@@ -0,0 +1,78 @@
+/*
+ * 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.Objects;
+
+/**
+ *
+ */
+public class CacheKey {
+    /** */
+    private final String schemaName;
+
+    /** */
+    private final String query;
+
+    /** */
+    private final Object contextKey;
+
+    /**
+     * @param schemaName Schema name.
+     * @param query Query string.
+     * @param contextKey Optional context key to differ queries with and without/different flags, having an impact
+     *                   on result plan (like LOCAL flag)
+     */
+    public CacheKey(String schemaName, String query, Object contextKey) {
+        this.schemaName = schemaName;
+        this.query = query;
+        this.contextKey = contextKey;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param query Query string.
+     */
+    public CacheKey(String schemaName, String query) {
+        this(schemaName, query, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CacheKey cacheKey = (CacheKey) o;
+
+        if (!schemaName.equals(cacheKey.schemaName))
+            return false;
+        if (!query.equals(cacheKey.query))
+            return false;
+        return Objects.equals(contextKey, cacheKey.contextKey);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName.hashCode();
+        result = 31 * result + query.hashCode();
+        result = 31 * result + (contextKey != null ? contextKey.hashCode() : 0);
+        return result;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java
index eda9fd9..af89e28 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Cloner.java
@@ -47,9 +47,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceH
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 import static org.apache.ignite.internal.util.ArrayUtils.asList;
@@ -231,17 +229,7 @@ public class Cloner implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteSingleMinus rel) {
-        return rel.clone(cluster, Commons.transform(rel.getInputs(), rel0 -> visit((IgniteRel) rel0)));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteMapMinus rel) {
-        return rel.clone(cluster, Commons.transform(rel.getInputs(), rel0 -> visit((IgniteRel) rel0)));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteReduceMinus rel) {
+    @Override public IgniteRel visit(IgniteSetOp rel) {
         return rel.clone(cluster, Commons.transform(rel.getInputs(), rel0 -> visit((IgniteRel) rel0)));
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DdlPlan.java
similarity index 73%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DdlPlan.java
index 9ef2cfb..79d3880 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DdlPlan.java
@@ -17,24 +17,30 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-/**
- * Distributed query plan.
- */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
-    /**
-     * @param fieldsMeta Fields metadata.
-     */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
+import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+
+/** */
+public class DdlPlan implements QueryPlan {
+    /** */
+    private final DdlCommand cmd;
+
+    /** */
+    public DdlPlan(DdlCommand cmd) {
+        this.cmd = cmd;
+    }
+
+    /** */
+    public DdlCommand command() {
+        return cmd;
     }
 
     /** {@inheritDoc} */
     @Override public Type type() {
-        return Type.QUERY;
+        return Type.DDL;
     }
 
     /** {@inheritDoc} */
     @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+        return this;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DummyPlanCache.java
similarity index 67%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DummyPlanCache.java
index 9ef2cfb..f07658e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DummyPlanCache.java
@@ -14,27 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-/**
- * Distributed query plan.
- */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
-    /**
-     * @param fieldsMeta Fields metadata.
-     */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
-    }
+import java.util.List;
 
+public class DummyPlanCache implements QueryPlanCache {
     /** {@inheritDoc} */
-    @Override public Type type() {
-        return Type.QUERY;
+    @Override public List<QueryPlan> queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory) {
+        return factory.create(ctx);
     }
 
     /** {@inheritDoc} */
-    @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+    @Override public void clear() {
+
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ExplainPlan.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ExplainPlan.java
index 9ef2cfb..50194c0 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ExplainPlan.java
@@ -18,23 +18,41 @@
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 /**
- * Distributed query plan.
+ * Query explain plan.
  */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
-    /**
-     * @param fieldsMeta Fields metadata.
-     */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
+public class ExplainPlan implements QueryPlan {
+    /** Column name. */
+    public static final String PLAN_COL_NAME = "PLAN";
+
+    /** */
+    private final FieldsMetadata fieldsMeta;
+
+    /** */
+    private final String plan;
+
+    /** */
+    public ExplainPlan(String plan, FieldsMetadata fieldsMeta) {
+        this.fieldsMeta = fieldsMeta;
+        this.plan = plan;
     }
 
     /** {@inheritDoc} */
     @Override public Type type() {
-        return Type.QUERY;
+        return Type.EXPLAIN;
     }
 
     /** {@inheritDoc} */
     @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+        return this;
+    }
+
+    /** */
+    public FieldsMetadata fieldsMeta() {
+        return fieldsMeta;
+    }
+
+    /** */
+    public String plan() {
+        return plan;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
index 093146c..82b7fe6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadata.java
@@ -14,21 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import java.util.List;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+import org.apache.calcite.rel.type.RelDataType;
+
+/**
+ *
+ */
+public interface FieldsMetadata {
+    /**
+     * @return Result row type.
+     */
+    RelDataType rowType();
+
+    /**
+     * @return Result row origins (or where a field value comes from).
+     */
+    List<List<String>> origins();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
index 9ef2cfb..b6c7750 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FieldsMetadataImpl.java
@@ -17,24 +17,31 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-/**
- * Distributed query plan.
- */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
-    /**
-     * @param fieldsMeta Fields metadata.
-     */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
+import java.util.List;
+
+import org.apache.calcite.rel.type.RelDataType;
+
+/** */
+public class FieldsMetadataImpl implements FieldsMetadata {
+    /** */
+    private final RelDataType rowType;
+
+    /** */
+    private final List<List<String>> origins;
+
+    /** */
+    public FieldsMetadataImpl(RelDataType rowType, List<List<String>> origins) {
+        this.rowType = rowType;
+        this.origins = origins;
     }
 
     /** {@inheritDoc} */
-    @Override public Type type() {
-        return Type.QUERY;
+    @Override public RelDataType rowType() {
+        return rowType;
     }
 
     /** {@inheritDoc} */
-    @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+    @Override public List<List<String>> origins() {
+        return origins;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java
index aaba73c..9f85b0e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Fragment.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.Supplier;
 
@@ -150,7 +149,7 @@ public class Fragment {
     }
 
     /** */
-    private FragmentMapping mapping(PlanningContext ctx, RelMetadataQuery mq, Supplier<List<UUID>> nodesSource) {
+    private FragmentMapping mapping(PlanningContext ctx, RelMetadataQuery mq, Supplier<List<String>> nodesSource) {
         try {
             FragmentMapping mapping = IgniteMdFragmentMapping._fragmentMapping(root, mq);
 
@@ -176,7 +175,7 @@ public class Fragment {
     }
 
     /** */
-    @NotNull private Supplier<List<UUID>> nodesSource(MappingService mappingSrvc, PlanningContext ctx) {
+    @NotNull private Supplier<List<String>> nodesSource(MappingService mappingSrvc, PlanningContext ctx) {
         return () -> mappingSrvc.executionNodes(ctx.topologyVersion(), single(), null);
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentPlan.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentPlan.java
index 9ef2cfb..c4485f3 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/FragmentPlan.java
@@ -17,24 +17,33 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-/**
- * Distributed query plan.
- */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
-    /**
-     * @param fieldsMeta Fields metadata.
-     */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+
+/** */
+public class FragmentPlan implements QueryPlan {
+    /** */
+    private final IgniteRel root;
+
+    /** */
+    public FragmentPlan(IgniteRel root) {
+        RelOptCluster cluster = PlanningContext.empty().cluster();
+
+        this.root = new Cloner(cluster).visit(root);
+    }
+
+    /** */
+    public IgniteRel root() {
+        return root;
     }
 
     /** {@inheritDoc} */
     @Override public Type type() {
-        return Type.QUERY;
+        return Type.FRAGMENT;
     }
 
     /** {@inheritDoc} */
     @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+        return this;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java
index 8f5aec2..7ca4c2e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteRelShuttle.java
@@ -47,9 +47,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceH
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /** */
@@ -180,28 +178,18 @@ public class IgniteRelShuttle implements IgniteRelVisitor<IgniteRel> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteRel rel) {
-        return rel.accept(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteSingleMinus rel) {
-        return processNode(rel);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteMapMinus rel) {
+    @Override public IgniteRel visit(IgniteSetOp rel) {
         return processNode(rel);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteReduceMinus rel) {
+    @Override public IgniteRel visit(IgniteTableFunctionScan rel) {
         return processNode(rel);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel visit(IgniteTableFunctionScan rel) {
-        return processNode(rel);
+    @Override public IgniteRel visit(IgniteRel rel) {
+        return rel.accept(this);
     }
 
     /**
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepDmlPlan.java
similarity index 80%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepDmlPlan.java
index 9ef2cfb..de2aae7 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepDmlPlan.java
@@ -18,23 +18,23 @@
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 /**
- * Distributed query plan.
+ * Distributed dml plan.
  */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
+public class MultiStepDmlPlan extends AbstractMultiStepPlan {
     /**
      * @param fieldsMeta Fields metadata.
      */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
+    public MultiStepDmlPlan(QueryTemplate queryTemplate, FieldsMetadata fieldsMeta) {
         super(queryTemplate, fieldsMeta);
     }
 
     /** {@inheritDoc} */
     @Override public Type type() {
-        return Type.QUERY;
+        return Type.DML;
     }
 
     /** {@inheritDoc} */
     @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+        return new MultiStepDmlPlan(queryTemplate, fieldsMetadata);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
index 6337018..39da9a1 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepPlan.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 
 import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMapping;
@@ -34,6 +33,11 @@ public interface MultiStepPlan extends QueryPlan {
     List<Fragment> fragments();
 
     /**
+     * @return Fields metadata.
+     */
+    FieldsMetadata fieldsMetadata();
+
+    /**
      * @param fragment Fragment.
      * @return Mapping for a given fragment.
      */
@@ -43,7 +47,7 @@ public interface MultiStepPlan extends QueryPlan {
     ColocationGroup target(Fragment fragment);
 
     /** */
-    Map<Long, List<UUID>> remotes(Fragment fragment);
+    Map<Long, List<String>> remotes(Fragment fragment);
 
     /**
      * Inits query fragments.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
index 9ef2cfb..866514f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
@@ -24,7 +24,7 @@ public class MultiStepQueryPlan extends AbstractMultiStepPlan {
     /**
      * @param fieldsMeta Fields metadata.
      */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
+    public MultiStepQueryPlan(QueryTemplate queryTemplate, FieldsMetadata fieldsMeta) {
         super(queryTemplate, fieldsMeta);
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
index 722e8f3..f2a479d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
@@ -48,9 +48,9 @@ import org.apache.ignite.internal.processors.query.calcite.rule.FilterSpoolMerge
 import org.apache.ignite.internal.processors.query.calcite.rule.HashAggregateConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.LogicalScanConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.MergeJoinConverterRule;
-import org.apache.ignite.internal.processors.query.calcite.rule.MinusConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.NestedLoopJoinConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.ProjectConverterRule;
+import org.apache.ignite.internal.processors.query.calcite.rule.SetOpConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.SortAggregateConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.SortConverterRule;
 import org.apache.ignite.internal.processors.query.calcite.rule.TableFunctionScanConverterRule;
@@ -144,6 +144,7 @@ public enum PlannerPhase {
 
                     CoreRules.UNION_MERGE,
                     CoreRules.MINUS_MERGE,
+                    CoreRules.INTERSECT_MERGE,
                     CoreRules.UNION_REMOVE,
                     CoreRules.JOIN_COMMUTE,
                     CoreRules.AGGREGATE_REMOVE,
@@ -178,8 +179,10 @@ public enum PlannerPhase {
                     HashAggregateConverterRule.MAP_REDUCE,
                     SortAggregateConverterRule.SINGLE,
                     SortAggregateConverterRule.MAP_REDUCE,
-                    MinusConverterRule.SINGLE,
-                    MinusConverterRule.MAP_REDUCE,
+                    SetOpConverterRule.SINGLE_MINUS,
+                    SetOpConverterRule.MAP_REDUCE_MINUS,
+                    SetOpConverterRule.SINGLE_INTERSECT,
+                    SetOpConverterRule.MAP_REDUCE_INTERSECT,
                     ProjectConverterRule.INSTANCE,
                     FilterConverterRule.INSTANCE,
                     TableModifyConverterRule.INSTANCE,
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
index 8a895a4..fe82832 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 import java.util.Properties;
-import java.util.UUID;
 import java.util.function.Function;
 
 import org.apache.calcite.config.CalciteConnectionConfig;
@@ -56,10 +55,10 @@ public final class PlanningContext implements Context {
     private final Context parentCtx;
 
     /** */
-    private final UUID locNodeId;
+    private final String locNodeId;
 
     /** */
-    private final UUID originatingNodeId;
+    private final String originatingNodeId;
 
     /** */
     private final String qry;
@@ -91,8 +90,8 @@ public final class PlanningContext implements Context {
     private PlanningContext(
         FrameworkConfig cfg,
         Context parentCtx,
-        UUID locNodeId,
-        UUID originatingNodeId,
+        String locNodeId,
+        String originatingNodeId,
         String qry,
         Object[] parameters,
         long topVer
@@ -114,14 +113,14 @@ public final class PlanningContext implements Context {
     /**
      * @return Local node ID.
      */
-    public UUID localNodeId() {
+    public String localNodeId() {
         return locNodeId;
     }
 
     /**
      * @return Originating node ID (the node, who started the execution).
      */
-    public UUID originatingNodeId() {
+    public String originatingNodeId() {
         return originatingNodeId == null ? locNodeId : originatingNodeId;
     }
 
@@ -305,10 +304,10 @@ public final class PlanningContext implements Context {
         private Context parentCtx = Contexts.empty();
 
         /** */
-        private UUID locNodeId;
+        private String locNodeId;
 
         /** */
-        private UUID originatingNodeId;
+        private String originatingNodeId;
 
         /** */
         private String qry;
@@ -323,7 +322,7 @@ public final class PlanningContext implements Context {
          * @param locNodeId Local node ID.
          * @return Builder for chaining.
          */
-        public Builder localNodeId(@NotNull UUID locNodeId) {
+        public Builder localNodeId(@NotNull String locNodeId) {
             this.locNodeId = locNodeId;
             return this;
         }
@@ -332,7 +331,7 @@ public final class PlanningContext implements Context {
          * @param originatingNodeId Originating node ID (the node, who started the execution).
          * @return Builder for chaining.
          */
-        public Builder originatingNodeId(@NotNull UUID originatingNodeId) {
+        public Builder originatingNodeId(@NotNull String originatingNodeId) {
             this.originatingNodeId = originatingNodeId;
             return this;
         }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
similarity index 64%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
index 9ef2cfb..07f94f8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
@@ -17,24 +17,22 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
+import java.util.List;
+
 /**
- * Distributed query plan.
+ *
  */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
+public interface QueryPlanCache {
     /**
-     * @param fieldsMeta Fields metadata.
+     * @param ctx Context.
+     * @param key Cache key.
+     * @param factory Factory method to generate a plan on cache miss.
+     * @return Query plan.
      */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Type type() {
-        return Type.QUERY;
-    }
+    List<QueryPlan> queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory);
 
-    /** {@inheritDoc} */
-    @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
-    }
+    /**
+     * Clear cache.
+     */
+    void clear();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
index 093146c..f3391d3 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
@@ -14,21 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import java.util.List;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ *
+ */
+public interface QueryPlanFactory {
+    /**
+     * @param ctx Planning context.
+     * @return Query plan.
+     */
+    List<QueryPlan> create(PlanningContext ctx);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
index 95e5f2f..444f254 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteRelVisitor.java
@@ -23,9 +23,7 @@ import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceH
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteReduceSortAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleHashAggregate;
 import org.apache.ignite.internal.processors.query.calcite.rel.agg.IgniteSingleSortAggregate;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
 
 /**
  * A visitor to traverse an Ignite relational nodes tree.
@@ -159,17 +157,7 @@ public interface IgniteRelVisitor<T> {
     /**
      * See {@link IgniteRelVisitor#visit(IgniteRel)}
      */
-    T visit(IgniteSingleMinus rel);
-
-    /**
-     * See {@link IgniteRelVisitor#visit(IgniteRel)}
-     */
-    T visit(IgniteMapMinus rel);
-
-    /**
-     * See {@link IgniteRelVisitor#visit(IgniteRel)}
-     */
-    T visit(IgniteReduceMinus rel);
+    T visit(IgniteSetOp rel);
 
     /**
      * See {@link IgniteRelVisitor#visit(IgniteRel)}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteIntersect.java
new file mode 100644
index 0000000..25e5f90
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteIntersect.java
@@ -0,0 +1,68 @@
+/*
+ * 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.set;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+
+/**
+ * Base class for physical INTERSECT set op.
+ */
+public abstract class IgniteIntersect extends Intersect implements IgniteSetOp {
+    /** */
+    IgniteIntersect(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
+        super(cluster, traits, inputs, all);
+    }
+
+    /** {@inheritDoc} */
+    protected IgniteIntersect(RelInput input) {
+        super(TraitUtils.changeTraits(input, IgniteConvention.INSTANCE));
+    }
+
+    /** {@inheritDoc} */
+    @Override public double estimateRowCount(RelMetadataQuery mq) {
+        final List<RelNode> inputs = getInputs();
+
+        double rows = mq.getRowCount(inputs.get(0));
+
+        for (int i = 1; i < inputs.size(); i++)
+            rows = 0.5 * Math.min(rows, mq.getRowCount(inputs.get(i)));
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return computeSetOpCost(planner, mq);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean all() {
+        return all;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapIntersect.java
new file mode 100644
index 0000000..0364f79
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapIntersect.java
@@ -0,0 +1,74 @@
+/*
+ * 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.set;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+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.util.Commons;
+
+/**
+ * Physical node for MAP phase of INTERSECT operator.
+ */
+public class IgniteMapIntersect extends IgniteIntersect implements IgniteMapSetOp {
+    /** */
+    public IgniteMapIntersect(
+        RelOptCluster cluster,
+        RelTraitSet traitSet,
+        List<RelNode> inputs,
+        boolean all
+    ) {
+        super(cluster, traitSet, inputs, all);
+    }
+
+    /** */
+    public IgniteMapIntersect(RelInput input) {
+        super(input);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteMapIntersect copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        return new IgniteMapIntersect(getCluster(), traitSet, inputs, all);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteRel clone(RelOptCluster cluster, List<IgniteRel> inputs) {
+        return new IgniteMapIntersect(cluster, getTraitSet(), Commons.cast(inputs), all);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RelDataType deriveRowType() {
+        return buildRowType();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int aggregateFieldsCount() {
+        return getInput(0).getRowType().getFieldCount() + getInputs().size();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
index 9fb6aac..a26d16d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
@@ -18,31 +18,20 @@
 package org.apache.ignite.internal.processors.query.calcite.rel.set;
 
 import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.util.Pair;
 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.trait.CorrelationTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * Physical node for MAP phase of MINUS (EXCEPT) operator.
  */
-public class IgniteMapMinus extends IgniteMinusBase {
+public class IgniteMapMinus extends IgniteMinus implements IgniteMapSetOp {
     /** */
     public IgniteMapMinus(
         RelOptCluster cluster,
@@ -75,67 +64,11 @@ public class IgniteMapMinus extends IgniteMinusBase {
 
     /** {@inheritDoc} */
     @Override protected RelDataType deriveRowType() {
-        RelDataTypeFactory typeFactory = Commons.typeFactory(getCluster());
-
-        assert typeFactory instanceof IgniteTypeFactory;
-
-        RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
-
-        builder.add("GROUP_KEY", typeFactory.createJavaType(Object.class/*GroupKey.class*/));
-        builder.add("COUNTERS", typeFactory.createJavaType(int[].class));
-
-        return builder.build();
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        boolean rewindable = inputTraits.stream()
-            .map(TraitUtils::rewindability)
-            .allMatch(RewindabilityTrait::rewindable);
-
-        if (rewindable)
-            return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
-
-        return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
-            Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        if (inputTraits.stream().allMatch(t -> TraitUtils.distribution(t).satisfies(IgniteDistributions.single())))
-            return ImmutableList.of(); // If all distributions are single or broadcast IgniteSingleMinus should be used.
-
-        if (inputTraits.stream().anyMatch(t -> TraitUtils.distribution(t) == IgniteDistributions.single()))
-            return ImmutableList.of(); // Mixing of single and random is prohibited.
-
-        return ImmutableList.of(
-            Pair.of(nodeTraits.replace(IgniteDistributions.random()), Commons.transform(inputTraits,
-                t -> t.replace(IgniteDistributions.random())))
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inTraits
-    ) {
-        Set<CorrelationId> correlationIds = inTraits.stream()
-            .map(TraitUtils::correlation)
-            .flatMap(corrTr -> corrTr.correlationIds().stream())
-            .collect(Collectors.toSet());
-
-        return ImmutableList.of(Pair.of(nodeTraits.replace(CorrelationTrait.correlations(correlationIds)),
-            inTraits));
+        return buildRowType();
     }
 
     /** {@inheritDoc} */
-    @Override protected int aggregateFieldsCount() {
+    @Override public int aggregateFieldsCount() {
         return getInput(0).getRowType().getFieldCount() + COUNTER_FIELDS_CNT;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapSetOp.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapSetOp.java
index 9fb6aac..22502ad 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMapSetOp.java
@@ -22,16 +22,13 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.util.Pair;
-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.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey;
 import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
 import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
@@ -40,55 +37,11 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactor
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
- * Physical node for MAP phase of MINUS (EXCEPT) operator.
+ * Physical node for MAP phase of set op (MINUS, INTERSECT).
  */
-public class IgniteMapMinus extends IgniteMinusBase {
-    /** */
-    public IgniteMapMinus(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        List<RelNode> inputs,
-        boolean all
-    ) {
-        super(cluster, traitSet, inputs, all);
-    }
-
-    /** */
-    public IgniteMapMinus(RelInput input) {
-        super(input);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteMapMinus copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        return new IgniteMapMinus(getCluster(), traitSet, inputs, all);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel clone(RelOptCluster cluster, List<IgniteRel> inputs) {
-        return new IgniteMapMinus(cluster, getTraitSet(), Commons.cast(inputs), all);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
-        return visitor.visit(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected RelDataType deriveRowType() {
-        RelDataTypeFactory typeFactory = Commons.typeFactory(getCluster());
-
-        assert typeFactory instanceof IgniteTypeFactory;
-
-        RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
-
-        builder.add("GROUP_KEY", typeFactory.createJavaType(Object.class/*GroupKey.class*/));
-        builder.add("COUNTERS", typeFactory.createJavaType(int[].class));
-
-        return builder.build();
-    }
-
+public interface IgniteMapSetOp extends IgniteSetOp {
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
         RelTraitSet nodeTraits,
         List<RelTraitSet> inputTraits
     ) {
@@ -104,24 +57,24 @@ public class IgniteMapMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
         RelTraitSet nodeTraits,
         List<RelTraitSet> inputTraits
     ) {
         if (inputTraits.stream().allMatch(t -> TraitUtils.distribution(t).satisfies(IgniteDistributions.single())))
             return ImmutableList.of(); // If all distributions are single or broadcast IgniteSingleMinus should be used.
 
-        if (inputTraits.stream().anyMatch(t -> TraitUtils.distribution(t) == IgniteDistributions.single()))
-            return ImmutableList.of(); // Mixing of single and random is prohibited.
-
         return ImmutableList.of(
             Pair.of(nodeTraits.replace(IgniteDistributions.random()), Commons.transform(inputTraits,
-                t -> t.replace(IgniteDistributions.random())))
+                t -> TraitUtils.distribution(t) == IgniteDistributions.broadcast() ?
+                    // Allow broadcast with trim-exchange to be used in map-reduce set-op.
+                    t.replace(IgniteDistributions.hash(ImmutableList.of(0))) :
+                    t.replace(IgniteDistributions.random())))
         );
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
         RelTraitSet nodeTraits,
         List<RelTraitSet> inTraits
     ) {
@@ -134,8 +87,22 @@ public class IgniteMapMinus extends IgniteMinusBase {
             inTraits));
     }
 
+    /** Build RowType for MAP node. */
+    public default RelDataType buildRowType() {
+        RelDataTypeFactory typeFactory = Commons.typeFactory(getCluster());
+
+        assert typeFactory instanceof IgniteTypeFactory;
+
+        RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
+
+        builder.add("GROUP_KEY", typeFactory.createJavaType(GroupKey.class));
+        builder.add("COUNTERS", typeFactory.createJavaType(int[].class));
+
+        return builder.build();
+    }
+
     /** {@inheritDoc} */
-    @Override protected int aggregateFieldsCount() {
-        return getInput(0).getRowType().getFieldCount() + COUNTER_FIELDS_CNT;
+    @Override public default AggregateType aggregateType() {
+        return AggregateType.MAP;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMinus.java
new file mode 100644
index 0000000..38b98ab
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteMinus.java
@@ -0,0 +1,71 @@
+/*
+ * 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.set;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+
+/**
+ * Base class for physical MINUS (EXCEPT) set op.
+ */
+public abstract class IgniteMinus extends Minus implements IgniteSetOp {
+    /** Count of counter fields used to aggregate results. */
+    protected static final int COUNTER_FIELDS_CNT = 2;
+
+    /** */
+    IgniteMinus(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
+        super(cluster, traits, inputs, all);
+    }
+
+    /** {@inheritDoc} */
+    protected IgniteMinus(RelInput input) {
+        super(TraitUtils.changeTraits(input, IgniteConvention.INSTANCE));
+    }
+
+    /** {@inheritDoc} */
+    @Override public double estimateRowCount(RelMetadataQuery mq) {
+        final List<RelNode> inputs = getInputs();
+
+        double rows = mq.getRowCount(inputs.get(0));
+
+        for (int i = 1; i < inputs.size(); i++)
+            rows -= 0.5 * Math.min(rows, mq.getRowCount(inputs.get(i)));
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return computeSetOpCost(planner, mq);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean all() {
+        return all;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceIntersect.java
similarity index 50%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceIntersect.java
index c068e8e..18c4e49 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceIntersect.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.calcite.rel.set;
 
 import java.util.List;
+
 import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
@@ -27,22 +28,16 @@ import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.util.Pair;
 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.rel.IgniteRelVisitor;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
- * Physical node for REDUCE phase of MINUS (EXCEPT) operator.
+ * Physical node for REDUCE phase of INTERSECT operator.
  */
-public class IgniteReduceMinus extends IgniteMinusBase {
+public class IgniteReduceIntersect extends IgniteIntersect implements IgniteReduceSetOp {
     /** */
-    public IgniteReduceMinus(
+    public IgniteReduceIntersect(
         RelOptCluster cluster,
         RelTraitSet traitSet,
         RelNode input,
@@ -55,7 +50,7 @@ public class IgniteReduceMinus extends IgniteMinusBase {
     }
 
     /** */
-    public IgniteReduceMinus(RelInput input) {
+    public IgniteReduceIntersect(RelInput input) {
         this(
             input.getCluster(),
             input.getTraitSet().replace(IgniteConvention.INSTANCE),
@@ -74,53 +69,13 @@ public class IgniteReduceMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        return ImmutableList.of(
-            Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY), ImmutableList.of(inputTraits.get(0))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
-        List<RelTraitSet> inTraits) {
-        IgniteDistribution distr = TraitUtils.distribution(nodeTraits);
-
-        if (IgniteDistributions.single().satisfies(distr)) {
-            return Pair.of(nodeTraits.replace(IgniteDistributions.single()),
-                Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()),
-            ImmutableList.of(sole(inputTraits).replace(IgniteDistributions.single()))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inTraits
-    ) {
-        return ImmutableList.of(Pair.of(nodeTraits.replace(TraitUtils.correlation(inTraits.get(0))),
-            inTraits));
-    }
-
-    /** {@inheritDoc} */
     @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        return new IgniteReduceMinus(getCluster(), traitSet, sole(inputs), all, rowType);
+        return new IgniteReduceIntersect(getCluster(), traitSet, sole(inputs), all, rowType);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteReduceMinus clone(RelOptCluster cluster, List<IgniteRel> inputs) {
-        return new IgniteReduceMinus(cluster, getTraitSet(), sole(inputs), all, rowType);
+    @Override public IgniteReduceIntersect clone(RelOptCluster cluster, List<IgniteRel> inputs) {
+        return new IgniteReduceIntersect(cluster, getTraitSet(), sole(inputs), all, rowType);
     }
 
     /** {@inheritDoc} */
@@ -129,7 +84,7 @@ public class IgniteReduceMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override protected int aggregateFieldsCount() {
-        return rowType.getFieldCount() + COUNTER_FIELDS_CNT;
+    @Override public int aggregateFieldsCount() {
+        return rowType.getFieldCount() + 2 /* At least two fields required for count aggregation. */;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
index c068e8e..f86646f 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceMinus.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.calcite.rel.set;
 
 import java.util.List;
+
 import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
@@ -27,20 +28,14 @@ import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.util.Pair;
 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.rel.IgniteRelVisitor;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * Physical node for REDUCE phase of MINUS (EXCEPT) operator.
  */
-public class IgniteReduceMinus extends IgniteMinusBase {
+public class IgniteReduceMinus extends IgniteMinus implements IgniteReduceSetOp {
     /** */
     public IgniteReduceMinus(
         RelOptCluster cluster,
@@ -74,46 +69,6 @@ public class IgniteReduceMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        return ImmutableList.of(
-            Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY), ImmutableList.of(inputTraits.get(0))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
-        List<RelTraitSet> inTraits) {
-        IgniteDistribution distr = TraitUtils.distribution(nodeTraits);
-
-        if (IgniteDistributions.single().satisfies(distr)) {
-            return Pair.of(nodeTraits.replace(IgniteDistributions.single()),
-                Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()),
-            ImmutableList.of(sole(inputTraits).replace(IgniteDistributions.single()))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inTraits
-    ) {
-        return ImmutableList.of(Pair.of(nodeTraits.replace(TraitUtils.correlation(inTraits.get(0))),
-            inTraits));
-    }
-
-    /** {@inheritDoc} */
     @Override public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
         return new IgniteReduceMinus(getCluster(), traitSet, sole(inputs), all, rowType);
     }
@@ -129,7 +84,7 @@ public class IgniteReduceMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override protected int aggregateFieldsCount() {
+    @Override public int aggregateFieldsCount() {
         return rowType.getFieldCount() + COUNTER_FIELDS_CNT;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceSetOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceSetOp.java
new file mode 100644
index 0000000..06669c2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteReduceSetOp.java
@@ -0,0 +1,75 @@
+/*
+ * 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.set;
+
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * Physical node for REDUCE phase of set op (MINUS, INTERSECT).
+ */
+public interface IgniteReduceSetOp extends IgniteSetOp {
+    /** {@inheritDoc} */
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+        RelTraitSet nodeTraits,
+        List<RelTraitSet> inputTraits
+    ) {
+        return ImmutableList.of(
+            Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY), ImmutableList.of(inputTraits.get(0))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public default Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
+        List<RelTraitSet> inTraits) {
+        if (TraitUtils.distribution(nodeTraits) == IgniteDistributions.single())
+            return Pair.of(nodeTraits, Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+        RelTraitSet nodeTraits,
+        List<RelTraitSet> inputTraits
+    ) {
+        return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()),
+            ImmutableList.of(inputTraits.get(0).replace(IgniteDistributions.single()))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+        RelTraitSet nodeTraits,
+        List<RelTraitSet> inTraits
+    ) {
+        return ImmutableList.of(Pair.of(nodeTraits.replace(TraitUtils.correlation(inTraits.get(0))),
+            inTraits));
+    }
+
+    /** {@inheritDoc} */
+    @Override public default AggregateType aggregateType() {
+        return AggregateType.REDUCE;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSetOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSetOp.java
new file mode 100644
index 0000000..127aa09
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSetOp.java
@@ -0,0 +1,78 @@
+/*
+ * 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.set;
+
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCost;
+import org.apache.ignite.internal.processors.query.calcite.metadata.cost.IgniteCostFactory;
+import org.apache.ignite.internal.processors.query.calcite.trait.TraitsAwareIgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * Base interface for physical set op node (MINUS, INTERSECT).
+ */
+public interface IgniteSetOp extends TraitsAwareIgniteRel {
+    /** ALL flag of set op. */
+    public boolean all();
+
+    /** {@inheritDoc} */
+    @Override public default Pair<RelTraitSet, List<RelTraitSet>> passThroughCollation(RelTraitSet nodeTraits,
+        List<RelTraitSet> inputTraits) {
+        // Operation erases collation.
+        return Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            Commons.transform(inputTraits, t -> t.replace(RelCollations.EMPTY)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCollation(RelTraitSet nodeTraits,
+        List<RelTraitSet> inputTraits) {
+        // Operation erases collation.
+        return ImmutableList.of(Pair.of(nodeTraits.replace(RelCollations.EMPTY),
+            Commons.transform(inputTraits, t -> t.replace(RelCollations.EMPTY))));
+    }
+
+    /** Gets count of fields for aggregation for this node. Required for memory consumption calculation. */
+    public int aggregateFieldsCount();
+
+    /** Compute cost for set op. */
+    public default RelOptCost computeSetOpCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        IgniteCostFactory costFactory = (IgniteCostFactory)planner.getCostFactory();
+
+        double inputRows = 0;
+
+        for (RelNode input : getInputs())
+            inputRows += mq.getRowCount(input);
+
+        double mem = 0.5 * inputRows * aggregateFieldsCount() * IgniteCost.AVERAGE_FIELD_SIZE;
+
+        return costFactory.makeCost(inputRows, inputRows * IgniteCost.ROW_PASS_THROUGH_COST, 0, mem, 0);
+    }
+
+    /** Aggregate type. */
+    public AggregateType aggregateType();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleIntersect.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleIntersect.java
new file mode 100644
index 0000000..65f129b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleIntersect.java
@@ -0,0 +1,68 @@
+/*
+ * 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.set;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+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.util.Commons;
+
+/**
+ * Physical node for INTERSECT operator which inputs satisfy SINGLE distribution.
+ */
+public class IgniteSingleIntersect extends IgniteIntersect implements IgniteSingleSetOp {
+    /** {@inheritDoc} */
+    public IgniteSingleIntersect(
+        RelOptCluster cluster,
+        RelTraitSet traitSet,
+        List<RelNode> inputs,
+        boolean all
+    ) {
+        super(cluster, traitSet, inputs, all);
+    }
+
+    /** */
+    public IgniteSingleIntersect(RelInput input) {
+        super(input);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSingleIntersect copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        return new IgniteSingleIntersect(getCluster(), traitSet, inputs, all);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteRel clone(RelOptCluster cluster, List<IgniteRel> inputs) {
+        return new IgniteSingleIntersect(cluster, getTraitSet(), Commons.cast(inputs), all);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int aggregateFieldsCount() {
+        return getInput(0).getRowType().getFieldCount() + getInputs().size();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
index 3165bb5..71a1d35 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
@@ -18,28 +18,19 @@
 package org.apache.ignite.internal.processors.query.calcite.rel.set;
 
 import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-import com.google.common.collect.ImmutableList;
+
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.util.Pair;
 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.trait.CorrelationTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * Physical node for MINUS (EXCEPT) operator which inputs satisfy SINGLE distribution.
  */
-public class IgniteSingleMinus extends IgniteMinusBase {
+public class IgniteSingleMinus extends IgniteMinus implements IgniteSingleSetOp {
     /** {@inheritDoc} */
     public IgniteSingleMinus(
         RelOptCluster cluster,
@@ -56,63 +47,6 @@ public class IgniteSingleMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        boolean rewindable = inputTraits.stream()
-            .map(TraitUtils::rewindability)
-            .allMatch(RewindabilityTrait::rewindable);
-
-        if (rewindable)
-            return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.REWINDABLE), inputTraits));
-
-        return ImmutableList.of(Pair.of(nodeTraits.replace(RewindabilityTrait.ONE_WAY),
-            Commons.transform(inputTraits, t -> t.replace(RewindabilityTrait.ONE_WAY))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inTraits) {
-        IgniteDistribution distr = TraitUtils.distribution(nodeTraits);
-
-        if (IgniteDistributions.single().satisfies(distr)) {
-            return Pair.of(nodeTraits.replace(IgniteDistributions.single()),
-                Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inputTraits
-    ) {
-        boolean single = inputTraits.stream()
-            .map(TraitUtils::distribution)
-            .allMatch(d -> d.satisfies(IgniteDistributions.single()));
-
-        if (!single)
-            return ImmutableList.of();
-
-        return ImmutableList.of(Pair.of(nodeTraits.replace(IgniteDistributions.single()), inputTraits));
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
-        RelTraitSet nodeTraits,
-        List<RelTraitSet> inTraits
-    ) {
-        Set<CorrelationId> correlationIds = inTraits.stream()
-            .map(TraitUtils::correlation)
-            .flatMap(corrTr -> corrTr.correlationIds().stream())
-            .collect(Collectors.toSet());
-
-        return ImmutableList.of(Pair.of(nodeTraits.replace(CorrelationTrait.correlations(correlationIds)),
-            inTraits));
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteSingleMinus copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
         return new IgniteSingleMinus(getCluster(), traitSet, inputs, all);
     }
@@ -128,7 +62,7 @@ public class IgniteSingleMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override protected int aggregateFieldsCount() {
+    @Override public int aggregateFieldsCount() {
         return getInput(0).getRowType().getFieldCount() + COUNTER_FIELDS_CNT;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleSetOp.java
similarity index 58%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleSetOp.java
index 3165bb5..3dd5249 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleMinus.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/set/IgniteSingleSetOp.java
@@ -20,43 +20,24 @@ package org.apache.ignite.internal.processors.query.calcite.rel.set;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
+
 import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.util.Pair;
-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.exec.exp.agg.AggregateType;
 import org.apache.ignite.internal.processors.query.calcite.trait.CorrelationTrait;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
 import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
- * Physical node for MINUS (EXCEPT) operator which inputs satisfy SINGLE distribution.
+ * Physical node for set op (MINUS, INTERSECT) operator which inputs satisfy SINGLE distribution.
  */
-public class IgniteSingleMinus extends IgniteMinusBase {
+public interface IgniteSingleSetOp extends IgniteSetOp {
     /** {@inheritDoc} */
-    public IgniteSingleMinus(
-        RelOptCluster cluster,
-        RelTraitSet traitSet,
-        List<RelNode> inputs,
-        boolean all
-    ) {
-        super(cluster, traitSet, inputs, all);
-    }
-
-    /** */
-    public IgniteSingleMinus(RelInput input) {
-        super(input);
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveRewindability(
         RelTraitSet nodeTraits,
         List<RelTraitSet> inputTraits
     ) {
@@ -72,19 +53,16 @@ public class IgniteSingleMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits, List<RelTraitSet> inTraits) {
-        IgniteDistribution distr = TraitUtils.distribution(nodeTraits);
-
-        if (IgniteDistributions.single().satisfies(distr)) {
-            return Pair.of(nodeTraits.replace(IgniteDistributions.single()),
-                Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
-        }
+    @Override public default Pair<RelTraitSet, List<RelTraitSet>> passThroughDistribution(RelTraitSet nodeTraits,
+        List<RelTraitSet> inTraits) {
+        if (TraitUtils.distribution(nodeTraits) == IgniteDistributions.single())
+            return Pair.of(nodeTraits, Commons.transform(inTraits, t -> t.replace(IgniteDistributions.single())));
 
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveDistribution(
         RelTraitSet nodeTraits,
         List<RelTraitSet> inputTraits
     ) {
@@ -99,7 +77,7 @@ public class IgniteSingleMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
+    @Override public default List<Pair<RelTraitSet, List<RelTraitSet>>> deriveCorrelation(
         RelTraitSet nodeTraits,
         List<RelTraitSet> inTraits
     ) {
@@ -113,22 +91,7 @@ public class IgniteSingleMinus extends IgniteMinusBase {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteSingleMinus copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        return new IgniteSingleMinus(getCluster(), traitSet, inputs, all);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel clone(RelOptCluster cluster, List<IgniteRel> inputs) {
-        return new IgniteSingleMinus(cluster, getTraitSet(), Commons.cast(inputs), all);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T accept(IgniteRelVisitor<T> visitor) {
-        return visitor.visit(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int aggregateFieldsCount() {
-        return getInput(0).getRowType().getFieldCount() + COUNTER_FIELDS_CNT;
+    @Override public default AggregateType aggregateType() {
+        return AggregateType.SINGLE;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SetOpConverterRule.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SetOpConverterRule.java
new file mode 100644
index 0000000..c2b0728
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/SetOpConverterRule.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.rule;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.PhysicalNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Util;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapIntersect;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceIntersect;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleIntersect;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+
+/**
+ * Set op (MINUS, INTERSECT) converter rule.
+ */
+public class SetOpConverterRule {
+    /** */
+    public static final RelOptRule SINGLE_MINUS = new SingleMinusConverterRule();
+
+    /** */
+    public static final RelOptRule SINGLE_INTERSECT = new SingleIntersectConverterRule();
+
+    /** */
+    public static final RelOptRule MAP_REDUCE_MINUS = new MapReduceMinusConverterRule();
+
+    /** */
+    public static final RelOptRule MAP_REDUCE_INTERSECT = new MapReduceIntersectConverterRule();
+
+    /** */
+    private SetOpConverterRule() {
+        // No-op.
+    }
+
+    /** */
+    private abstract static class SingleSetOpConverterRule<T extends SetOp> extends AbstractIgniteConverterRule<T> {
+        /** */
+        SingleSetOpConverterRule(Class<T> cls, String desc) {
+            super(cls, desc);
+        }
+
+        /** Node factory method. */
+        abstract PhysicalNode createNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all);
+
+        /** {@inheritDoc} */
+        @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, T setOp) {
+            RelOptCluster cluster = setOp.getCluster();
+            RelTraitSet inTrait = cluster.traitSetOf(IgniteConvention.INSTANCE).replace(IgniteDistributions.single());
+            RelTraitSet outTrait = cluster.traitSetOf(IgniteConvention.INSTANCE).replace(IgniteDistributions.single());
+            List<RelNode> inputs = Util.transform(setOp.getInputs(), rel -> convert(rel, inTrait));
+
+            return createNode(cluster, outTrait, inputs, setOp.all);
+        }
+    }
+
+    /** */
+    private static class SingleMinusConverterRule extends SingleSetOpConverterRule<LogicalMinus> {
+        /** */
+        SingleMinusConverterRule() {
+            super(LogicalMinus.class, "SingleMinusConverterRule");
+        }
+
+        /** {@inheritDoc} */
+        @Override PhysicalNode createNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+            boolean all) {
+            return new IgniteSingleMinus(cluster, traits, inputs, all);
+        }
+    }
+
+    /** */
+    private static class SingleIntersectConverterRule extends SingleSetOpConverterRule<LogicalIntersect> {
+        /** */
+        SingleIntersectConverterRule() {
+            super(LogicalIntersect.class, "SingleIntersectConverterRule");
+        }
+
+        /** {@inheritDoc} */
+        @Override PhysicalNode createNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+            boolean all) {
+            return new IgniteSingleIntersect(cluster, traits, inputs, all);
+        }
+    }
+
+    /** */
+    private abstract static class MapReduceSetOpConverterRule<T extends SetOp> extends AbstractIgniteConverterRule<T> {
+        /** */
+        MapReduceSetOpConverterRule(Class<T> cls, String desc) {
+            super(cls, desc);
+        }
+
+        /** Map node factory method. */
+        abstract PhysicalNode createMapNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+            boolean all);
+
+        /** Reduce node factory method. */
+        abstract PhysicalNode createReduceNode(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+            boolean all, RelDataType rowType);
+
+        /** {@inheritDoc} */
+        @Override protected PhysicalNode convert(RelOptPlanner planner, RelMetadataQuery mq, T setOp) {
+            RelOptCluster cluster = setOp.getCluster();
+            RelTraitSet inTrait = cluster.traitSetOf(IgniteConvention.INSTANCE);
+            RelTraitSet outTrait = cluster.traitSetOf(IgniteConvention.INSTANCE);
+            List<RelNode> inputs = Util.transform(setOp.getInputs(), rel -> convert(rel, inTrait));
+
+            RelNode map = createMapNode(cluster, outTrait, inputs, setOp.all);
+
+            return createReduceNode(
+                cluster,
+                outTrait.replace(IgniteDistributions.single()),
+                convert(map, inTrait.replace(IgniteDistributions.single())),
+                setOp.all,
+                cluster.getTypeFactory().leastRestrictive(Util.transform(inputs, RelNode::getRowType))
+            );
+        }
+    }
+
+    /** */
+    private static class MapReduceMinusConverterRule extends MapReduceSetOpConverterRule<LogicalMinus> {
+        /** */
+        MapReduceMinusConverterRule() {
+            super(LogicalMinus.class, "MapReduceMinusConverterRule");
+        }
+
+        /** {@inheritDoc} */
+        @Override PhysicalNode createMapNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+            boolean all) {
+            return new IgniteMapMinus(cluster, traits, inputs, all);
+        }
+
+        /** {@inheritDoc} */
+        @Override PhysicalNode createReduceNode(RelOptCluster cluster, RelTraitSet traits, RelNode input, boolean all,
+            RelDataType rowType) {
+            return new IgniteReduceMinus(cluster, traits, input, all, rowType);
+        }
+    }
+
+    /** */
+    private static class MapReduceIntersectConverterRule extends MapReduceSetOpConverterRule<LogicalIntersect> {
+        /** */
+        MapReduceIntersectConverterRule() {
+            super(LogicalIntersect.class, "MapReduceIntersectConverterRule");
+        }
+
+        /** {@inheritDoc} */
+        @Override PhysicalNode createMapNode(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs,
+            boolean all) {
+            return new IgniteMapIntersect(cluster, traits, inputs, all);
+        }
+
+        /** {@inheritDoc} */
+        @Override PhysicalNode createReduceNode(RelOptCluster cluster, RelTraitSet traits, RelNode input, boolean all,
+            RelDataType rowType) {
+            return new IgniteReduceIntersect(cluster, traits, input, all, rowType);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java
index 5d6ef72..c0aba90 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptor.java
@@ -22,9 +22,6 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactor
 /** */
 public interface ColumnDescriptor {
     /** */
-    boolean field();
-
-    /** */
     boolean key();
 
     /** */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptorImpl.java
new file mode 100644
index 0000000..74cd5c1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/ColumnDescriptorImpl.java
@@ -0,0 +1,93 @@
+/*
+ * 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.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.jetbrains.annotations.Nullable;
+
+/** */
+public class ColumnDescriptorImpl implements ColumnDescriptor {
+    private final boolean key;
+
+    private final String name;
+
+    /** */
+    private final @Nullable Supplier<Object> dfltVal;
+
+    /** */
+    private final int fieldIdx;
+
+    /** */
+    private final Class<?> storageType;
+
+    public ColumnDescriptorImpl(
+        String name,
+        boolean key,
+        int fieldIdx,
+        Class<?> storageType,
+        @Nullable Supplier<Object> dfltVal
+    ) {
+        this.key = key;
+        this.name = name;
+        this.dfltVal = dfltVal;
+        this.fieldIdx = fieldIdx;
+        this.storageType = storageType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean key() {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasDefaultValue() {
+        return dfltVal != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object defaultValue() {
+        return dfltVal != null ? dfltVal.get() : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int fieldIndex() {
+        return fieldIdx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType logicalType(IgniteTypeFactory f) {
+        return f.createJavaType(storageType);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> storageType() {
+        return storageType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void set(Object dst, Object val) {
+        throw new AssertionError();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
index 9554cb4..14a0bc6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
@@ -17,6 +17,8 @@
 package org.apache.ignite.internal.processors.query.calcite.schema;
 
 import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Predicate;
 
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
@@ -25,11 +27,13 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalIndexScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Ignite table.
@@ -78,6 +82,23 @@ public interface IgniteTable extends TranslatableTable {
     IgniteLogicalIndexScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName);
 
     /**
+     * Creates rows iterator over the table.
+     *
+     * @param execCtx Execution context.
+     * @param group Colocation group.
+     * @param filter Row filter.
+     * @param rowTransformer Row transformer.
+     * @param usedColumns Used columns enumeration.
+     * @return Rows iterator.
+     */
+    <Row> Iterable<Row> scan(
+            ExecutionContext<Row> execCtx,
+            ColocationGroup group,
+            Predicate<Row> filter,
+            Function<Row, Row> rowTransformer,
+            @Nullable ImmutableBitSet usedColumns);
+
+    /**
      * Returns nodes mapping.
      *
      * @param ctx Planning context.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
new file mode 100644
index 0000000..f70ac14
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
@@ -0,0 +1,207 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import com.google.common.collect.ImmutableList;
+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.RelReferentialConstraint;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.RewindabilityTrait;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.calcite.Stubs.resultSetGenerator;
+
+/**
+ * Ignite table implementation.
+ */
+public class IgniteTableImpl extends AbstractTable implements IgniteTable {
+    /** */
+    private final TableDescriptor desc;
+
+    /** */
+    private final Statistic statistic;
+
+    /** */
+    private volatile Collection<Object[]> rows;
+
+    /** */
+    private final Map<String, IgniteIndex> indexes = new ConcurrentHashMap<>();
+
+    /**
+     * @param desc Table descriptor.
+     */
+    public IgniteTableImpl(TableDescriptor desc) {
+        this.desc = desc;
+        statistic = new StatisticsImpl();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType getRowType(RelDataTypeFactory typeFactory, ImmutableBitSet requiredColumns) {
+        return desc.rowType((IgniteTypeFactory)typeFactory, requiredColumns);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Statistic getStatistic() {
+        return statistic;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public TableDescriptor descriptor() {
+        return desc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogicalTableScan toRel(RelOptCluster cluster, RelOptTable relOptTbl) {
+        RelTraitSet traitSet = cluster.traitSetOf(distribution())
+            .replace(RewindabilityTrait.REWINDABLE);
+
+        return IgniteLogicalTableScan.create(cluster, traitSet, relOptTbl, null, null, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogicalIndexScan toRel(RelOptCluster cluster, RelOptTable relOptTbl, String idxName) {
+        RelTraitSet traitSet = cluster.traitSetOf(Convention.Impl.NONE)
+            .replace(distribution())
+            .replace(RewindabilityTrait.REWINDABLE)
+            .replace(getIndex(idxName).collation());
+
+        return IgniteLogicalIndexScan.create(cluster, traitSet, relOptTbl, idxName, null, null, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <Row> Iterable<Row> scan(
+        ExecutionContext<Row> execCtx,
+        ColocationGroup group,
+        Predicate<Row> filter,
+        Function<Row, Row> rowTransformer,
+        @Nullable ImmutableBitSet usedColumns
+    ) {
+        String locNodeId = execCtx.planningContext().localNodeId();
+
+        if (group.nodeIds().contains(locNodeId))
+            return (Iterable<Row>) rows(execCtx.getTypeFactory());
+
+        return Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteDistribution distribution() {
+        return desc.distribution();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ColocationGroup colocationGroup(PlanningContext ctx) {
+        return desc.colocationGroup(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, IgniteIndex> indexes() {
+        return Collections.unmodifiableMap(indexes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addIndex(IgniteIndex idxTbl) {
+        indexes.put(idxTbl.name(), idxTbl);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteIndex getIndex(String idxName) {
+        return indexes.get(idxName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeIndex(String idxName) {
+        indexes.remove(idxName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <C> C unwrap(Class<C> aCls) {
+        if (aCls.isInstance(desc))
+            return aCls.cast(desc);
+
+        return super.unwrap(aCls);
+    }
+
+    private Collection<Object[]> rows(IgniteTypeFactory typeFactory) {
+        if (rows == null) {
+            synchronized (this) {
+                if (rows == null)
+                    rows = resultSetGenerator(10, typeFactory, getRowType(typeFactory));
+            }
+        }
+
+        return rows;
+    }
+
+    /** */
+    private class StatisticsImpl implements Statistic {
+        /** {@inheritDoc} */
+        @Override public Double getRowCount() {
+            return (double)rows(new IgniteTypeFactory()).size();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isKey(ImmutableBitSet cols) {
+            return false; // TODO
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<ImmutableBitSet> getKeys() {
+            return null; // TODO
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelReferentialConstraint> getReferentialConstraints() {
+            return ImmutableList.of();
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelCollation> getCollations() {
+            return ImmutableList.of(); // The method isn't used
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteDistribution getDistribution() {
+            return distribution();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
index 093146c..3368bb4 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
@@ -14,21 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.schema;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import org.apache.calcite.schema.SchemaPlus;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ *
+ */
+public interface SchemaHolder {
+    /**
+     * @return Schema.
+     */
+    SchemaPlus schema();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
new file mode 100644
index 0000000..935bb8b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
@@ -0,0 +1,124 @@
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.network.TopologyService;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.nativeTypeToClass;
+
+/**
+ * Holds actual schema and mutates it on schema change, requested by Ignite.
+ */
+public class SchemaHolderImpl implements SchemaHolder {
+    /** */
+    private final Map<String, IgniteSchema> igniteSchemas = new HashMap<>();
+
+    private final TopologyService topSrvc;
+
+    /** */
+    private volatile SchemaPlus calciteSchema;
+
+    public SchemaHolderImpl(
+        TopologyService topSrvc
+    ) {
+        this.topSrvc = topSrvc;
+
+        SchemaPlus newCalciteSchema = Frameworks.createRootSchema(false);
+        newCalciteSchema.add("PUBLIC", new IgniteSchema("PUBLIC"));
+        calciteSchema = newCalciteSchema;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SchemaPlus schema() {
+        return calciteSchema;
+    }
+
+    public synchronized void onSchemaCreated(String schemaName) {
+        igniteSchemas.putIfAbsent(schemaName, new IgniteSchema(schemaName));
+        rebuild();
+    }
+
+    public synchronized void onSchemaDropped(String schemaName) {
+        igniteSchemas.remove(schemaName);
+        rebuild();
+    }
+
+    public synchronized void onSqlTypeCreated(
+        String schemaName,
+        String tableName,
+        SchemaDescriptor descriptor
+    ) {
+        IgniteSchema schema = igniteSchemas.computeIfAbsent(schemaName, IgniteSchema::new);
+
+        tableName = tableName.substring(schemaName.length() + 1);
+
+        List<ColumnDescriptor> colDescriptors = descriptor.columnNames().stream()
+            .map(descriptor::column)
+            .sorted(Comparator.comparingInt(Column::schemaIndex))
+            .map(col -> new ColumnDescriptorImpl(
+                col.name(),
+                descriptor.isKeyColumn(col.schemaIndex()),
+                col.schemaIndex(),
+                nativeTypeToClass(col.type()),
+                col::defaultValue
+            ))
+            .collect(Collectors.toList());
+
+        TableDescriptorImpl desc = new TableDescriptorImpl(topSrvc, colDescriptors);
+
+        schema.addTable(tableName, new IgniteTableImpl(desc));
+
+        rebuild();
+    }
+
+     public void onSqlTypeUpdated(
+         String schemaName,
+         String tableName,
+         SchemaDescriptor descriptor
+    ) {
+        onSqlTypeCreated(schemaName, tableName, descriptor);
+    }
+
+    public synchronized void onSqlTypeDropped(
+        String schemaName,
+        String tableName
+    ) {
+        IgniteSchema schema = igniteSchemas.computeIfAbsent(schemaName, IgniteSchema::new);
+
+        schema.removeTable(tableName);
+
+        rebuild();
+    }
+
+    private void rebuild() {
+        SchemaPlus newCalciteSchema = Frameworks.createRootSchema(false);
+        newCalciteSchema.add("PUBLIC", new IgniteSchema("PUBLIC"));
+        igniteSchemas.forEach(newCalciteSchema::add);
+        calciteSchema = newCalciteSchema;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
index a8a16d0..9f0c124 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
@@ -23,6 +23,8 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.sql2rel.InitializerExpressionFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 
@@ -35,6 +37,14 @@ public interface TableDescriptor extends RelProtoDataType, InitializerExpression
      */
     IgniteDistribution distribution();
 
+    /**
+     * Returns nodes mapping.
+     *
+     * @param ctx Planning context.
+     * @return Nodes mapping.
+     */
+    ColocationGroup colocationGroup(PlanningContext ctx);
+
     /** {@inheritDoc} */
     @Override default RelDataType apply(RelDataTypeFactory factory) {
         return rowType((IgniteTypeFactory)factory, null);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
new file mode 100644
index 0000000..2d0761e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.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.schema;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.plan.RelOptTable;
+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;
+import org.apache.calcite.schema.ColumnStrategy;
+import org.apache.calcite.sql2rel.InitializerContext;
+import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+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.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ *
+ */
+public class TableDescriptorImpl extends NullInitializerExpressionFactory implements TableDescriptor {
+    /** */
+    private static final ColumnDescriptor[] DUMMY = new ColumnDescriptor[0];
+
+    private final TopologyService topSrvc;
+
+    /** */
+    private final ColumnDescriptor[] descriptors;
+
+    /** */
+    private final Map<String, ColumnDescriptor> descriptorsMap;
+
+    /** */
+    private final ImmutableBitSet insertFields;
+
+    /** */
+    public TableDescriptorImpl(
+        TopologyService topSrvc,
+        List<ColumnDescriptor> columnDescriptors
+    ) {
+        Map<String, ColumnDescriptor> descriptorsMap = new HashMap<>(columnDescriptors.size());
+        for (ColumnDescriptor descriptor : columnDescriptors)
+            descriptorsMap.put(descriptor.name(), descriptor);
+
+        this.descriptors = columnDescriptors.toArray(DUMMY);
+        this.descriptorsMap = descriptorsMap;
+        this.topSrvc = topSrvc;
+
+        insertFields = ImmutableBitSet.range(columnDescriptors.size());
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType insertRowType(IgniteTypeFactory factory) {
+        return rowType(factory, insertFields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteDistribution distribution() {
+        return IgniteDistributions.random();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isUpdateAllowed(RelOptTable tbl, int colIdx) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ColumnStrategy generationStrategy(RelOptTable tbl, int colIdx) {
+        if (descriptors[colIdx].hasDefaultValue())
+            return ColumnStrategy.DEFAULT;
+
+        return super.generationStrategy(tbl, colIdx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RexNode newColumnDefaultValue(RelOptTable tbl, int colIdx, InitializerContext ctx) {
+        final ColumnDescriptor desc = descriptors[colIdx];
+
+        if (!desc.hasDefaultValue())
+            return super.newColumnDefaultValue(tbl, colIdx, ctx);
+
+        final RexBuilder rexBuilder = ctx.getRexBuilder();
+        final IgniteTypeFactory typeFactory = (IgniteTypeFactory) rexBuilder.getTypeFactory();
+
+        return rexBuilder.makeLiteral(desc.defaultValue(), desc.logicalType(typeFactory), false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType rowType(IgniteTypeFactory factory, ImmutableBitSet usedColumns) {
+        RelDataTypeFactory.Builder b = new RelDataTypeFactory.Builder(factory);
+
+        if (usedColumns == null) {
+            for (int i = 0; i < descriptors.length; i++)
+                b.add(descriptors[i].name(), descriptors[i].logicalType(factory));
+        }
+        else {
+            for (int i = usedColumns.nextSetBit(0); i != -1; i = usedColumns.nextSetBit(i + 1))
+                b.add(descriptors[i].name(), descriptors[i].logicalType(factory));
+        }
+
+        return TypeUtils.sqlType(factory, b.build());
+    }
+
+    /** {@inheritDoc} */
+    @Override public ColumnDescriptor columnDescriptor(String fieldName) {
+        return fieldName == null ? null : descriptorsMap.get(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ColocationGroup colocationGroup(PlanningContext ctx) {
+        return partitionedGroup();
+    }
+
+    /** */
+    private ColocationGroup partitionedGroup() {
+        List<List<String>> assignments = topSrvc.allMembers().stream()
+            .map(ClusterNode::id)
+            .map(Collections::singletonList)
+            .collect(Collectors.toList());
+
+        return ColocationGroup.forAssignments(assignments);
+    }
+}
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/AffinityAdapter.java
similarity index 52%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AffinityAdapter.java
index 22da20a..09db5c1 100644
--- 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/AffinityAdapter.java
@@ -17,22 +17,35 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.util.function.ToIntFunction;
 
-/**
- * Ignite distribution trait.
- */
-public interface IgniteDistribution extends RelDistribution {
-    /**
-     * @return Distribution function.
-     */
-    DistributionFunction function();
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.util.IgniteUtils;
 
-    /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+/** */
+final class AffinityAdapter<Row> implements ToIntFunction<Row> {
+    /** */
+    private final ToIntFunction<Object> affinity;
+
+    /** */
+    private final int[] keys;
+
+    /** */
+    private final RowHandler<Row> hndlr;
+
+    /** */
+    AffinityAdapter(ToIntFunction<Object> affinity, int[] keys, RowHandler<Row> hndlr) {
+        this.affinity = affinity;
+        this.keys = keys;
+        this.hndlr = hndlr;
+    }
 
     /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    @Override public int applyAsInt(Row r) {
+        int hash = 0;
+        for (int i = 0; i < keys.length; i++)
+            hash = 31 * hash + affinity.applyAsInt(hndlr.get(keys[i], r));
+
+        return IgniteUtils.safeAbs(hash);
+    }
 }
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/AllNodes.java
similarity index 67%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/AllNodes.java
index 22da20a..f6ae3c1 100644
--- 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/AllNodes.java
@@ -17,22 +17,25 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.util.List;
 
-/**
- * Ignite distribution trait.
- */
-public interface IgniteDistribution extends RelDistribution {
-    /**
-     * @return Distribution function.
-     */
-    DistributionFunction function();
+/** */
+public final class AllNodes<Row> implements Destination<Row> {
+    /** */
+    private final List<String> nodes;
+
+    /** */
+    public AllNodes(List<String> nodes) {
+        this.nodes = nodes;
+    }
 
     /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    @Override public List<String> targets(Row row) {
+        return nodes;
+    }
 
     /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    @Override public List<String> targets() {
+        return nodes;
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
index 093146c..56d5eaa 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Destination.java
@@ -14,21 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.trait;
 
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import java.util.List;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ * Determines where to send a row to.
+ */
+public interface Destination<Row> {
+    /**
+     * @param row Row.
+     * @return Target nodes list for specific row.
+     */
+    List<String> targets(Row row);
+
+    /**
+     * @return All target nodes.
+     */
+    List<String> targets();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
index 2586dae..074682a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionFunction.java
@@ -17,12 +17,21 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
+import java.util.Collections;
+import java.util.List;
 import java.util.Objects;
 
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.util.IgniteUtils;
 
 import static org.apache.ignite.internal.processors.query.calcite.Stubs.intFoo;
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 
 /**
  * Distribution function.
@@ -62,6 +71,18 @@ public abstract class DistributionFunction {
     }
 
     /**
+     * Creates a destination based on this function algorithm, given nodes mapping and given distribution keys.
+     *
+     * @param ctx Execution context.
+     * @param affinityService Affinity function source.
+     * @param group Target mapping.
+     * @param keys Distribution keys.
+     * @return Destination function.
+     */
+    abstract <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
+        ColocationGroup group, ImmutableIntList keys);
+
+    /**
      * @return Function name. This name used for equality checking and in {@link RelNode#getDigest()}.
      */
     protected String name0() {
@@ -135,6 +156,12 @@ public abstract class DistributionFunction {
         @Override public RelDistribution.Type type() {
             return RelDistribution.Type.ANY;
         }
+
+        /** {@inheritDoc} */
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
+            ColocationGroup m, ImmutableIntList k) {
+            throw new IllegalStateException();
+        }
     }
 
     /** */
@@ -146,6 +173,14 @@ public abstract class DistributionFunction {
         @Override public RelDistribution.Type type() {
             return RelDistribution.Type.BROADCAST_DISTRIBUTED;
         }
+
+        /** {@inheritDoc} */
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
+            ColocationGroup m, ImmutableIntList k) {
+            assert m != null && !nullOrEmpty(m.nodeIds());
+
+            return new AllNodes<>(m.nodeIds());
+        }
     }
 
     /** */
@@ -157,6 +192,14 @@ public abstract class DistributionFunction {
         @Override public RelDistribution.Type type() {
             return RelDistribution.Type.RANDOM_DISTRIBUTED;
         }
+
+        /** {@inheritDoc} */
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
+            ColocationGroup m, ImmutableIntList k) {
+            assert m != null && !nullOrEmpty(m.nodeIds());
+
+            return new RandomNode<>(m.nodeIds());
+        }
     }
 
     /** */
@@ -168,6 +211,15 @@ public abstract class DistributionFunction {
         @Override public RelDistribution.Type type() {
             return RelDistribution.Type.SINGLETON;
         }
+
+        /** {@inheritDoc} */
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affinityService,
+            ColocationGroup m, ImmutableIntList k) {
+            if (m == null || m.nodeIds() == null || m.nodeIds().size() != 1)
+                throw new IllegalStateException();
+
+            return new AllNodes<>(Collections.singletonList(Objects.requireNonNull(first(m.nodeIds()))));
+        }
     }
 
     /** */
@@ -178,6 +230,24 @@ public abstract class DistributionFunction {
         @Override public RelDistribution.Type type() {
             return RelDistribution.Type.HASH_DISTRIBUTED;
         }
+
+        /** {@inheritDoc} */
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affSrvc,
+            ColocationGroup m, ImmutableIntList k) {
+            assert m != null && !nullOrEmpty(m.assignments()) && !k.isEmpty();
+
+            List<List<String>> assignments = m.assignments();
+
+            if (IgniteUtils.assertionsEnabled()) {
+                for (List<String> assignment : assignments)
+                    assert nullOrEmpty(assignment) || assignment.size() == 1;
+            }
+
+            AffinityAdapter<Row> affinity = new AffinityAdapter<>(affSrvc.affinity(intFoo()/*CU.UNDEFINED_CACHE_ID*/), k.toIntArray(),
+                ctx.rowHandler());
+
+            return new Partitioned<>(assignments, affinity);
+        }
     }
 
     /** */
@@ -212,6 +282,23 @@ public abstract class DistributionFunction {
             return RelDistribution.Type.HASH_DISTRIBUTED;
         }
 
+        /** {@inheritDoc} */
+        @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ctx, AffinityService affSrvc,
+            ColocationGroup m, ImmutableIntList k) {
+            assert m != null && !nullOrEmpty(m.assignments()) && k.size() == 1;
+
+            List<List<String>> assignments = m.assignments();
+
+            if (IgniteUtils.assertionsEnabled()) {
+                for (List<String> assignment : assignments)
+                    assert nullOrEmpty(assignment) || assignment.size() == 1;
+            }
+
+            AffinityAdapter<Row> affinity = new AffinityAdapter<>(affSrvc.affinity(cacheId), k.toIntArray(), ctx.rowHandler());
+
+            return new Partitioned<>(assignments, affinity);
+        }
+
         /** */
         public Object identity() {
             return identity;
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
index f0ee64a..6ef5cbf 100644
--- 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
@@ -24,10 +24,12 @@ import com.google.common.collect.Ordering;
 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.Mapping;
 import org.apache.calcite.util.mapping.Mappings;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 
 import static org.apache.calcite.rel.RelDistribution.Type.ANY;
 import static org.apache.calcite.rel.RelDistribution.Type.BROADCAST_DISTRIBUTED;
@@ -70,7 +72,7 @@ public final class DistributionTrait implements IgniteDistribution {
     }
 
     /** {@inheritDoc} */
-    @Override public RelDistribution.Type getType() {
+    @Override public Type getType() {
         return function.type();
     }
 
@@ -80,6 +82,11 @@ public final class DistributionTrait implements IgniteDistribution {
     }
 
     /** {@inheritDoc} */
+    @Override public <Row> Destination<Row> destination(ExecutionContext<Row> ectx, AffinityService affSrvc, ColocationGroup target) {
+        return function.destination(ectx, affSrvc, target, keys);
+    }
+
+    /** {@inheritDoc} */
     @Override public ImmutableIntList getKeys() {
         return keys;
     }
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
index 22da20a..5e53013 100644
--- 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
@@ -20,6 +20,9 @@ package org.apache.ignite.internal.processors.query.calcite.trait;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.mapping.Mappings;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.metadata.AffinityService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 
 /**
  * Ignite distribution trait.
@@ -30,6 +33,17 @@ public interface IgniteDistribution extends RelDistribution {
      */
     DistributionFunction function();
 
+    /**
+     * Creates a destination based on this function algorithm, given nodes mapping and distribution keys.
+     *
+     * @param ectx Execution context.
+     * @param affinityService Affinity function source.
+     * @param targetGroup Target mapping.
+     * @return Destination function.
+     */
+    <Row> Destination<Row> destination(ExecutionContext<Row> ectx, AffinityService affinityService,
+        ColocationGroup targetGroup);
+
     /** {@inheritDoc} */
     @Override ImmutableIntList getKeys();
 
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/Partitioned.java
similarity index 53%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/Partitioned.java
index 22da20a..e2ec63c 100644
--- 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/Partitioned.java
@@ -17,22 +17,34 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.util.Collection;
+import java.util.List;
+import java.util.function.ToIntFunction;
+import java.util.stream.Collectors;
 
-/**
- * Ignite distribution trait.
- */
-public interface IgniteDistribution extends RelDistribution {
-    /**
-     * @return Distribution function.
-     */
-    DistributionFunction function();
+/** */
+public final class Partitioned<Row> implements Destination<Row> {
+    /** */
+    private final List<List<String>> assignments;
+
+    /** */
+    private final ToIntFunction<Row> partFun;
+
+    /** */
+    public Partitioned(List<List<String>> assignments, ToIntFunction<Row> partFun) {
+        this.assignments = assignments;
+        this.partFun = partFun;
+    }
 
     /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    @Override public List<String> targets(Row row) {
+        return assignments.get(partFun.applyAsInt(row) % assignments.size());
+    }
 
     /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    @Override public List<String> targets() {
+        return assignments.stream()
+            .flatMap(Collection::stream)
+            .distinct().collect(Collectors.toList());
+    }
 }
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/RandomNode.java
similarity index 61%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/IgniteDistribution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/RandomNode.java
index 22da20a..33e8e0f 100644
--- 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/RandomNode.java
@@ -17,22 +17,32 @@
 
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
-import org.apache.calcite.rel.RelDistribution;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.mapping.Mappings;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
 
-/**
- * Ignite distribution trait.
- */
-public interface IgniteDistribution extends RelDistribution {
-    /**
-     * @return Distribution function.
-     */
-    DistributionFunction function();
+/** */
+public final class RandomNode<Row> implements Destination<Row> {
+    /** */
+    private final Random random;
+
+    /** */
+    private final List<String> nodes;
+
+    /** */
+    public RandomNode(List<String> nodes) {
+        this.nodes = nodes;
+
+        random = new Random();
+    }
 
     /** {@inheritDoc} */
-    @Override ImmutableIntList getKeys();
+    @Override public List<String> targets(Row row) {
+        return Collections.singletonList(nodes.get(random.nextInt(nodes.size())));
+    }
 
     /** {@inheritDoc} */
-    @Override IgniteDistribution apply(Mappings.TargetMapping mapping);
+    @Override public List<String> targets() {
+        return nodes;
+    }
 }
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
index 31e9a74..595f79c 100644
--- 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
@@ -18,16 +18,20 @@
 package org.apache.ignite.internal.processors.query.calcite.util;
 
 import java.io.StringReader;
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.UUID;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
@@ -63,8 +67,11 @@ import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningConte
 import org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable;
 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.schema.NativeType;
 import org.apache.ignite.internal.util.ArrayUtils;
+import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteLogger;
 import org.codehaus.commons.compiler.CompilerFactoryFactory;
 import org.codehaus.commons.compiler.IClassBodyEvaluator;
 import org.codehaus.commons.compiler.ICompilerFactory;
@@ -125,6 +132,35 @@ public final class Commons {
     /** */
     private Commons(){}
 
+    public static <T> Cursor<T> createCursor(Iterable<T> iterable) {
+        return createCursor(iterable.iterator());
+    }
+
+    public static <T> Cursor<T> createCursor(Iterator<T> iter) {
+        return new Cursor<>() {
+            @Override public void remove() {
+                iter.remove();
+            }
+
+            @Override public boolean hasNext() {
+                return iter.hasNext();
+            }
+
+            @Override public T next() {
+                return iter.next();
+            }
+
+            @NotNull @Override public Iterator<T> iterator() {
+                return iter;
+            }
+
+            @Override public void close() throws Exception {
+                if (iter instanceof AutoCloseable)
+                    ((AutoCloseable)iter).close();
+            }
+        };
+    }
+
     /**
      * Combines two lists.
      */
@@ -254,13 +290,22 @@ public final class Commons {
             ((AutoCloseable) o).close();
     }
 
-//    /**
-//     * @param o Object to close.
-//     */
-//    public static void close(Object o, IgniteLogger log) {
-//        if (o instanceof AutoCloseable)
-//            U.close((AutoCloseable) o, log);
-//    }
+    /**
+     * Closes given resource logging possible checked exception.
+     *
+     * @param o Resource to close. If it's {@code null} - it's no-op.
+     * @param log Logger to log possible checked exception.
+     */
+    public static void close(Object o, @NotNull IgniteLogger log) {
+        if (o instanceof AutoCloseable) {
+            try {
+                ((AutoCloseable)o).close();
+            }
+            catch (Exception e) {
+                log.warn("Failed to close resource: " + e.getMessage(), e);
+            }
+        }
+    }
 //
 //    /**
 //     * @param o Object to close.
@@ -446,4 +491,46 @@ public final class Commons {
             }
         }
     }
+
+    public static Class<?> nativeTypeToClass(NativeType type) {
+        assert type != null;
+
+        switch (type.spec()) {
+            case INT8:
+                return Byte.class;
+
+            case INT16:
+                return Short.class;
+
+            case INT32:
+                return Integer.class;
+
+            case INT64:
+                return Long.class;
+
+            case FLOAT:
+                return Float.class;
+
+            case DOUBLE:
+                return Double.class;
+
+            case DECIMAL:
+                return BigDecimal.class;
+
+            case UUID:
+                return UUID.class;
+
+            case STRING:
+                return String.class;
+
+            case BYTES:
+                return byte[].class;
+
+            case BITMASK:
+                return BitSet.class;
+
+            default:
+                throw new IllegalArgumentException("Unsupported type " + type.spec());
+        }
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/NodeLeaveHandler.java
similarity index 58%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/NodeLeaveHandler.java
index 9ef2cfb..5aa03fe 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/MultiStepQueryPlan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/NodeLeaveHandler.java
@@ -14,27 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.ignite.internal.processors.query.calcite.util;
 
-package org.apache.ignite.internal.processors.query.calcite.prepare;
+import java.util.function.Consumer;
 
-/**
- * Distributed query plan.
- */
-public class MultiStepQueryPlan extends AbstractMultiStepPlan {
-    /**
-     * @param fieldsMeta Fields metadata.
-     */
-    public MultiStepQueryPlan(QueryTemplate queryTemplate, Object fieldsMeta) {
-        super(queryTemplate, fieldsMeta);
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyEventHandler;
+
+public class NodeLeaveHandler implements TopologyEventHandler {
+    private final Consumer<ClusterNode> onDisappeared;
+
+    public NodeLeaveHandler(Consumer<ClusterNode> onDisappeared) {
+        this.onDisappeared = onDisappeared;
     }
 
     /** {@inheritDoc} */
-    @Override public Type type() {
-        return Type.QUERY;
+    @Override public void onAppeared(ClusterNode member) {
+        // NO-OP
     }
 
     /** {@inheritDoc} */
-    @Override public QueryPlan copy() {
-        return new MultiStepQueryPlan(queryTemplate, fieldsMetadata);
+    @Override public void onDisappeared(ClusterNode member) {
+        onDisappeared.accept(member);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java
index 962e678..d84c953 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/StripedThreadPoolExecutor.java
@@ -47,7 +47,6 @@ public class StripedThreadPoolExecutor implements ExecutorService {
      * Create striped thread pool.
      *
      * @param concurrentLvl Concurrency level.
-     * @param igniteInstanceName Node name.
      * @param threadNamePrefix Thread name prefix.
      * @param allowCoreThreadTimeOut Sets the policy governing whether core threads may time out and
      * terminate if no tasks arrive within the keep-alive time.
@@ -56,7 +55,6 @@ public class StripedThreadPoolExecutor implements ExecutorService {
      */
     public StripedThreadPoolExecutor(
         int concurrentLvl,
-        String igniteInstanceName,
         String threadNamePrefix,
         UncaughtExceptionHandler eHnd,
         boolean allowCoreThreadTimeOut,
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java
index 92742b8..0b0f46f 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java
@@ -50,17 +50,16 @@ public class AbstractExecutionTest extends IgniteAbstractTest {
 
     /** */
     @BeforeEach
-    public void setup() throws Exception {
-        taskExecutor = new QueryTaskExecutorImpl();
-
-        taskExecutor.stripedThreadPoolExecutor(new StripedThreadPoolExecutor(
-            4,
-            "calcite-exec-test-ignite",
-            "calciteQry",
-            this::handle,
-            true,
-            60_000L
-        ));
+    public void setup() {
+        taskExecutor = new QueryTaskExecutorImpl(
+            new StripedThreadPoolExecutor(
+                4,
+                "calciteQry",
+                this::handle,
+                true,
+                60_000L
+            )
+        );
     }
 
     /** */
@@ -78,7 +77,7 @@ public class AbstractExecutionTest extends IgniteAbstractTest {
         return new ExecutionContext<>(
             taskExecutor,
             PlanningContext.builder()
-                .localNodeId(UUID.randomUUID())
+                .localNodeId(UUID.randomUUID().toString())
                 .build(),
             UUID.randomUUID(),
             fragmentDesc,
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
index 778df9b..dd1f5cd 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.function.BiFunction;
+import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
@@ -61,6 +62,7 @@ import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql2rel.InitializerContext;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.processors.query.calcite.externalize.RelJsonReader;
 import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 import org.apache.ignite.internal.processors.query.calcite.prepare.Cloner;
@@ -88,6 +90,7 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem
 import org.apache.ignite.internal.testframework.IgniteAbstractTest;
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
 import org.apache.ignite.internal.util.ArrayUtils;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.calcite.tools.Frameworks.createRootSchema;
 import static org.apache.calcite.tools.Frameworks.newConfigBuilder;
@@ -435,10 +438,10 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
             CorrelationTraitDef.INSTANCE
         };
 
-        List<UUID> nodes = new ArrayList<>(4);
+        List<String> nodes = new ArrayList<>(4);
 
         for (int i = 0; i < 4; i++)
-            nodes.add(UUID.randomUUID());
+            nodes.add(UUID.randomUUID().toString());
 
         PlanningContext ctx = PlanningContext.builder()
             .localNodeId(first(nodes))
@@ -553,6 +556,17 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public <Row> Iterable<Row> scan(
+                ExecutionContext<Row> execCtx,
+                ColocationGroup group,
+                Predicate<Row> filter,
+                Function<Row, Row> rowTransformer,
+                @Nullable ImmutableBitSet usedColumns
+        ) {
+            throw new AssertionError();
+        }
+
+        /** {@inheritDoc} */
         @Override public RelDataType getRowType(RelDataTypeFactory typeFactory, ImmutableBitSet bitSet) {
             RelDataType rowType = protoType.apply(typeFactory);
 
@@ -710,6 +724,11 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public ColocationGroup colocationGroup(PlanningContext ctx) {
+            throw new AssertionError();
+        }
+
+        /** {@inheritDoc} */
         @Override public ColumnStrategy generationStrategy(RelOptTable table, int iColumn) {
             throw new AssertionError();
         }
@@ -746,11 +765,6 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public boolean field() {
-            return true;
-        }
-
-        /** {@inheritDoc} */
         @Override public boolean key() {
             return false;
         }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/ExceptPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/ExceptPlannerTest.java
deleted file mode 100644
index 429ce93..0000000
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/ExceptPlannerTest.java
+++ /dev/null
@@ -1,329 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.planner;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
-import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
-import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.TestInstance;
-
-/**
- * Test to verify EXCEPT operator.
- */
-@TestInstance(TestInstance.Lifecycle.PER_CLASS)
-public class ExceptPlannerTest extends AbstractPlannerTest {
-    /** Public schema. */
-    private IgniteSchema publicSchema;
-
-    /** */
-    @BeforeAll
-    public void setup() {
-        publicSchema = new IgniteSchema("PUBLIC");
-
-        IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
-
-        RelDataType type = new RelDataTypeFactory.Builder(f)
-            .add("ID", f.createJavaType(Integer.class))
-            .add("NAME", f.createJavaType(String.class))
-            .add("SALARY", f.createJavaType(Double.class))
-            .build();
-
-        createTable(publicSchema, "RANDOM_TBL1", type, IgniteDistributions.random());
-        createTable(publicSchema, "RANDOM_TBL2", type, IgniteDistributions.random());
-        createTable(publicSchema, "BROADCAST_TBL1", type, IgniteDistributions.broadcast());
-        createTable(publicSchema, "BROADCAST_TBL2", type, IgniteDistributions.broadcast());
-        createTable(publicSchema, "SINGLE_TBL1", type, IgniteDistributions.single());
-        createTable(publicSchema, "SINGLE_TBL2", type, IgniteDistributions.single());
-
-        createTable(publicSchema, "AFFINITY_TBL1", type,
-            IgniteDistributions.affinity(0, "Test1", "hash"));
-
-        createTable(publicSchema, "AFFINITY_TBL2", type,
-            IgniteDistributions.affinity(0, "Test2", "hash"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptRandom() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM random_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> !n.all)
-            .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                .and(input(0, isTableScan("random_tbl1")))
-                .and(input(1, isTableScan("random_tbl2")))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptAllRandom() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl1 " +
-            "EXCEPT ALL " +
-            "SELECT * FROM random_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> n.all)
-            .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                .and(input(0, isTableScan("random_tbl1")))
-                .and(input(1, isTableScan("random_tbl2")))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptBroadcast() throws Exception {
-        String sql = "" +
-            "SELECT * FROM broadcast_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM broadcast_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, isTableScan("broadcast_tbl1")))
-            .and(input(1, isTableScan("broadcast_tbl2")))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptSingle() throws Exception {
-        String sql = "" +
-            "SELECT * FROM single_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM single_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, isTableScan("single_tbl1")))
-            .and(input(1, isTableScan("single_tbl2"))));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptSingleAndRandom() throws Exception {
-        String sql = "" +
-            "SELECT * FROM single_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM random_tbl1 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, isTableScan("single_tbl1")))
-            .and(input(1, hasChildThat(isTableScan("random_tbl1")))));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptSingleAndAffinity() throws Exception {
-        String sql = "" +
-            "SELECT * FROM single_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM affinity_tbl1 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, isTableScan("single_tbl1")))
-            .and(input(1, hasChildThat(isTableScan("affinity_tbl1")))));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptSingleAndBroadcast() throws Exception {
-        String sql = "" +
-            "SELECT * FROM single_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM broadcast_tbl1 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, isTableScan("single_tbl1")))
-            .and(input(1, isTableScan("broadcast_tbl1")))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptAffinity() throws Exception {
-        String sql = "" +
-            "SELECT * FROM affinity_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM affinity_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class)
-            .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                .and(input(0, isTableScan("affinity_tbl1")))
-                .and(input(1, isTableScan("affinity_tbl2")))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptBroadcastAndRandom() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM broadcast_tbl1 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, hasChildThat(isTableScan("random_tbl1"))))
-            .and(input(1, isTableScan("broadcast_tbl1")))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptRandomNested() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl2 EXCEPT (" +
-            "   SELECT * FROM random_tbl1 " +
-            "   EXCEPT " +
-            "   SELECT * FROM random_tbl2" +
-            ")";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, hasChildThat(isTableScan("random_tbl2"))))
-            .and(input(1, isInstanceOf(IgniteReduceMinus.class)
-                .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                    .and(input(0, isTableScan("random_tbl1")))
-                    .and(input(1, isTableScan("random_tbl2")))
-                ))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptBroadcastAndRandomNested() throws Exception {
-        String sql = "" +
-            "SELECT * FROM broadcast_tbl1 EXCEPT (" +
-            "   SELECT * FROM random_tbl1 " +
-            "   EXCEPT " +
-            "   SELECT * FROM random_tbl2" +
-            ")";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteSingleMinus.class)
-            .and(input(0, isTableScan("broadcast_tbl1")))
-            .and(input(1, isInstanceOf(IgniteReduceMinus.class)
-                .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                    .and(input(0, isTableScan("random_tbl1")))
-                    .and(input(1, isTableScan("random_tbl2")))
-                ))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptMerge() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM random_tbl2 " +
-            "EXCEPT " +
-            "SELECT * FROM affinity_tbl1 " +
-            "EXCEPT " +
-            "SELECT * FROM affinity_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class)
-            .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                .and(input(0, isTableScan("random_tbl1")))
-                .and(input(1, isTableScan("random_tbl2")))
-                .and(input(2, isTableScan("affinity_tbl1")))
-                .and(input(3, isTableScan("affinity_tbl2")))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptAllMerge() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl1 " +
-            "EXCEPT ALL " +
-            "SELECT * FROM random_tbl2 " +
-            "EXCEPT ALL " +
-            "SELECT * FROM affinity_tbl1 " +
-            "EXCEPT ALL " +
-            "SELECT * FROM affinity_tbl2 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> n.all)
-            .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                .and(input(0, isTableScan("random_tbl1")))
-                .and(input(1, isTableScan("random_tbl2")))
-                .and(input(2, isTableScan("affinity_tbl1")))
-                .and(input(3, isTableScan("affinity_tbl2")))
-            ))
-        );
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testExceptAllWithExceptMerge() throws Exception {
-        String sql = "" +
-            "SELECT * FROM random_tbl1 " +
-            "EXCEPT ALL " +
-            "SELECT * FROM random_tbl2 " +
-            "EXCEPT " +
-            "SELECT * FROM affinity_tbl1 ";
-
-        assertPlan(sql, publicSchema, isInstanceOf(IgniteReduceMinus.class).and(n -> !n.all)
-            .and(hasChildThat(isInstanceOf(IgniteMapMinus.class)
-                .and(input(0, isTableScan("random_tbl1")))
-                .and(input(1, isTableScan("random_tbl2")))
-                .and(input(2, isTableScan("affinity_tbl1")))
-            ))
-        );
-    }
-}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java
index a0115b4..1001446 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java
@@ -87,14 +87,14 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
  */
 public class PlannerTest extends AbstractPlannerTest {
     /** */
-    private static List<UUID> NODES;
+    private static List<String> NODES;
     
     @BeforeAll
     public static void init() {
         NODES = new ArrayList<>(4);
 
         for (int i = 0; i < 4; i++)
-            NODES.add(UUID.randomUUID());
+            NODES.add(UUID.randomUUID().toString());
     }
 
     /**
@@ -1840,7 +1840,7 @@ public class PlannerTest extends AbstractPlannerTest {
     }
 
     /** */
-    private List<UUID> intermediateMapping(long topVer, boolean single,
+    private List<String> intermediateMapping(long topVer, boolean single,
         @Nullable Predicate<ClusterNode> filter) {
         return single ? select(NODES, 0) : select(NODES, 0, 1, 2, 3);
     }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SetOpPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SetOpPlannerTest.java
new file mode 100644
index 0000000..3559492
--- /dev/null
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/SetOpPlannerTest.java
@@ -0,0 +1,421 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.planner;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapIntersect;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteMapSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceIntersect;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteReduceSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSetOp;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleIntersect;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleMinus;
+import org.apache.ignite.internal.processors.query.calcite.rel.set.IgniteSingleSetOp;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+/**
+ * Test to verify set op (EXCEPT, INTERSECT).
+ */
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public class SetOpPlannerTest extends AbstractPlannerTest {
+    /** Public schema. */
+    private IgniteSchema publicSchema;
+
+    /** */
+    @BeforeAll
+    public void setup() {
+        publicSchema = new IgniteSchema("PUBLIC");
+
+        IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
+
+        RelDataType type = new RelDataTypeFactory.Builder(f)
+            .add("ID", f.createJavaType(Integer.class))
+            .add("NAME", f.createJavaType(String.class))
+            .add("SALARY", f.createJavaType(Double.class))
+            .build();
+
+        createTable(publicSchema, "RANDOM_TBL1", type, IgniteDistributions.random());
+        createTable(publicSchema, "RANDOM_TBL2", type, IgniteDistributions.random());
+        createTable(publicSchema, "BROADCAST_TBL1", type, IgniteDistributions.broadcast());
+        createTable(publicSchema, "BROADCAST_TBL2", type, IgniteDistributions.broadcast());
+        createTable(publicSchema, "SINGLE_TBL1", type, IgniteDistributions.single());
+        createTable(publicSchema, "SINGLE_TBL2", type, IgniteDistributions.single());
+
+        createTable(publicSchema, "AFFINITY_TBL1", type,
+            IgniteDistributions.affinity(0, "Test1", "hash"));
+
+        createTable(publicSchema, "AFFINITY_TBL2", type,
+            IgniteDistributions.affinity(0, "Test2", "hash"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpRandom(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM random_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(n -> !n.all())
+            .and(hasChildThat(isInstanceOf(setOp.map)
+                .and(input(0, isTableScan("random_tbl1")))
+                .and(input(1, isTableScan("random_tbl2")))
+            ))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpAllRandom(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl1 " +
+                setOpAll(setOp) +
+                "SELECT * FROM random_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(IgniteSetOp::all)
+            .and(hasChildThat(isInstanceOf(setOp.map)
+                .and(input(0, isTableScan("random_tbl1")))
+                .and(input(1, isTableScan("random_tbl2")))
+            ))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpBroadcast(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM broadcast_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM broadcast_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+            .and(input(0, isTableScan("broadcast_tbl1")))
+            .and(input(1, isTableScan("broadcast_tbl2")))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpSingle(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM single_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM single_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+            .and(input(0, isTableScan("single_tbl1")))
+            .and(input(1, isTableScan("single_tbl2"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpSingleAndRandom(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM single_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM random_tbl1 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+            .and(input(0, isTableScan("single_tbl1")))
+            .and(input(1, hasChildThat(isTableScan("random_tbl1")))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpSingleAndAffinity(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM single_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM affinity_tbl1 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+            .and(input(0, isTableScan("single_tbl1")))
+            .and(input(1, hasChildThat(isTableScan("affinity_tbl1")))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpSingleAndBroadcast(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM single_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM broadcast_tbl1 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+            .and(input(0, isTableScan("single_tbl1")))
+            .and(input(1, isTableScan("broadcast_tbl1")))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpAffinity(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM affinity_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM affinity_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
+            .and(hasChildThat(isInstanceOf(setOp.map)
+                .and(input(0, isTableScan("affinity_tbl1")))
+                .and(input(1, isTableScan("affinity_tbl2")))
+            ))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpBroadcastAndRandom(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM broadcast_tbl1 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+            .and(input(0, hasChildThat(isTableScan("random_tbl1"))))
+            .and(input(1, isTableScan("broadcast_tbl1")))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpRandomNested(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl2 " + setOp(setOp) + "(" +
+                "   SELECT * FROM random_tbl1 " +
+                setOp(setOp) +
+                "   SELECT * FROM random_tbl2" +
+                ")";
+
+        if (setOp == SetOp.EXCEPT) {
+            assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+                .and(input(0, hasChildThat(isTableScan("random_tbl2"))))
+                .and(input(1, isInstanceOf(setOp.reduce)
+                    .and(hasChildThat(isInstanceOf(setOp.map)
+                        .and(input(0, isTableScan("random_tbl1")))
+                        .and(input(1, isTableScan("random_tbl2")))
+                    ))
+                ))
+            );
+        }
+        else {
+            // INTERSECT operator is commutative and can be merged.
+            assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
+                .and(hasChildThat(isInstanceOf(setOp.map)
+                    .and(input(0, isTableScan("random_tbl2")))
+                    .and(input(1, isTableScan("random_tbl1")))
+                    .and(input(2, isTableScan("random_tbl2")))
+                ))
+            );
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpBroadcastAndRandomNested(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM broadcast_tbl1 " + setOp(setOp) + "(" +
+                "   SELECT * FROM random_tbl1 " +
+                setOp(setOp) +
+                "   SELECT * FROM random_tbl2" +
+                ")";
+
+        if (setOp == SetOp.EXCEPT) {
+            assertPlan(sql, publicSchema, isInstanceOf(setOp.single)
+                .and(input(0, isTableScan("broadcast_tbl1")))
+                .and(input(1, isInstanceOf(setOp.reduce)
+                    .and(hasChildThat(isInstanceOf(setOp.map)
+                        .and(input(0, isTableScan("random_tbl1")))
+                        .and(input(1, isTableScan("random_tbl2")))
+                    ))
+                ))
+            );
+        }
+        else {
+            // INTERSECT operator is commutative and can be merged.
+            assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
+                .and(hasChildThat(isInstanceOf(setOp.map)
+                    .and(input(0, nodeOrAnyChild(isTableScan("broadcast_tbl1"))))
+                    .and(input(1, isTableScan("random_tbl1")))
+                    .and(input(2, isTableScan("random_tbl2")))
+                ))
+            );
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpMerge(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM random_tbl2 " +
+                setOp(setOp) +
+                "SELECT * FROM affinity_tbl1 " +
+                setOp(setOp) +
+                "SELECT * FROM affinity_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce)
+            .and(hasChildThat(isInstanceOf(setOp.map)
+                .and(input(0, isTableScan("random_tbl1")))
+                .and(input(1, isTableScan("random_tbl2")))
+                .and(input(2, isTableScan("affinity_tbl1")))
+                .and(input(3, isTableScan("affinity_tbl2")))
+            ))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpAllMerge(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl1 " +
+                setOpAll(setOp) +
+                "SELECT * FROM random_tbl2 " +
+                setOpAll(setOp) +
+                "SELECT * FROM affinity_tbl1 " +
+                setOpAll(setOp) +
+                "SELECT * FROM affinity_tbl2 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(IgniteSetOp::all)
+            .and(hasChildThat(isInstanceOf(setOp.map)
+                .and(input(0, isTableScan("random_tbl1")))
+                .and(input(1, isTableScan("random_tbl2")))
+                .and(input(2, isTableScan("affinity_tbl1")))
+                .and(input(3, isTableScan("affinity_tbl2")))
+            ))
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void testSetOpAllWithExceptMerge(SetOp setOp) throws Exception {
+        String sql =
+            "SELECT * FROM random_tbl1 " +
+                setOpAll(setOp) +
+                "SELECT * FROM random_tbl2 " +
+                setOp(setOp) +
+                "SELECT * FROM affinity_tbl1 ";
+
+        assertPlan(sql, publicSchema, isInstanceOf(setOp.reduce).and(n -> !n.all())
+            .and(hasChildThat(isInstanceOf(setOp.map)
+                .and(input(0, isTableScan("random_tbl1")))
+                .and(input(1, isTableScan("random_tbl2")))
+                .and(input(2, isTableScan("affinity_tbl1")))
+            ))
+        );
+    }
+
+    /** */
+    private String setOp(SetOp setOp) {
+        return setOp.name() + ' ';
+    }
+
+    /** */
+    private String setOpAll(SetOp setOp) {
+        return setOp.name() + " ALL ";
+    }
+
+    /** */
+    enum SetOp {
+        /** */
+        EXCEPT(
+            IgniteSingleMinus.class,
+            IgniteMapMinus.class,
+            IgniteReduceMinus.class
+        ),
+
+        /** */
+        INTERSECT(
+            IgniteSingleIntersect.class,
+            IgniteMapIntersect.class,
+            IgniteReduceIntersect.class
+        );
+
+        /** */
+        public final Class<? extends IgniteSingleSetOp> single;
+
+        /** */
+        public final Class<? extends IgniteMapSetOp> map;
+
+        /** */
+        public final Class<? extends IgniteReduceSetOp> reduce;
+
+        /** */
+        SetOp(
+            Class<? extends IgniteSingleSetOp> single,
+            Class<? extends IgniteMapSetOp> map,
+            Class<? extends IgniteReduceSetOp> reduce) {
+            this.single = single;
+            this.map = map;
+            this.reduce = reduce;
+        }
+    }
+}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionPlannerTest.java
similarity index 98%
rename from modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionTest.java
rename to modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionPlannerTest.java
index e6f1b04..c2c9dc3 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/TableFunctionPlannerTest.java
@@ -35,7 +35,7 @@ import org.junit.jupiter.api.TestInstance;
  * Test table functions.
  */
 @TestInstance(TestInstance.Lifecycle.PER_CLASS)
-public class TableFunctionTest extends AbstractPlannerTest {
+public class TableFunctionPlannerTest extends AbstractPlannerTest {
     /** Public schema. */
     private IgniteSchema publicSchema;
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/core/src/main/java/org/apache/ignite/internal/util/Cancellable.java
similarity index 65%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/core/src/main/java/org/apache/ignite/internal/util/Cancellable.java
index 093146c..84397cc 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/Cancellable.java
@@ -14,21 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
+package org.apache.ignite.internal.util;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
-
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
-
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ * A {@code Cancellable} represents a process or an operation that can be
+ * canceled.
+ */
+public interface Cancellable {
+    /**
+     * Cancels the ongoing operation or process or do nothing if it has been
+     * already cancelled.
+     */
+    void cancel();
 }
diff --git a/modules/runner/pom.xml b/modules/runner/pom.xml
index 711ec38..c0f82b8 100644
--- a/modules/runner/pom.xml
+++ b/modules/runner/pom.xml
@@ -84,6 +84,11 @@
             <artifactId>ignite-table</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-calcite</artifactId>
+        </dependency>
+
         <!-- 3rd party dependencies -->
         <dependency>
             <groupId>org.slf4j</groupId>
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 5f276d4..54a93b9 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.app;
 
 import org.apache.ignite.app.Ignite;
 import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.internal.processors.query.calcite.SqlQueryProcessor;
 import org.apache.ignite.table.manager.IgniteTables;
 import org.apache.ignite.tx.IgniteTransactions;
 
@@ -32,13 +33,17 @@ public class IgniteImpl implements Ignite {
     /** Vault manager */
     private final VaultManager vaultManager;
 
+    private final SqlQueryProcessor qryEngine;
+
     /**
      * @param tableManager Table manager.
      * @param vaultManager Vault manager.
+     * @param qryEngine Query processor.
      */
-    IgniteImpl(IgniteTables tableManager, VaultManager vaultManager) {
+    IgniteImpl(IgniteTables tableManager, VaultManager vaultManager, SqlQueryProcessor qryEngine) {
         this.distributedTableManager = tableManager;
         this.vaultManager = vaultManager;
+        this.qryEngine = qryEngine;
     }
 
     /** {@inheritDoc} */
@@ -46,6 +51,10 @@ public class IgniteImpl implements Ignite {
         return distributedTableManager;
     }
 
+    public SqlQueryProcessor queryEngine() {
+        return qryEngine;
+    }
+
     /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
         return null;
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
index bf6f300..a3face6 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgnitionImpl.java
@@ -28,6 +28,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
+
 import org.apache.ignite.app.Ignite;
 import org.apache.ignite.app.Ignition;
 import org.apache.ignite.configuration.RootKey;
@@ -42,6 +43,7 @@ import org.apache.ignite.internal.baseline.BaselineManager;
 import org.apache.ignite.internal.configuration.ConfigurationManager;
 import org.apache.ignite.internal.configuration.storage.ConfigurationStorage;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.processors.query.calcite.SqlQueryProcessor;
 import org.apache.ignite.internal.raft.Loza;
 import org.apache.ignite.internal.schema.SchemaManager;
 import org.apache.ignite.internal.storage.DistributedConfigurationStorage;
@@ -58,7 +60,6 @@ import org.apache.ignite.network.ClusterService;
 import org.apache.ignite.network.MessageSerializationRegistryImpl;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.scalecube.ScaleCubeClusterServiceFactory;
-import org.apache.ignite.table.manager.IgniteTables;
 import org.apache.ignite.utils.IgniteProperties;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -220,7 +221,7 @@ public class IgnitionImpl implements Ignition {
         SchemaManager schemaMgr = new SchemaManager(configurationMgr, metaStorageMgr, vaultMgr);
 
         // Distributed table manager startup.
-        IgniteTables distributedTblMgr = new TableManager(
+        TableManager distributedTblMgr = new TableManager(
             configurationMgr,
             metaStorageMgr,
             schemaMgr,
@@ -229,6 +230,11 @@ public class IgnitionImpl implements Ignition {
             vaultMgr
         );
 
+        SqlQueryProcessor qryProc = new SqlQueryProcessor(
+            clusterNetSvc,
+            distributedTblMgr
+        );
+
         // TODO IGNITE-14579 Start rest manager.
 
         // Deploy all resisted watches cause all components are ready and have registered their listeners.
@@ -236,7 +242,7 @@ public class IgnitionImpl implements Ignition {
 
         ackSuccessStart();
 
-        return new IgniteImpl(distributedTblMgr, vaultMgr);
+        return new IgniteImpl(distributedTblMgr, vaultMgr, qryProc);
     }
 
     /**
diff --git a/modules/schema/README.md b/modules/schema/README.md
index 7ec07c2..f9f6742 100644
--- a/modules/schema/README.md
+++ b/modules/schema/README.md
@@ -3,7 +3,7 @@
 This module provides implementation for schema management components:
 
 * Public API for schema definition and evolution
-* Schema manager component that implements necessary machinary to translate schema management commands to corresponding
+* Schema manager component that implements necessary machinery to translate schema management commands to corresponding
   metastorage modifications, as well as schema modification event processing logic 
 * Necessary logic to build and upgrade rows of specific schema that encode user data in schema-defined format.
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java b/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
index e35068e..e146f3c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
@@ -18,9 +18,9 @@
 package org.apache.ignite.internal.table.event;
 
 import java.util.UUID;
+
 import org.apache.ignite.internal.manager.EventParameters;
 import org.apache.ignite.internal.table.TableImpl;
-import org.apache.ignite.table.Table;
 
 /**
  * Table event parameters. There are properties which associate with a concrete table.
@@ -84,7 +84,7 @@ public class TableEventParameters implements EventParameters {
      *
      * @return Table.
      */
-    public Table table() {
+    public TableImpl table() {
         return table;
     }
 }
diff --git a/parent/pom.xml b/parent/pom.xml
index ad90c1b..0130ec1 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -150,6 +150,12 @@
 
             <dependency>
                 <groupId>org.apache.ignite</groupId>
+                <artifactId>ignite-calcite</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.apache.ignite</groupId>
                 <artifactId>ignite-cli-common</artifactId>
                 <version>${project.version}</version>
             </dependency>