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

[ignite] branch ignite-12248 updated: IGNITE-12708: Calcite integration. Expressions factory base implementation. This closes #7489

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

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


The following commit(s) were added to refs/heads/ignite-12248 by this push:
     new 98f3950  IGNITE-12708: Calcite integration. Expressions factory base implementation. This closes #7489
98f3950 is described below

commit 98f39505099ffa63b5556dbef577e14d043b682f
Author: Igor Seliverstov <gv...@gmail.com>
AuthorDate: Tue Mar 10 13:37:44 2020 +0300

    IGNITE-12708: Calcite integration. Expressions factory base implementation. This closes #7489
---
 .../calcite/interpreter/InterpreterUtils.java      |  34 --
 .../query/calcite/CalciteQueryProcessor.java       |   5 +
 .../query/calcite/exec/ExchangeService.java        |  19 +-
 .../query/calcite/exec/ExchangeServiceImpl.java    |  30 +-
 .../query/calcite/exec/ExecutionContext.java       |   5 +-
 .../query/calcite/exec/ExecutionServiceImpl.java   |  66 +--
 .../query/calcite/exec/ExpressionFactory.java      | 408 -----------------
 .../processors/query/calcite/exec/FilterNode.java  |  58 ---
 .../processors/query/calcite/exec/JoinNode.java    | 130 ------
 ...Implementor.java => LogicalRelImplementor.java} |  68 ++-
 .../query/calcite/exec/MailboxRegistry.java        |   2 +
 .../query/calcite/exec/MailboxRegistryImpl.java    |   2 +
 .../query/calcite/exec/PhysicalRelImplementor.java | 175 +++++++
 .../processors/query/calcite/exec/ScanNode.java    | 113 -----
 .../processors/query/calcite/exec/exp/Call.java    | 117 +++++
 .../exp/CallOperation.java}                        |   9 +-
 .../exp/DynamicParam.java}                         |  28 +-
 .../query/calcite/exec/exp/ExpToRexTranslator.java |  97 ++++
 .../DataType.java => exec/exp/Expression.java}     |  37 +-
 .../query/calcite/exec/exp/ExpressionFactory.java  | 502 +++++++++++++++++++++
 .../exp/ExpressionVisitor.java}                    |  29 +-
 .../exp/InputRef.java}                             |  28 +-
 .../exp/Literal.java}                              |  32 +-
 .../exp}/RexToExpTranslator.java                   |  45 +-
 .../{type/SystemType.java => exec/exp/Scalar.java} |   9 +-
 .../query/calcite/exec/exp/type/BasicType.java     | 153 +++++++
 .../exp/type/CharacterDataType.java}               |  19 +-
 .../query/calcite/exec/exp/type/CharacterType.java | 143 ++++++
 .../query/calcite/exec/exp/type/DataType.java      |  89 ++++
 .../query/calcite/exec/exp/type/JavaType.java      |  84 ++++
 .../query/calcite/exec/exp/type/StructType.java    | 101 +++++
 .../query/calcite/exec/{ => rel}/AbstractNode.java |  92 ++--
 .../exec/{Sink.java => rel/Downstream.java}        |  15 +-
 .../query/calcite/exec/rel/FilterNode.java         | 145 ++++++
 .../query/calcite/exec/{ => rel}/Inbox.java        | 198 ++++----
 .../query/calcite/exec/rel/JoinNode.java           | 226 ++++++++++
 .../query/calcite/exec/{ => rel}/ModifyNode.java   | 197 ++++----
 .../query/calcite/exec/{ => rel}/Node.java         |  48 +-
 .../query/calcite/exec/{ => rel}/Outbox.java       | 216 +++++----
 .../query/calcite/exec/{ => rel}/ProjectNode.java  |  60 ++-
 .../exec/{ConsumerNode.java => rel/RootNode.java}  | 192 ++++----
 .../query/calcite/exec/rel/ScanNode.java           | 131 ++++++
 .../query/calcite/exec/{ => rel}/SingleNode.java   |  19 +-
 .../query/calcite/message/QueryStartRequest.java   |  26 +-
 .../query/calcite/prepare/IgnitePlanner.java       |  61 +--
 .../query/calcite/prepare/IgniteSqlValidator.java  |  35 +-
 .../query/calcite/prepare/PlannerPhase.java        |   2 -
 .../query/calcite/prepare/PlanningContext.java     |  16 +
 .../processors/query/calcite/prepare/Splitter.java |   7 +-
 .../query/calcite/rule/TableConverter.java         |  48 --
 .../query/calcite/schema/TableDescriptorImpl.java  |  16 +-
 .../query/calcite/serialize/FilterPhysicalRel.java |  84 ++++
 .../query/calcite/serialize/JoinPhysicalRel.java   |  94 ++++
 .../Expression.java => PhysicalRel.java}           |  16 +-
 ...ExpImplementor.java => PhysicalRelVisitor.java} |  49 +-
 .../calcite/serialize/ProjectPhysicalRel.java      |  99 ++++
 .../calcite/serialize/ReceiverPhysicalRel.java     | 153 +++++++
 .../calcite/serialize/RelToPhysicalConverter.java  | 132 ++++++
 .../query/calcite/serialize/SenderPhysicalRel.java |  98 ++++
 .../calcite/serialize/TableModifyPhysicalRel.java  | 115 +++++
 ...ableScanNode.java => TableScanPhysicalRel.java} |  58 ++-
 .../query/calcite/serialize/ValuesPhysicalRel.java |  79 ++++
 .../serialize/expression/CallExpression.java       |  87 ----
 .../serialize/expression/ExpToRexTranslator.java   | 117 -----
 .../serialize/expression/LocalRefExpression.java   |  61 ---
 .../serialize/relation/ConversionContext.java      |  49 --
 .../calcite/serialize/relation/FilterNode.java     |  66 ---
 .../serialize/relation/GraphToRelConverter.java    | 111 -----
 .../query/calcite/serialize/relation/JoinNode.java |  87 ----
 .../calcite/serialize/relation/ProjectNode.java    |  74 ---
 .../calcite/serialize/relation/ReceiverNode.java   |  68 ---
 .../query/calcite/serialize/relation/RelGraph.java | 114 -----
 .../calcite/serialize/relation/RelGraphNode.java   |  91 ----
 .../serialize/relation/RelToGraphConverter.java    | 142 ------
 .../calcite/serialize/relation/SenderNode.java     |  63 ---
 .../serialize/relation/TableModifyNode.java        |  94 ----
 .../calcite/serialize/relation/ValuesNode.java     |  93 ----
 .../query/calcite/serialize/type/Types.java        | 187 --------
 .../query/calcite/trait/DistributionTrait.java     |  27 +-
 .../query/calcite/type/IgniteTypeFactory.java      |  75 ---
 .../processors/query/calcite/util/Commons.java     |  27 ++
 .../query/calcite/util/IgniteMethod.java           |   5 +
 .../SingleNode.java => util/IgniteResource.java}   |  30 +-
 .../query/calcite/CalciteQueryProcessorTest.java   |   2 +
 .../processors/query/calcite/PlannerTest.java      | 313 ++++++++-----
 .../calcite/exec/ClosableIteratorsHolderTest.java  |   3 +
 .../calcite/exec/ExpressionEvaluationTest.java     |  73 +++
 .../processors/query/calcite/exec/OutboxTest.java  | 179 --------
 .../exec/{ => rel}/AbstractExecutionTest.java      |   9 +-
 .../exec/{ => rel}/ContinuousExecutionTest.java    |  30 +-
 .../calcite/exec/{ => rel}/ExecutionTest.java      |  18 +-
 .../query/calcite/jdbc/JdbcQueryTest.java          |   6 +-
 .../ignite/testsuites/IgniteCalciteTestSuite.java  |   8 +-
 93 files changed, 4141 insertions(+), 3532 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java b/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java
deleted file mode 100644
index 271ef50..0000000
--- a/modules/calcite/src/main/java/org/apache/calcite/interpreter/InterpreterUtils.java
+++ /dev/null
@@ -1,34 +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.calcite.interpreter;
-
-import org.apache.calcite.DataContext;
-
-/**
- * Utility methods to exploit package private API.
- */
-public class InterpreterUtils {
-    /**
-     * Creates a Interpreter context.
-     * @param ctx Runtime context allowing access to the tables in a database and query parameters.
-     * @return Interpreter context.
-     */
-    public static Context createContext(DataContext ctx) {
-        return new Context(ctx);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
index 27a092a..23e3ff9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
@@ -24,6 +24,7 @@ import org.apache.calcite.plan.RelOptCostImpl;
 import org.apache.calcite.sql.fun.SqlLibrary;
 import org.apache.calcite.sql.fun.SqlLibraryOperatorTableFactory;
 import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
@@ -62,6 +63,10 @@ import org.jetbrains.annotations.Nullable;
 public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
     /** */
     public static final FrameworkConfig FRAMEWORK_CONFIG = Frameworks.newConfigBuilder()
+            .sqlToRelConverterConfig(SqlToRelConverter.configBuilder()
+                .withConvertTableAccess(true)
+                .withTrimUnusedFields(false)
+                .build())
             .parserConfig(SqlParser.configBuilder()
                 // Lexical configuration defines how identifiers are quoted, whether they are converted to upper or lower
                 // case when they are read, and whether identifiers are matched case-sensitively.
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
index 4f9a05e..5600f9c 100644
--- 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
@@ -19,22 +19,15 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import java.util.List;
 import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
  *
  */
 public interface ExchangeService extends Service {
-    /** A number of rows in a single batch. */
-    int BATCH_SIZE = 200;
-
-    /** A maximum allowed unprocessed batches count per node. */
-    int PER_NODE_BATCH_COUNT = 10;
-
     /**
      * Sends a batch of data to remote node.
-     *
-     * @param caller Caller.
      * @param nodeId Target node ID.
      * @param queryId Query ID.
      * @param fragmentId Target fragment ID.
@@ -42,29 +35,25 @@ public interface ExchangeService extends Service {
      * @param batchId Batch ID.
      * @param rows Data rows.
      */
-    void sendBatch(Outbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows);
+    void sendBatch(UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) throws IgniteCheckedException;
 
     /**
      * Acknowledges a batch with given ID is processed.
-     *
-     * @param caller Caller.
      * @param nodeId Node ID to notify.
      * @param queryId Query ID.
      * @param fragmentId Target fragment ID.
      * @param exchangeId Exchange ID.
      * @param batchId Batch ID.
      */
-    void acknowledge(Inbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId);
+    void acknowledge(UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) throws IgniteCheckedException;
 
     /**
      * Sends cancel request.
-     *
-     * @param caller Caller.
      * @param nodeId Target node ID.
      * @param queryId Query ID.
      * @param fragmentId Target fragment ID.
      * @param exchangeId Exchange ID.
      * @param batchId Batch ID.
      */
-    void cancel(Outbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId);
+    void cancel(UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) throws IgniteCheckedException;
 }
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
index b94940b..bac38b0 100644
--- 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
@@ -24,6 +24,8 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+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.InboxCancelMessage;
 import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
 import org.apache.ignite.internal.processors.query.calcite.message.MessageType;
@@ -96,32 +98,18 @@ public class ExchangeServiceImpl extends AbstractService implements ExchangeServ
     }
 
     /** {@inheritDoc} */
-    @Override public void sendBatch(Outbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) {
-        try {
-            messageService().send(nodeId, new QueryBatchMessage(queryId, fragmentId, exchangeId, batchId, rows));
-        }
-        catch (IgniteCheckedException e) {
-            caller.cancel();
-        }
+    @Override public void sendBatch(UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) throws IgniteCheckedException {
+        messageService().send(nodeId, new QueryBatchMessage(queryId, fragmentId, exchangeId, batchId, rows));
     }
 
     /** {@inheritDoc} */
-    @Override public void acknowledge(Inbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
-        try {
-            messageService().send(nodeId, new QueryBatchAcknowledgeMessage(queryId, fragmentId, exchangeId, batchId));
-        }
-        catch (IgniteCheckedException e) {
-            caller.cancel();
-        }
+    @Override public void acknowledge(UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) throws IgniteCheckedException {
+        messageService().send(nodeId, new QueryBatchAcknowledgeMessage(queryId, fragmentId, exchangeId, batchId));
     }
 
     /** {@inheritDoc} */
-    @Override public void cancel(Outbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
-        try {
-            messageService().send(nodeId, new InboxCancelMessage(queryId, fragmentId, exchangeId, batchId));
-        }
-        catch (IgniteCheckedException ignored) {
-        }
+    @Override public void cancel(UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) throws IgniteCheckedException {
+        messageService().send(nodeId, new InboxCancelMessage(queryId, fragmentId, exchangeId, batchId));
     }
 
     /** {@inheritDoc} */
@@ -181,7 +169,7 @@ public class ExchangeServiceImpl extends AbstractService implements ExchangeServ
         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 = mailboxRegistry().register(new Inbox<>(this, mailboxRegistry(), baseInboxContext(msg.queryId(), msg.fragmentId()), msg.exchangeId(), msg.exchangeId()));
+            inbox = mailboxRegistry().register(new Inbox<>(baseInboxContext(msg.queryId(), msg.fragmentId()), this, mailboxRegistry(), msg.exchangeId(), msg.exchangeId()));
 
         if (inbox != null)
             inbox.onBatchReceived(nodeId, msg.batchId(), msg.rows());
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 b967f3f..6381d7f 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
@@ -146,8 +146,9 @@ public class ExecutionContext implements DataContext {
     /**
      * Sets cancelled flag.
      */
-    public void setCancelled() {
-        cancelled = true;
+    public void markCancelled() {
+        if (!cancelled)
+            cancelled = true;
     }
 
     /**
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index 18beb95..353bda9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -62,6 +62,9 @@ import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryCancellable;
 import org.apache.ignite.internal.processors.query.QueryContext;
 import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+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.MessageService;
 import org.apache.ignite.internal.processors.query.calcite.message.MessageType;
 import org.apache.ignite.internal.processors.query.calcite.message.QueryCancelRequest;
@@ -86,12 +89,9 @@ 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.rel.IgniteConvention;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
-import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
 import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.SenderNode;
+import org.apache.ignite.internal.processors.query.calcite.serialize.RelToPhysicalConverter;
+import org.apache.ignite.internal.processors.query.calcite.serialize.SenderPhysicalRel;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
@@ -616,21 +616,18 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
         ExecutionContext ectx = new ExecutionContext(taskExecutor(), pctx, queryId, local.fragmentId(),
             local.mapping().partitions(pctx.localNodeId()), Commons.parametersMap(pctx.parameters()));
 
-        Node<Object[]> node = new Implementor(partitionService(), mailboxRegistry(), exchangeService(), failureProcessor(), ectx, log).go(local.root());
+        Node<Object[]> node = new LogicalRelImplementor(ectx, partitionService(), mailboxRegistry(), exchangeService(), failureProcessor()).go(local.root());
 
-        assert !(node instanceof SenderNode);
+        assert !(node instanceof SenderPhysicalRel);
 
         QueryInfo info = new QueryInfo(ectx, fragments, node);
 
         // register query
         register(info);
 
-        // start local execution
-        info.consumer.request();
-
         // start remote execution
         if (fragments.size() > 1) {
-            RelOp<IgniteRel, RelGraph> converter = new RelToGraphConverter();
+            RelToPhysicalConverter converter = new RelToPhysicalConverter(ectx.getTypeFactory());
 
             for (int i = 1; i < fragments.size(); i++) {
                 Fragment fragment = fragments.get(i);
@@ -734,11 +731,7 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
 
         PlanningContext ctx = createContext(msg.schema(), nodeId, msg.topologyVersion());
         RelMetadataQuery.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(IgniteMetadata.METADATA_PROVIDER));
-        try (IgnitePlanner planner = ctx.planner()) {
-            IgniteRel root = planner.convert(msg.plan());
-
-            assert root instanceof IgniteSender : root;
-
+        try {
             ExecutionContext execCtx = new ExecutionContext(
                 taskExecutor(),
                 ctx,
@@ -748,20 +741,23 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
                 Commons.parametersMap(msg.parameters())
             );
 
-            Node<Object[]> node = new Implementor(partitionService(), mailboxRegistry(), exchangeService(), failureProcessor(), execCtx, log).go(root);
+            Node<Object[]> node = new PhysicalRelImplementor(execCtx, partitionService(),
+                mailboxRegistry(), exchangeService(), failureProcessor()).go(msg.root());
 
             assert node instanceof Outbox : node;
 
-            node.context().execute(node::request);
+            node.context().execute(((Outbox<Object[]>) node)::init);
 
             messageService().send(nodeId, new QueryStartResponse(msg.queryId(), msg.fragmentId()));
         }
-        catch (Exception ex) {
+        catch (Throwable ex) { // TODO don't catch errors!
             cancelQuery(msg.queryId());
 
             if (ex instanceof ClusterTopologyCheckedException)
                 return;
 
+            U.warn(log, "Failed to start query. [nodeId=" + nodeId + ']', ex);
+
             try {
                 messageService().send(nodeId, new QueryStartResponse(msg.queryId(), msg.fragmentId(), ex));
             }
@@ -770,6 +766,9 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
 
                 U.warn(log, "Failed to send reply. [nodeId=" + nodeId + ']', e);
             }
+
+            if (ex instanceof Error)
+                throw (Error)ex;
         }
         finally {
             RelMetadataQuery.THREAD_PROVIDERS.remove();
@@ -794,11 +793,19 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
     }
 
     /** */
-    private void onConsumerClose(ConsumerNode consumer) {
-        if (consumer.canceled())
-            cancelQuery(consumer.queryId());
-        else
-            running.remove(consumer.queryId());
+    private void onCursorClose(RootNode rootNode) {
+        switch (rootNode.state()) {
+            case CANCELLED:
+                cancelQuery(rootNode.queryId());
+
+                break;
+            case END:
+                running.remove(rootNode.queryId());
+
+                break;
+            default:
+                throw new AssertionError();
+        }
     }
 
     /** */
@@ -866,7 +873,7 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
         private final ExecutionContext ctx;
 
         /** */
-        private final ConsumerNode consumer;
+        private final RootNode root;
 
         /** remote nodes */
         private final Set<UUID> remotes;
@@ -884,7 +891,10 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
         private QueryInfo(ExecutionContext ctx, List<Fragment> fragments, Node<Object[]> root) {
             this.ctx = ctx;
 
-            consumer = new ConsumerNode(ctx, root, ExecutionServiceImpl.this::onConsumerClose);
+            RootNode rootNode = new RootNode(ctx, ExecutionServiceImpl.this::onCursorClose);
+            rootNode.register(root);
+
+            this.root = rootNode;
 
             remotes = new HashSet<>();
             waiting = new HashSet<>();
@@ -905,7 +915,7 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
 
         /** */
         public Iterator<Object[]> iterator() {
-            return iteratorsHolder().iterator(consumer);
+            return iteratorsHolder().iterator(root);
         }
 
         /** {@inheritDoc} */
@@ -955,7 +965,7 @@ public class ExecutionServiceImpl extends AbstractService implements ExecutionSe
             }
 
             if (cancelLocal)
-                consumer.cancel();
+                root.cancel();
 
             if (cancelRemote) {
                 QueryCancelRequest msg = new QueryCancelRequest(ctx.queryId());
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExpressionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExpressionFactory.java
deleted file mode 100644
index 32e35fb..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExpressionFactory.java
+++ /dev/null
@@ -1,408 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.exec;
-
-import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.InterpreterUtils;
-import org.apache.calcite.interpreter.JaninoRexCompiler;
-import org.apache.calcite.interpreter.Scalar;
-import org.apache.calcite.rel.RelCollation;
-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.commons.lang.text.StrBuilder;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.failure.FailureContext;
-import org.apache.ignite.failure.FailureType;
-import org.apache.ignite.internal.processors.failure.FailureProcessor;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Implements rex expression into a function object. Uses JaninoRexCompiler under the hood.
- * Each expression compiles into a class and a wrapper over it is returned.
- */
-public class ExpressionFactory {
-    /** */
-    private static final int CACHE_SIZE = 1024;
-
-    /** */
-    private static final Map<String, Scalar> CACHE = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
-
-    /** */
-    private final IgniteTypeFactory typeFactory;
-
-    /** */
-    private final JaninoRexCompiler rexCompiler;
-
-    /** */
-    private final ExceptionHandler handler;
-
-    /**
-     * @param typeFactory Type factory.
-     * @param failure Failure processor.
-     * @param log Logger.
-     */
-    public ExpressionFactory(IgniteTypeFactory typeFactory, FailureProcessor failure, IgniteLogger log) {
-        this.typeFactory = typeFactory;
-
-        rexCompiler = new JaninoRexCompiler(new RexBuilder(typeFactory));
-        handler = new ExceptionHandler(failure, log.getLogger(ExpressionFactory.class));
-    }
-
-    /**
-     * Creates a comparator for given data type and collations. Mainly used for sorted exchange.
-     *
-     * @param root Execution context, holds a planner context, query and its parameters,
-     *             execution specific variables (like queryId, current user, session, etc).
-     * @param collations Collations.
-     * @param rowType Input row type.
-     * @return Row comparator.
-     */
-    public <T> Comparator<T> comparator(ExecutionContext root, List<RelCollation> collations, RelDataType rowType) {
-        return null; // TODO
-    }
-
-    /**
-     * Creates a Filter predicate.
-     * @param root Execution context, holds a planner context, query and its parameters,
-     *             execution specific variables (like queryId, current user, session, etc).
-     * @param filter Filter expression.
-     * @param rowType Input row type.
-     * @return Filter predicate.
-     */
-    public <T> Predicate<T> filterPredicate(ExecutionContext root, RexNode filter, RelDataType rowType) {
-        Scalar scalar = scalar(ImmutableList.of(filter), rowType);
-        Context ctx = InterpreterUtils.createContext(root);
-
-        return new FilterPredicate<>(ctx, scalar, handler);
-    }
-
-    /**
-     * Creates a Project function. Resulting function returns a row with different fields,
-     * fields order, fields types, etc.
-     * @param root Execution context, holds a planner context, query and its parameters,
-     *             execution specific variables (like queryId, current user, session, etc).
-     * @param projects Projection expressions.
-     * @param rowType Input row type.
-     * @return Project function.
-     */
-    public <T> Function<T, T> projectExpression(ExecutionContext root, List<RexNode> projects, RelDataType rowType) {
-        Scalar scalar = scalar(projects, rowType);
-        Context ctx = InterpreterUtils.createContext(root);
-        int count = projects.size();
-
-        return new ProjectExpression<>(ctx, scalar, count, handler);
-    }
-
-    /**
-     * Creates a Values relational node rows source.
-     *
-     * @param root Execution context, holds a planner context, query and its parameters,
-     *             execution specific variables (like queryId, current user, session, etc).
-     * @param tuples Values tuples.
-     * @param rowType Output row type.
-     * @return Values relational node rows source.
-     */
-    public <T> Iterable<T> values(ExecutionContext root, List<List<RexNode>> tuples, RelDataType rowType) {
-        try {
-            List<RexNode> nodes = new ArrayList<>();
-
-            for (List<RexNode> tuple : tuples)
-                nodes.addAll(tuple);
-
-            Scalar scalar = scalar(nodes, null);
-            Context ctx = InterpreterUtils.createContext(root);
-
-            int rowLen = rowType.getFieldCount();
-            Object[] values = new Object[nodes.size()];
-            scalar.execute(ctx, values);
-
-            return () -> new ValuesIterator<>(values, rowLen);
-        }
-        catch (Throwable e) {
-            handler.onException(e);
-
-            throw e;
-        }
-    }
-
-    /**
-     * Creates a Join expression. Function consumes two rows and returns non null value in case the rows satisfy join condition.
-     * @param root Execution context, holds a planner context, query and its parameters,
-     *             execution specific variables (like queryId, current user, session, etc).
-     * @param expression Join expression (condition).
-     * @param joinType Output row type.
-     * @return Join function.
-     */
-    public <T> BiFunction<T, T, T> joinExpression(ExecutionContext root, RexNode expression, RelDataType joinType) {
-        Scalar scalar = scalar(ImmutableList.of(expression), joinType);
-        Context ctx = InterpreterUtils.createContext(root);
-        ctx.values = new Object[joinType.getFieldCount()];
-
-        return new JoinExpression<>(ctx, scalar, handler);
-    }
-
-    /** */
-    private Scalar scalar(List<RexNode> nodes, RelDataType type) {
-        assert !F.isEmpty(nodes);
-
-        return CACHE.computeIfAbsent(cacheKey(nodes, type), k -> compile(nodes, type));
-    }
-
-    /** */
-    private String cacheKey(List<RexNode> nodes, RelDataType type) {
-        StrBuilder b = new StrBuilder();
-
-        b.append('[');
-
-        for (int i = 0; i < nodes.size(); i++) {
-            if (i > 0)
-                b.append(';');
-
-            b.append(nodes.get(i));
-        }
-
-        b.append(']');
-
-        if (type != null)
-            b.append(':').append(type.getFullTypeString());
-
-        return b.toString();
-    }
-
-    /** */
-    private Scalar compile(List<RexNode> nodes, RelDataType type) {
-        if (type == null)
-            type = new RelDataTypeFactory.Builder(typeFactory).build();
-
-        return rexCompiler.compile(nodes, type);
-    }
-
-    /** */
-    private static class FilterPredicate<T> implements Predicate<T> {
-        /** */
-        private final Context ctx;
-
-        /** */
-        private final Scalar scalar;
-
-        /** */
-        private final ExceptionHandler handler;
-
-        /** */
-        private final Object[] vals;
-
-        /**
-         * @param ctx Interpreter context.
-         * @param scalar Scalar.
-         */
-        private FilterPredicate(Context ctx, Scalar scalar, ExceptionHandler handler) {
-            this.ctx = ctx;
-            this.scalar = scalar;
-            this.handler = handler;
-
-            vals = new Object[1];
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean test(T r) {
-            try {
-                ctx.values = (Object[]) r;
-                scalar.execute(ctx, vals);
-                return (Boolean) vals[0];
-            }
-            catch (Throwable e) {
-                handler.onException(e);
-
-                throw e;
-            }
-        }
-    }
-
-    /** */
-    private static class JoinExpression<T> implements BiFunction<T, T, T> {
-        /** */
-        private final Object[] vals;
-
-        /** */
-        private final Context ctx;
-
-        /** */
-        private final Scalar scalar;
-
-        /** */
-        private final ExceptionHandler handler;
-
-        /** */
-        private Object[] left0;
-
-        /**
-         * @param ctx Interpreter context.
-         * @param scalar Scalar.
-         */
-        private JoinExpression(Context ctx, Scalar scalar, ExceptionHandler handler) {
-            this.ctx = ctx;
-            this.scalar = scalar;
-            this.handler = handler;
-
-            vals = new Object[1];
-        }
-
-        /** {@inheritDoc} */
-        @Override public T apply(T left, T right) {
-            try {
-                if (left0 != left) {
-                    left0 = (Object[]) left;
-                    System.arraycopy(left0, 0, ctx.values, 0, left0.length);
-                }
-
-                Object[] right0 = (Object[]) right;
-                System.arraycopy(right0, 0, ctx.values, left0.length, right0.length);
-
-                scalar.execute(ctx, vals);
-
-                if ((Boolean) vals[0])
-                    return (T) Arrays.copyOf(ctx.values, ctx.values.length);
-
-                return null;
-            }
-            catch (Throwable e) {
-                handler.onException(e);
-
-                throw e;
-            }
-        }
-    }
-
-    /** */
-    private static class ProjectExpression<T> implements Function<T, T> {
-        /** */
-        private final Context ctx;
-
-        /** */
-        private final Scalar scalar;
-
-        /** */
-        private final int count;
-
-        /** */
-        private final ExceptionHandler handler;
-
-        /**
-         * @param ctx Interpreter context.
-         * @param scalar Scalar.
-         * @param count Resulting columns count.
-         */
-        private ProjectExpression(Context ctx, Scalar scalar, int count, ExceptionHandler handler) {
-            this.ctx = ctx;
-            this.scalar = scalar;
-            this.count = count;
-            this.handler = handler;
-        }
-
-        /** {@inheritDoc} */
-        @Override public T apply(T r) {
-            try {
-                ctx.values = (Object[]) r;
-                Object[] res = new Object[count];
-                scalar.execute(ctx, res);
-
-                return (T) res;
-            }
-            catch (Throwable e) {
-                handler.onException(e);
-
-                throw e;
-            }
-        }
-    }
-
-    /** */
-    private static class ValuesIterator<T> implements Iterator<T> {
-        /** */
-        private final Object[] values;
-
-        /** */
-        private final int rowLen;
-
-        /** */
-        private int idx;
-
-        /** */
-        private ValuesIterator(Object[] values, int rowLen) {
-            this.values = values;
-            this.rowLen = rowLen;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean hasNext() {
-            return idx < values.length;
-        }
-
-        /** {@inheritDoc} */
-        @Override public T next() {
-            if (!hasNext())
-                throw new NoSuchElementException();
-
-            Object[] res = new Object[rowLen];
-            System.arraycopy(values, idx, res, 0, rowLen);
-            idx += rowLen;
-
-            return (T) res;
-        }
-    }
-
-    /** */
-    private static class ExceptionHandler {
-        /** */
-        private final FailureProcessor failure;
-
-        /** */
-        private final IgniteLogger log;
-
-        /** */
-        private ExceptionHandler(@Nullable FailureProcessor failure, IgniteLogger log) {
-            this.failure = failure;
-            this.log = log;
-        }
-
-        /** */
-        void onException(Throwable ex) {
-            U.error(log, ex, ex);
-
-            if (failure != null)
-                failure.process(new FailureContext(FailureType.CRITICAL_ERROR, ex));
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java
deleted file mode 100644
index 5da1db2..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/FilterNode.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.exec;
-
-import java.util.function.Predicate;
-
-/**
- *
- */
-public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
-    /** */
-    private final Predicate<Object[]> predicate;
-
-    /**
-     * @param ctx Execution context.
-     * @param predicate Predicate.
-     */
-    public FilterNode(ExecutionContext ctx, Node<Object[]> input, Predicate<Object[]> predicate) {
-        super(ctx, input);
-
-        this.predicate = predicate;
-
-        link();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Sink<Object[]> sink(int idx) {
-        if (idx != 0)
-            throw new IndexOutOfBoundsException();
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean push(Object[] row) {
-        return !predicate.test(row) || target().push(row);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void end() {
-        target().end();
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java
deleted file mode 100644
index 46b5bdb..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/JoinNode.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.exec;
-
-import com.google.common.collect.ImmutableList;
-import java.util.ArrayList;
-import java.util.function.BiFunction;
-
-/**
- * TODO remove buffers.
- */
-public class JoinNode extends AbstractNode<Object[]> {
-    /** */
-    private final BiFunction<Object[], Object[], Object[]> expression;
-
-    /** */
-    private final ArraySink<Object[]> left;
-
-    /** */
-    private final ArraySink<Object[]> right;
-
-    /** */
-    private int leftIdx;
-
-    /** */
-    private int rightIdx;
-
-    /** */
-    private boolean end;
-
-    /**
-     * @param ctx Execution context.
-     * @param expression Join expression.
-     */
-    public JoinNode(ExecutionContext ctx, Node<Object[]> left, Node<Object[]> right, BiFunction<Object[], Object[], Object[]> expression) {
-        super(ctx, ImmutableList.of(left, right));
-
-        this.expression = expression;
-        this.left = new ArraySink<>();
-        this.right = new ArraySink<>();
-
-        link();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Sink<Object[]> sink(int idx) {
-        switch (idx) {
-            case 0:
-                return left;
-            case 1:
-                return right;
-            default:
-                throw new IndexOutOfBoundsException();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void request() {
-        checkThread();
-
-        if (context().cancelled() || end)
-            return;
-
-        if (!left.end)
-            input(0).request();
-        if (!right.end)
-            input(1).request();
-        if (!end)
-            tryFlush();
-    }
-
-    /** */
-    public void tryFlush() {
-        assert !end;
-
-        if (left.end && right.end) {
-            for (int i = leftIdx; i < left.size(); i++) {
-                for (int j = rightIdx; j < right.size(); j++) {
-                    if (context().cancelled())
-                        return;
-
-                    Object[] row = expression.apply(left.get(i), right.get(j));
-
-                    if (row != null && !target().push(row)) {
-                        leftIdx = i;
-                        rightIdx = j;
-
-                        return;
-                    }
-                }
-            }
-
-            end = true;
-            target().end();
-        }
-    }
-
-    /** */
-    private final class ArraySink<T> extends ArrayList<T> implements Sink<T> {
-        /** */
-        private boolean end;
-
-        /** {@inheritDoc} */
-        @Override public boolean push(T row) {
-            return add(row);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void end() {
-            end = true;
-
-            tryFlush();
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
similarity index 65%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
index c4b93d1..49b92ef 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementor.java
@@ -17,13 +17,23 @@
 
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
-import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.ScannableTable;
-import org.apache.ignite.IgniteLogger;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.FilterNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.JoinNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.ModifyNode;
+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.metadata.PartitionService;
 import org.apache.ignite.internal.processors.query.calcite.prepare.RelSource;
 import org.apache.ignite.internal.processors.query.calcite.prepare.RelTarget;
@@ -38,16 +48,17 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
-import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
 import org.apache.ignite.internal.processors.query.calcite.schema.TableDescriptor;
 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.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.F;
 
 /**
  * Implements a query plan.
  */
-public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<IgniteRel, Node<Object[]>> {
+public class LogicalRelImplementor implements IgniteRelVisitor<Node<Object[]>> {
     /** */
     private final ExecutionContext ctx;
 
@@ -64,20 +75,23 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
     private final ExpressionFactory expressionFactory;
 
     /**
+     * @param ctx Root context.
      * @param partitionService Affinity service.
      * @param mailboxRegistry Mailbox registry.
      * @param exchangeService Exchange service.
      * @param failure Failure processor.
-     * @param ctx Root context.
-     * @param log Logger.
      */
-    public Implementor(PartitionService partitionService, MailboxRegistry mailboxRegistry, ExchangeService exchangeService, FailureProcessor failure, ExecutionContext ctx, IgniteLogger log) {
+    public LogicalRelImplementor(ExecutionContext ctx, PartitionService partitionService, MailboxRegistry mailboxRegistry, ExchangeService exchangeService, FailureProcessor failure) {
         this.partitionService = partitionService;
         this.mailboxRegistry = mailboxRegistry;
         this.exchangeService = exchangeService;
         this.ctx = ctx;
 
-        expressionFactory = new ExpressionFactory(ctx.getTypeFactory(), failure, log);
+        final IgniteTypeFactory typeFactory = ctx.getTypeFactory();
+        final SqlConformance conformance = ctx.parent().conformance();
+        final SqlOperatorTable opTable = ctx.parent().opTable();
+
+        expressionFactory = new ExpressionFactory(typeFactory, conformance, opTable);
     }
 
     /** {@inheritDoc} */
@@ -88,7 +102,8 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
         Destination destination = distribution.function().destination(partitionService, target.mapping(), distribution.getKeys());
 
         // Outbox fragment ID is used as exchange ID as well.
-        Outbox<Object[]> outbox = new Outbox<>(exchangeService, mailboxRegistry, ctx, targetFragmentId, ctx.fragmentId(), visit(rel.getInput()), destination);
+        Outbox<Object[]> outbox = new Outbox<>(ctx, exchangeService, mailboxRegistry, ctx.fragmentId(), targetFragmentId, destination);
+        outbox.register(visit(rel.getInput()));
 
         mailboxRegistry.register(outbox);
 
@@ -97,20 +112,30 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteFilter rel) {
-        Predicate<Object[]> predicate = expressionFactory.filterPredicate(ctx, rel.getCondition(), rel.getRowType());
-        return new FilterNode(ctx, visit(rel.getInput()), predicate);
+        Predicate<Object[]> predicate = expressionFactory.predicate(ctx, rel.getCondition(), rel.getRowType());
+        FilterNode node = new FilterNode(ctx, predicate);
+        node.register(visit(rel.getInput()));
+
+        return node;
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteProject rel) {
-        Function<Object[], Object[]> projection = expressionFactory.projectExpression(ctx, rel.getProjects(), rel.getInput().getRowType());
-        return new ProjectNode(ctx, visit(rel.getInput()), projection);
+        Function<Object[], Object[]> projection = expressionFactory.project(ctx, rel.getProjects(), rel.getInput().getRowType());
+        ProjectNode node = new ProjectNode(ctx, projection);
+        node.register(visit(rel.getInput()));
+
+        return node;
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteJoin rel) {
-        BiFunction<Object[], Object[], Object[]> expression = expressionFactory.joinExpression(ctx, rel.getCondition(), rel.getRowType());
-        return new JoinNode(ctx, visit(rel.getLeft()), visit(rel.getRight()), expression);
+        RelDataType rowType = Commons.combinedRowType(ctx.getTypeFactory(), rel.getLeft().getRowType(), rel.getRight().getRowType());
+        Predicate<Object[]> condition = expressionFactory.predicate(ctx, rel.getCondition(), rowType);
+        JoinNode node = new JoinNode(ctx, condition);
+        node.register(F.asList(visit(rel.getLeft()), visit(rel.getRight())));
+
+        return node;
     }
 
     /** {@inheritDoc} */
@@ -120,7 +145,7 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteValues rel) {
-        return new ScanNode(ctx, expressionFactory.values(ctx, Commons.cast(rel.getTuples()), rel.getRowType()));
+        return new ScanNode(ctx, expressionFactory.valuesRex(ctx, Commons.flat(Commons.cast(rel.getTuples())), rel.getRowType().getFieldCount()));
     }
 
     /** {@inheritDoc} */
@@ -129,7 +154,10 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
             case INSERT:
             case UPDATE:
             case DELETE:
-                return new ModifyNode(ctx, rel.getTable().unwrap(TableDescriptor.class), rel.getOperation(), rel.getUpdateColumnList(), visit(rel.getInput()));
+                ModifyNode node = new ModifyNode(ctx, rel.getTable().unwrap(TableDescriptor.class), rel.getOperation(), rel.getUpdateColumnList());
+                node.register(visit(rel.getInput()));
+
+                return node;
             case MERGE:
                 throw new UnsupportedOperationException();
             default:
@@ -142,7 +170,7 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
         RelSource source = rel.source();
 
         // Corresponding outbox fragment ID is used as exchange ID as well.
-        Inbox<Object[]> inbox = (Inbox<Object[]>) mailboxRegistry.register(new Inbox<>(exchangeService, mailboxRegistry, ctx, source.fragmentId(), source.fragmentId()));
+        Inbox<Object[]> inbox = (Inbox<Object[]>) mailboxRegistry.register(new Inbox<>(ctx, exchangeService, mailboxRegistry, source.fragmentId(), source.fragmentId()));
 
         // 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.
@@ -166,8 +194,8 @@ public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<Igni
         return visit((IgniteRel) rel);
     }
 
-    /** {@inheritDoc} */
-    @Override public Node<Object[]> go(IgniteRel rel) {
+    /** */
+    public Node<Object[]> go(IgniteRel rel) {
         return 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
index b5b80f6..cfe7df7 100644
--- 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
@@ -19,6 +19,8 @@ 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.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
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
index 1846e3b..9ff53a8 100644
--- 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
@@ -23,6 +23,8 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 import org.apache.ignite.internal.GridKernalContext;
+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.util.AbstractService;
 import org.jetbrains.annotations.Nullable;
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PhysicalRelImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PhysicalRelImplementor.java
new file mode 100644
index 0000000..fbcac21
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PhysicalRelImplementor.java
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.FilterNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.Inbox;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.JoinNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.rel.ModifyNode;
+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.metadata.PartitionService;
+import org.apache.ignite.internal.processors.query.calcite.schema.TableDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.serialize.FilterPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.JoinPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.PhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.PhysicalRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.serialize.ProjectPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.ReceiverPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.SenderPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.TableModifyPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.TableScanPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.serialize.ValuesPhysicalRel;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Converts RelGraph to logical or physical tree.
+ */
+public class PhysicalRelImplementor implements PhysicalRelVisitor<Node<Object[]>> {
+    /** */
+    private final ExecutionContext ctx;
+
+    /** */
+    private final PartitionService partitionService;
+
+    /** */
+    private final ExchangeService exchangeService;
+
+    /** */
+    private final MailboxRegistry mailboxRegistry;
+
+    /** */
+    private final ExpressionFactory expressionFactory;
+
+    /** */
+    public PhysicalRelImplementor(ExecutionContext ctx, PartitionService partitionService, MailboxRegistry mailboxRegistry, ExchangeService exchangeService, FailureProcessor failure) {
+        this.ctx = ctx;
+        this.partitionService = partitionService;
+        this.mailboxRegistry = mailboxRegistry;
+        this.exchangeService = exchangeService;
+
+        IgniteTypeFactory typeFactory = ctx.getTypeFactory();
+        SqlOperatorTable opTable = ctx.parent().opTable();
+        SqlConformance conformance = ctx.parent().conformance();
+
+        expressionFactory = new ExpressionFactory(typeFactory, conformance, opTable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(TableScanPhysicalRel rel) {
+        Iterable<Object[]> source = ctx
+            .parent()
+            .catalogReader()
+            .getTable(rel.tableName())
+            .unwrap(ScannableTable.class)
+            .scan(ctx);
+
+        return new ScanNode(ctx, source);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(FilterPhysicalRel rel) {
+        FilterNode node = new FilterNode(ctx, expressionFactory.predicate(ctx, rel.condition(), rel.rowType()));
+        node.register(visit(rel.input()));
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(ProjectPhysicalRel rel) {
+        ProjectNode node = new ProjectNode(ctx, expressionFactory.project(ctx, rel.projects(), rel.rowType()));
+        node.register(visit(rel.input()));
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(JoinPhysicalRel rel) {
+        JoinNode node = new JoinNode(ctx, expressionFactory.predicate(ctx, rel.condition(), rel.rowType()));
+        node.register(F.asList(visit(rel.left()), visit(rel.right())));
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(SenderPhysicalRel rel) {
+        Destination destination = rel.distributionFunction()
+            .destination(partitionService, rel.mapping(), rel.distributionKeys());
+
+        Outbox<Object[]> outbox = new Outbox<>(ctx, exchangeService, mailboxRegistry,
+            ctx.fragmentId(), rel.targetFragmentId(), destination);
+        outbox.register(visit(rel.input()));
+
+        mailboxRegistry.register(outbox);
+
+        return outbox;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(ReceiverPhysicalRel rel) {
+        Inbox<?> inbox = mailboxRegistry.register(new Inbox<>(ctx, exchangeService, mailboxRegistry, rel.sourceFragmentId(), rel.sourceFragmentId()));
+
+        inbox.init(ctx, rel.sources(), expressionFactory.comparator(ctx, rel.collations(), rel.rowType()));
+
+        return (Node<Object[]>) inbox;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(ValuesPhysicalRel rel) {
+        Iterable<Object[]> source = expressionFactory.valuesExp(ctx, rel.values(), rel.rowLength());
+
+        return new ScanNode(ctx, source);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(TableModifyPhysicalRel rel) {
+        switch (rel.operation()){
+            case INSERT:
+            case UPDATE:
+            case DELETE:
+                TableDescriptor desc = ctx.parent().catalogReader().getTable(rel.tableName()).unwrap(TableDescriptor.class);
+                ModifyNode node = new ModifyNode(ctx, desc, rel.operation(), rel.updateColumnList());
+                node.register(visit(rel.input()));
+
+                return node;
+            case MERGE:
+                throw new UnsupportedOperationException();
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(PhysicalRel rel) {
+        return rel.accept(this);
+    }
+
+    /** */
+    public Node<Object[]> go(PhysicalRel root) {
+        return visit(root);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
deleted file mode 100644
index 5405534..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.exec;
-
-import java.util.Iterator;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- * Scan node.
- */
-public class ScanNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, AutoCloseable {
-    /** */
-    private final Iterable<Object[]> source;
-
-    /** */
-    private Iterator<Object[]> it;
-
-    /** */
-    private Object row;
-
-    /**
-     * @param ctx Execution context.
-     * @param source Source.
-     */
-    public ScanNode(ExecutionContext ctx, Iterable<Object[]> source) {
-        super(ctx);
-
-        this.source = source;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void request() {
-        try {
-            requestInternal();
-        }
-        catch (Exception e) {
-            Commons.closeQuiet(it, e);
-
-            throw e;
-        }
-    }
-
-    /** */
-    private void requestInternal() {
-        checkThread();
-
-        if (context().cancelled()
-            || row == EndMarker.INSTANCE
-            || row != null && !target().push((Object[]) row))
-            return;
-
-        if (it == null)
-            it = source.iterator();
-
-        Thread thread = Thread.currentThread();
-
-        while (it.hasNext()) {
-            if (context().cancelled() || thread.isInterrupted()) {
-                row = null;
-
-                close();
-
-                return;
-            }
-
-            row = it.next();
-
-            // TODO load balancing - resubmit this::request() in case of long execution
-
-            if (!target().push((Object[]) row))
-                return;
-        }
-
-        try {
-            row = EndMarker.INSTANCE;
-
-            target().end();
-        }
-        finally {
-            close();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        Commons.closeQuiet(it);
-
-        it = null;
-
-        if (row != EndMarker.INSTANCE)
-            row = null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Sink<Object[]> sink(int idx) {
-        throw new AssertionError();
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Call.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Call.java
new file mode 100644
index 0000000..d128d2d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Call.java
@@ -0,0 +1,117 @@
+/*
+ * 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.exp;
+
+import java.util.List;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.util.MutableSingletonList;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Describes {@link org.apache.calcite.rex.RexCall}.
+ */
+public class Call implements Expression {
+    /** */
+    private DataType type;
+
+    /** */
+    private String opName;
+
+    /** */
+    private SqlSyntax syntax;
+
+    /** */
+    private List<Expression> operands;
+
+    /** */
+    private transient CallOperation opImpl;
+
+    /**
+     * @param type Result type.
+     * @param operands Operands.
+     */
+    public Call(DataType type, String opName, SqlSyntax syntax, List<Expression> operands) {
+        this.type = type;
+        this.opName = opName;
+        this.syntax = syntax;
+        this.operands = operands;
+    }
+
+    /** {@inheritDoc} */
+    @Override public DataType resultType() {
+        return type;
+    }
+
+    /**
+     * @param opTable Operators table.
+     * @return Sql logical operator.
+     */
+    public SqlOperator sqlOperator(SqlOperatorTable opTable) {
+        List<SqlOperator> bag = new MutableSingletonList<>();
+
+        opTable.lookupOperatorOverloads(
+            new SqlIdentifier(opName, SqlParserPos.ZERO), null,
+            syntax, bag, SqlNameMatchers.withCaseSensitive(true));
+
+        return F.first(bag);
+    }
+
+    /** */
+    public String opName() {
+        return opName;
+    }
+
+    /** */
+    public SqlSyntax syntax() {
+        return syntax;
+    }
+
+    /**
+     * @return Operands.
+     */
+    public List<Expression> operands() {
+        return operands;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T evaluate(ExecutionContext ctx, Object... args) {
+        if (opImpl == null) {
+            IgniteTypeFactory typeFactory = ctx.parent().typeFactory();
+            SqlConformance conformance = ctx.parent().conformance();
+            SqlOperatorTable opTable = ctx.parent().opTable();
+
+            opImpl = new ExpressionFactory(typeFactory, conformance, opTable).implement(this);
+        }
+
+        return (T) opImpl.apply(operands.stream().map(o -> o.evaluate(ctx, args)).toArray());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/SystemType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallOperation.java
similarity index 78%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/SystemType.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallOperation.java
index 0b8e454..3150bc6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/SystemType.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallOperation.java
@@ -15,10 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.type;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.util.function.Function;
 
 /**
- * Marker interface.
+ *
  */
-public interface SystemType {
+public interface CallOperation extends Function<Object[], Object> {
+    @Override Object apply(Object... args);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/DynamicParam.java
similarity index 64%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/DynamicParam.java
index 09e5290..256609b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/DynamicParam.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
 
 /**
  * Describes {@link org.apache.calcite.rex.RexDynamicParam}.
  */
-public class DynamicParamExpression implements Expression {
+public class DynamicParam implements Expression {
     /** */
     private final DataType type;
 
@@ -35,15 +34,13 @@ public class DynamicParamExpression implements Expression {
      * @param type Data type.
      * @param index Index.
      */
-    public DynamicParamExpression(RelDataType type, int index) {
-        this.type = Types.fromType(type);
+    public DynamicParam(DataType type, int index) {
+        this.type = type;
         this.index = index;
     }
 
-    /**
-     * @return Data type;
-     */
-    public DataType dataType() {
+    /** {@inheritDoc} */
+    @Override public DataType resultType() {
         return type;
     }
 
@@ -55,7 +52,12 @@ public class DynamicParamExpression implements Expression {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> T implement(ExpImplementor<T> implementor) {
-        return implementor.implement(this);
+    @Override public <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T evaluate(ExecutionContext ctx, Object... args) {
+        return (T) ctx.get("?" + index);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpToRexTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpToRexTranslator.java
new file mode 100644
index 0000000..8e9a9e8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpToRexTranslator.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.util.List;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * A translator of Expressions into Rex nodes.
+ */
+public class ExpToRexTranslator implements ExpressionVisitor<RexNode> {
+    /** */
+    private final IgniteTypeFactory typeFactory;
+
+    /** */
+    private final SqlOperatorTable opTable;
+
+    /** */
+    private final RexBuilder builder;
+
+    /**
+     * Creates a Translator.
+     *
+     * @param typeFactory Ignite type factory.
+     * @param opTable Operators table.
+     */
+    public ExpToRexTranslator(IgniteTypeFactory typeFactory, SqlOperatorTable opTable) {
+        this.typeFactory = typeFactory;
+        this.opTable = opTable;
+
+        builder = new RexBuilder(typeFactory);
+    }
+
+    /**
+     * Translates a list of expressions into a list of Rex nodes.
+     *
+     * @param exps List of expressions.
+     * @return List of Rex nodes.
+     */
+    public List<RexNode> translate(List<Expression> exps) {
+        return Commons.transform(exps, this::translate);
+    }
+
+    /**
+     * Translates an expression into a RexNode.
+     *
+     * @param exp Expression.
+     * @return RexNode.
+     */
+    public RexNode translate(Expression exp) {
+        return exp.accept(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RexNode visit(InputRef exp) {
+        return builder.makeInputRef(exp.logicalType(typeFactory), exp.index());
+    }
+
+    /** {@inheritDoc} */
+    @Override public RexNode visit(Literal exp) {
+        return builder.makeLiteral(exp.value(), exp.logicalType(typeFactory),false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RexNode visit(DynamicParam exp) {
+        return builder.makeDynamicParam(exp.logicalType(typeFactory), exp.index());
+    }
+
+    /** {@inheritDoc} */
+    @Override public RexNode visit(Call exp) {
+        return builder.makeCall(exp.logicalType(typeFactory), exp.sqlOperator(opTable), translate(exp.operands()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public RexNode visit(Expression exp) {
+        return exp.accept(this);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Expression.java
similarity index 50%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Expression.java
index 0791737..ef44a80 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/DataType.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Expression.java
@@ -15,21 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.type;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
 import java.io.Serializable;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 
 /**
- * Serializable RelDataType representation.
+ * Describes {@link org.apache.calcite.rex.RexNode}
  */
-public interface DataType extends Serializable {
+public interface Expression extends Serializable {
     /**
-     * Perform back conversion from data type representation to RelDataType itself.
+     * Accepts a visit from a visitor.
      *
-     * @param factory Type factory.
-     * @return RelDataType.
+     * @param visitor Expression visitor.
+     * @return Visit result.
      */
-    RelDataType toRelDataType(IgniteTypeFactory factory);
+    <T> T accept(ExpressionVisitor<T> visitor);
+
+    /** */
+    DataType resultType();
+
+    /** */
+    default RelDataType logicalType(IgniteTypeFactory typeFactory) {
+        return resultType().logicalType(typeFactory);
+    }
+
+    /** */
+    default Class<?> javaType(IgniteTypeFactory typeFactory) {
+        return resultType().javaType(typeFactory);
+    }
+
+    /**
+     * Evaluates expression.
+     *
+     * @param ctx Execution context.
+     * @return Evaluated value.
+     */
+    <T> T evaluate(ExecutionContext ctx, Object... args);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
new file mode 100644
index 0000000..0c026b6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
@@ -0,0 +1,502 @@
+/*
+ * 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.exp;
+
+import com.google.common.collect.ImmutableList;
+import java.io.StringReader;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.config.CalciteSystemProperty;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.RelCollation;
+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.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+import org.apache.ignite.internal.util.typedef.F;
+import org.codehaus.commons.compiler.CompilerFactoryFactory;
+import org.codehaus.commons.compiler.IClassBodyEvaluator;
+import org.codehaus.commons.compiler.ICompilerFactory;
+
+/**
+ * Implements rex expression into a function object. Uses JaninoRexCompiler under the hood.
+ * Each expression compiles into a class and a wrapper over it is returned.
+ */
+public class ExpressionFactory {
+    /** */
+    private static final Map<String, Scalar> CACHE_1 = new GridBoundedConcurrentLinkedHashMap<>(1024);
+
+    /** */
+    private static final Map<String, CallOperation> CACHE_2 = new GridBoundedConcurrentLinkedHashMap<>(1024);
+
+    /** */
+    private final IgniteTypeFactory typeFactory;
+
+    /** */
+    private final SqlConformance conformance;
+
+    /** */
+    private final SqlOperatorTable opTable;
+
+    /** */
+    private final RexBuilder rexBuilder;
+
+    /** */
+    private final ExpToRexTranslator expToRexTranslator;
+
+    /** */
+    private final RexToExpTranslator rexToExpTranslator;
+
+    /** */
+    private final RelDataType emptyType;
+
+    /** */
+    public ExpressionFactory(IgniteTypeFactory typeFactory, SqlConformance conformance, SqlOperatorTable opTable) {
+        this.typeFactory = typeFactory;
+        this.conformance = conformance;
+        this.opTable = opTable;
+
+        rexBuilder = new RexBuilder(typeFactory);
+        expToRexTranslator = new ExpToRexTranslator(typeFactory, opTable);
+        rexToExpTranslator = new RexToExpTranslator(typeFactory);
+        emptyType = new RelDataTypeFactory.Builder(typeFactory).build();
+    }
+
+    /**
+     * Creates a comparator for given data type and collations. Mainly used for sorted exchange.
+     *
+     * @param root Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param collations Collations.
+     * @param rowType Input row type.
+     * @return Row comparator.
+     */
+    public <T> Comparator<T> comparator(ExecutionContext root, List<RelCollation> collations, RelDataType rowType) {
+        return null; // TODO
+    }
+
+    /**
+     * Creates a comparator for given data type and collations. Mainly used for sorted exchange.
+     *
+     * @param root Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param collations Collations.
+     * @param rowType Input row type.
+     * @return Row comparator.
+     */
+    public <T> Comparator<T> comparator(ExecutionContext root, List<RelCollation> collations, DataType rowType) {
+        return null; // TODO
+    }
+
+    /**
+     * Creates a Filter predicate.
+     * @param ctx Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param filter Filter expression.
+     * @param rowType Input row type.
+     * @return Filter predicate.
+     */
+    public <T> Predicate<T> predicate(ExecutionContext ctx, Expression filter, DataType rowType) {
+        return predicate(ctx, expToRexTranslator.translate(filter), rowType.logicalType(typeFactory));
+    }
+
+    /**
+     * Creates a Filter predicate.
+     * @param ctx Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param filter Filter expression.
+     * @param rowType Input row type.
+     * @return Filter predicate.
+     */
+    public <T> Predicate<T> predicate(ExecutionContext ctx, RexNode filter, RelDataType rowType) {
+        return new PredicateImpl<>(ctx, scalar(filter, rowType));
+    }
+
+    /**
+     * Creates a Project function. Resulting function returns a row with different fields,
+     * fields order, fields types, etc.
+     * @param ctx Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param projects Projection expressions.
+     * @param rowType Input row type.
+     * @return Project function.
+     */
+    public <T> Function<T, T> project(ExecutionContext ctx, List<Expression> projects, DataType rowType) {
+        return project(ctx, expToRexTranslator.translate(projects), rowType.logicalType(typeFactory));
+    }
+
+    /**
+     * Creates a Project function. Resulting function returns a row with different fields,
+     * fields order, fields types, etc.
+     * @param ctx Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param projects Projection expressions.
+     * @param rowType Input row type.
+     * @return Project function.
+     */
+    public <T> Function<T, T> project(ExecutionContext ctx, List<RexNode> projects, RelDataType rowType) {
+        return new ProjectImpl<>(ctx, scalar(projects, rowType), projects.size());
+    }
+
+    /**
+     * Creates a Values relational node rows source.
+     *
+     * @param ctx Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param values Values.
+     * @param rowLen Row length.
+     * @return Values relational node rows source.
+     */
+    public <T> Iterable<T> valuesExp(ExecutionContext ctx, List<Expression> values, int rowLen) {
+        Object[] out = new Object[values.size()];
+
+        for (int i = 0; i < values.size(); i++)
+            out[i] = values.get(i).evaluate(ctx);
+
+        return () -> new ValuesIterator<>(out, rowLen);
+    }
+
+    /**
+     * Creates a Values relational node rows source.
+     *
+     * @param ctx Execution context, holds a planner context, query and its parameters,
+     *             execution specific variables (like queryId, current user, session, etc).
+     * @param values Values.
+     * @param rowLen Row length.
+     * @return Values relational node rows source.
+     */
+    public <T> Iterable<T> valuesRex(ExecutionContext ctx, List<RexNode> values, int rowLen) {
+        // it's safe to just interpret literals
+        return valuesExp(ctx, rexToExpTranslator.translate(values), rowLen);
+    }
+
+    /** */
+    public CallOperation implement(Call call) {
+        return CACHE_2.computeIfAbsent(digest(call), k -> compile(call));
+    }
+
+    /** */
+    private Scalar scalar(RexNode node, RelDataType type) {
+        return scalar(ImmutableList.of(node), type);
+    }
+
+    /** */
+    private Scalar scalar(List<RexNode> nodes, RelDataType type) {
+        return CACHE_1.computeIfAbsent(digest(nodes, type), k -> compile(nodes, type));
+    }
+
+    /** */
+    private Scalar compile(List<RexNode> nodes, RelDataType type) {
+        if (type == null)
+            type = emptyType;
+
+        ParameterExpression context_ =
+            Expressions.parameter(ExecutionContext.class, "ctx");
+
+        ParameterExpression inputValues_ =
+            Expressions.parameter(Object[].class, "in");
+
+        ParameterExpression outputValues_ =
+            Expressions.parameter(Object[].class, "out");
+
+        RexToLixTranslator.InputGetter inputGetter =
+            new RexToLixTranslator.InputGetterImpl(
+                ImmutableList.of(
+                    Pair.of(inputValues_,
+                        PhysTypeImpl.of(typeFactory, type,
+                            JavaRowFormat.ARRAY, false))));
+
+        RexProgramBuilder programBuilder = new RexProgramBuilder(type, rexBuilder);
+
+        for (RexNode node : nodes)
+            programBuilder.addProject(node, null);
+
+        RexProgram program = programBuilder.getProgram();
+
+        BlockBuilder builder = new BlockBuilder();
+
+        List<org.apache.calcite.linq4j.tree.Expression> list = RexToLixTranslator.translateProjects(program,
+            typeFactory, conformance, builder, null, context_, inputGetter, null);
+
+        for (int i = 0; i < list.size(); i++) {
+            builder.add(
+                Expressions.statement(
+                    Expressions.assign(
+                        Expressions.arrayIndex(outputValues_,
+                            Expressions.constant(i)),
+                        list.get(i))));
+        }
+
+        MethodDeclaration declaration = Expressions.methodDecl(
+            Modifier.PUBLIC, void.class, IgniteMethod.SCALAR_EXECUTE.method().getName(),
+            ImmutableList.of(context_, inputValues_, outputValues_), builder.toBlock());
+
+        return compile(Scalar.class, Expressions.toString(F.asList(declaration), "\n", false));
+    }
+
+    /** */
+    private CallOperation compile(Call call) {
+        RexCall rexCall = rexCall(call);
+
+        BlockBuilder builder = new BlockBuilder();
+
+        ParameterExpression params_ =
+            Expressions.parameter(Object[].class, "in");
+
+        RexToLixTranslator.InputGetter inputGetter =
+            new RexToLixTranslator.InputGetterImpl(
+                ImmutableList.of(
+                    Pair.of(params_,
+                        PhysTypeImpl.of(typeFactory, inputType(rexCall),
+                            JavaRowFormat.ARRAY, false))));
+
+        RexToLixTranslator translator = RexToLixTranslator.forAggregation(typeFactory,
+            builder, inputGetter, conformance);
+
+        builder.add(
+            RexImpTable.INSTANCE.get(rexCall.op)
+                .implement(translator,
+                    rexCall, RexImpTable.NullAs.NULL));
+
+        MethodDeclaration declaration = Expressions.methodDecl(
+            Modifier.PUBLIC, Object.class, IgniteMethod.CALL_APPLY.method().getName(), ImmutableList.of(params_), builder.toBlock());
+
+        return compile(CallOperation.class, Expressions.toString(F.asList(declaration), "\n", true));
+    }
+
+    /** */
+    private RexCall rexCall(Call call) {
+        List<Expression> operands = call.operands();
+        List<RexNode> refs = new ArrayList<>(operands.size());
+
+        for (int i = 0; i < operands.size(); i++)
+            refs.add(rexBuilder.makeInputRef(operands.get(i).logicalType(typeFactory), i));
+
+        return (RexCall) rexBuilder.makeCall(call.logicalType(typeFactory), call.sqlOperator(opTable), refs);
+    }
+
+    /** */
+    private RelDataType inputType(RexCall rexCall) {
+        RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(typeFactory);
+        ImmutableList<RexNode> rexNodes = rexCall.operands;
+        for (int i = 0; i < rexNodes.size(); i++)
+            typeBuilder.add("EXP$" + i, rexNodes.get(i).getType());
+        return typeBuilder.build();
+    }
+
+    /** */
+    private String digest(Call call) {
+        List<Expression> operands = call.operands();
+
+        final StringBuilder sb = new StringBuilder(call.opName());
+
+        if (!operands.isEmpty() || call.syntax() != SqlSyntax.FUNCTION_ID) {
+            sb.append("(");
+
+            for (int i = 0; i < operands.size(); i++) {
+                if (i != 0)
+                    sb.append(", ");
+
+                sb.append("$").append(i).append(":").append(operands.get(i).resultType());
+            }
+
+            sb.append(")");
+        }
+
+        sb.append(":").append(call.resultType());
+
+        return sb.toString();
+    }
+
+    /** */
+    private String digest(List<RexNode> nodes, RelDataType type) {
+        StringBuilder b = new StringBuilder();
+
+        b.append('[');
+
+        for (int i = 0; i < nodes.size(); i++) {
+            if (i > 0)
+                b.append(';');
+
+            b.append(nodes.get(i));
+        }
+
+        b.append(']');
+
+        if (type != null)
+            b.append(':').append(type.getFullTypeString());
+
+        return b.toString();
+    }
+
+    /** */
+    private static <T> T compile(Class<T> interfaceType, String body) {
+        final boolean debug = CalciteSystemProperty.DEBUG.value();
+
+        if (debug)
+            Util.debugCode(System.out, body);
+
+        try {
+            final ICompilerFactory compilerFactory;
+
+            try {
+                compilerFactory = CompilerFactoryFactory.getDefaultCompilerFactory();
+            } catch (Exception e) {
+                throw new IllegalStateException(
+                    "Unable to instantiate java compiler", e);
+            }
+
+            IClassBodyEvaluator cbe = compilerFactory.newClassBodyEvaluator();
+
+            cbe.setImplementedInterfaces(new Class[]{ interfaceType });
+            cbe.setParentClassLoader(ExpressionFactory.class.getClassLoader());
+
+            if (debug)
+                // Add line numbers to the generated janino class
+                cbe.setDebuggingInformation(true, true, true);
+
+            return (T) cbe.createInstance(new StringReader(body));
+        } catch (Exception e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** */
+    private static class PredicateImpl<T> implements Predicate<T> {
+        /** */
+        private final ExecutionContext ctx;
+
+        /** */
+        private final Scalar scalar;
+
+        /** */
+        private final Object[] out;
+
+        /**
+         * @param ctx Interpreter context.
+         * @param scalar Scalar.
+         */
+        private PredicateImpl(ExecutionContext ctx, Scalar scalar) {
+            this.ctx = ctx;
+            this.scalar = scalar;
+
+            out = new Object[1];
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean test(T r) {
+            scalar.execute(ctx, (Object[]) r, out);
+
+            return (Boolean) out[0];
+        }
+    }
+
+    /** */
+    private static class ProjectImpl<T> implements Function<T, T> {
+        /** */
+        private final ExecutionContext ctx;
+
+        /** */
+        private final Scalar scalar;
+
+        /** */
+        private final int count;
+
+        /**
+         * @param ctx Interpreter context.
+         * @param scalar Scalar.
+         * @param count Resulting columns count.
+         */
+        private ProjectImpl(ExecutionContext ctx, Scalar scalar, int count) {
+            this.ctx = ctx;
+            this.scalar = scalar;
+            this.count = count;
+        }
+
+        /** {@inheritDoc} */
+        @Override public T apply(T r) {
+            Object[] out = new Object[count];
+            scalar.execute(ctx, (Object[]) r, out);
+
+            return (T) out;
+        }
+    }
+
+    /** */
+    private static class ValuesIterator<T> implements Iterator<T> {
+        /** */
+        private final Object[] values;
+
+        /** */
+        private final int rowLen;
+
+        /** */
+        private int idx;
+
+        /** */
+        private ValuesIterator(Object[] values, int rowLen) {
+            this.values = values;
+            this.rowLen = rowLen;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return idx < values.length;
+        }
+
+        /** {@inheritDoc} */
+        @Override public T next() {
+            if (!hasNext())
+                throw new NoSuchElementException();
+
+            Object[] res = new Object[rowLen];
+            System.arraycopy(values, idx, res, 0, rowLen);
+            idx += rowLen;
+
+            return (T) res;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionVisitor.java
similarity index 58%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionVisitor.java
index b2cff09..e3d74bf 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionVisitor.java
@@ -15,36 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
 /**
- * Implements Expression tree recursively using Visitor pattern.
+ * Visitor pattern for traversing a tree of {@link Expression} objects.
  */
-public interface ExpImplementor<T> {
+public interface ExpressionVisitor<T> {
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link ExpressionVisitor#visit(Expression)}
      */
-    T implement(CallExpression exp);
+    T visit(InputRef exp);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link ExpressionVisitor#visit(Expression)}
      */
-    T implement(InputRefExpression exp);
+    T visit(Literal exp);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link ExpressionVisitor#visit(Expression)}
      */
-    T implement(LiteralExpression exp);
+    T visit(DynamicParam exp);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link ExpressionVisitor#visit(Expression)}
      */
-    T implement(LocalRefExpression exp);
-
-    /**
-     * See {@link ExpImplementor#implement(Expression)}
-     */
-    T implement(DynamicParamExpression exp);
+    T visit(Call exp);
 
     /**
      * Implements given expression.
@@ -52,5 +47,5 @@ public interface ExpImplementor<T> {
      * @param exp Expression.
      * @return Implementation result.
      */
-    T implement(Expression exp);
+    T visit(Expression exp);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/InputRef.java
similarity index 64%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/InputRef.java
index 840b198..dfbfb18 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/InputRef.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
 
 /**
  * Describes {@link org.apache.calcite.rex.RexInputRef}.
  */
-public class InputRefExpression implements Expression {
+public class InputRef implements Expression {
     /** */
     private final DataType type;
 
@@ -35,15 +34,13 @@ public class InputRefExpression implements Expression {
      * @param type Data type.
      * @param index Index.
      */
-    public InputRefExpression(RelDataType type, int index) {
-        this.type = Types.fromType(type);
+    public InputRef(DataType type, int index) {
+        this.type = type;
         this.index = index;
     }
 
-    /**
-     * @return Data type.
-     */
-    public DataType dataType() {
+    /** {@inheritDoc} */
+    @Override public DataType resultType() {
         return type;
     }
 
@@ -55,7 +52,12 @@ public class InputRefExpression implements Expression {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> T implement(ExpImplementor<T> implementor) {
-        return implementor.implement(this);
+    @Override public <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T evaluate(ExecutionContext ctx, Object... args) {
+        return (T) args[index];
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Literal.java
similarity index 61%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Literal.java
index ccac4ea..2108537 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LiteralExpression.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Literal.java
@@ -15,47 +15,49 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
 
 /**
  * Describes {@link org.apache.calcite.rex.RexLiteral}.
  */
-public class LiteralExpression implements Expression {
+public class Literal implements Expression {
     /** */
     private final DataType type;
 
     /** */
-    private final Comparable<?> value;
+    private final Object value;
 
     /**
      * @param type Data type.
      * @param value Value.
      */
-    public LiteralExpression(RelDataType type, Comparable<?> value) {
-        this.type = Types.fromType(type);
+    public Literal(DataType type, Object value) {
+        this.type = type;
         this.value = value;
     }
 
-    /**
-     * @return Data type.
-     */
-    public DataType dataType() {
+    /** {@inheritDoc} */
+    @Override public DataType resultType() {
         return type;
     }
 
     /**
      * @return Value.
      */
-    public Comparable<?> value() {
+    public Object value() {
         return value;
     }
 
     /** {@inheritDoc} */
-    @Override public <T> T implement(ExpImplementor<T> implementor) {
-        return implementor.implement(this);
+    @Override public <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T evaluate(ExecutionContext ctx, Object... args) {
+        return (T) value;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToExpTranslator.java
similarity index 69%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToExpTranslator.java
index 45a47ac..1642be3 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/RexToExpTranslator.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToExpTranslator.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
 import java.util.List;
 import org.apache.calcite.rex.RexCall;
@@ -32,12 +32,20 @@ import org.apache.calcite.rex.RexRangeRef;
 import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexTableInputRef;
 import org.apache.calcite.rex.RexVisitor;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * A translator of Rex nodes into Expressions.
  */
 public class RexToExpTranslator implements RexVisitor<Expression> {
+    /** */
+    private final IgniteTypeFactory typeFactory;
+
+    public RexToExpTranslator(IgniteTypeFactory typeFactory) {
+        this.typeFactory = typeFactory;
+    }
 
     /**
      * Translates a list of Rex nodes into a list of expressions.
@@ -60,62 +68,63 @@ public class RexToExpTranslator implements RexVisitor<Expression> {
     }
 
     /** {@inheritDoc} */
-     @Override public Expression visitInputRef(RexInputRef inputRef) {
-        return new InputRefExpression(inputRef.getType(), inputRef.getIndex());
+     @Override public Expression visitInputRef(RexInputRef rex) {
+        return new InputRef(DataType.fromType(rex.getType()), rex.getIndex());
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitLocalRef(RexLocalRef localRef) {
-        return new LocalRefExpression(localRef.getType(), localRef.getIndex());
+    @Override public Expression visitLiteral(RexLiteral rex) {
+        final DataType type = DataType.fromType(rex.getType());
+        return new Literal(type, rex.getValueAs(Commons.boxType(type.javaType(typeFactory))));
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitLiteral(RexLiteral literal) {
-        return new LiteralExpression(literal.getType(), literal.getValue());
+    @Override public Expression visitDynamicParam(RexDynamicParam rex) {
+        return new DynamicParam(DataType.fromType(rex.getType()), rex.getIndex());
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitCall(RexCall call) {
-        return new CallExpression(call.getOperator(), call.getType(), translate(call.getOperands()));
+    @Override public Expression visitCall(RexCall rex) {
+        return new Call(DataType.fromType(rex.getType()), rex.op.getName(), rex.op.getSyntax(), translate(rex.getOperands()));
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitOver(RexOver over) {
+    @Override public Expression visitLocalRef(RexLocalRef rex) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitCorrelVariable(RexCorrelVariable correlVariable) {
+    @Override public Expression visitOver(RexOver rex) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitDynamicParam(RexDynamicParam dynamicParam) {
-        return new DynamicParamExpression(dynamicParam.getType(), dynamicParam.getIndex());
+    @Override public Expression visitCorrelVariable(RexCorrelVariable rex) {
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitRangeRef(RexRangeRef rangeRef) {
+    @Override public Expression visitRangeRef(RexRangeRef rex) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitFieldAccess(RexFieldAccess fieldAccess) {
+    @Override public Expression visitFieldAccess(RexFieldAccess rex) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitSubQuery(RexSubQuery subQuery) {
+    @Override public Expression visitSubQuery(RexSubQuery rex) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitTableInputRef(RexTableInputRef fieldRef) {
+    @Override public Expression visitTableInputRef(RexTableInputRef rex) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public Expression visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+    @Override public Expression visitPatternFieldRef(RexPatternFieldRef rex) {
         throw new UnsupportedOperationException();
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/SystemType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
similarity index 76%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/SystemType.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
index 0b8e454..02ec14e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/SystemType.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
@@ -15,10 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.type;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 
 /**
- * Marker interface.
+ *
  */
-public interface SystemType {
+public interface Scalar {
+    void execute(ExecutionContext ctx, Object[] in, Object[] out);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/BasicType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/BasicType.java
new file mode 100644
index 0000000..a66db57e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/BasicType.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.type;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.commons.lang.text.StrBuilder;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
+import static org.apache.calcite.rel.type.RelDataType.SCALE_NOT_SPECIFIED;
+
+/** */
+public class BasicType implements DataType, Externalizable {
+    /** */
+    private SqlTypeName typeName;
+
+    /** */
+    private int precision = PRECISION_NOT_SPECIFIED;
+
+    /** */
+    private int scale = SCALE_NOT_SPECIFIED;
+
+    /** */
+    private boolean nullable;
+
+    /** */
+    public BasicType() {
+    }
+
+    /**
+     * @param type Source type.
+     */
+    public BasicType(BasicSqlType type) {
+        typeName = type.getSqlTypeName();
+        precision = type.getPrecision();
+        scale = type.getScale();
+        nullable = type.isNullable();
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlTypeName typeName() {
+        return typeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int precision() {
+        return precision;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int scale() {
+        return scale;
+    }
+
+    /** */
+    @Override public boolean nullable() {
+        return nullable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType logicalType(IgniteTypeFactory factory) {
+        RelDataType type;
+
+        if (!typeName.allowsPrec())
+            type = factory.createSqlType(typeName);
+        else if (!typeName.allowsScale())
+            type = factory.createSqlType(typeName, precision);
+        else
+            type = factory.createSqlType(typeName, precision, scale);
+
+        return nullable ? factory.createTypeWithNullability(type, nullable) : type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> javaType(IgniteTypeFactory typeFactory) {
+        return (Class<?>) typeFactory.getJavaClass(logicalType(typeFactory));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeByte(typeName.ordinal());
+
+        if (typeName.allowsScale())
+            out.writeInt(scale);
+
+        if (typeName.allowsPrec())
+            out.writeInt(precision);
+
+        out.writeBoolean(nullable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        typeName = SqlTypeName.values()[in.readByte()];
+
+        if (typeName.allowsScale())
+            scale = in.readInt();
+
+        if (typeName.allowsPrec())
+            precision = in.readInt();
+
+        nullable = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return toString(true);
+    }
+
+    /** */
+    protected String toString(boolean withNullability) {
+        StrBuilder sb = new StrBuilder(typeName.name());
+
+        boolean printPrecision = precision != PRECISION_NOT_SPECIFIED;
+        boolean printScale = scale != SCALE_NOT_SPECIFIED;
+
+        if (printPrecision) {
+            sb.append('(');
+            sb.append(precision);
+            if (printScale) {
+                sb.append(", ");
+                sb.append(scale);
+            }
+            sb.append(')');
+        }
+
+        if (withNullability && !nullable)
+            sb.append(" NOT NULL");
+
+        return sb.toString();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/CharacterDataType.java
similarity index 73%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/CharacterDataType.java
index bb8040c..9c43998 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/RelOp.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/CharacterDataType.java
@@ -15,17 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.rel;
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.type;
 
-import org.apache.calcite.rel.RelNode;
+import java.nio.charset.Charset;
+import org.apache.calcite.sql.SqlCollation;
 
 /**
- * Operation on relational expression.
+ *
  */
-public interface RelOp<T extends RelNode, R> {
-    /**
-     * @param rel Relational expression.
-     * @return Operation result.
-     */
-    R go(T rel);
+public interface CharacterDataType extends DataType {
+    /** */
+    Charset charset();
+
+    /** */
+    SqlCollation collation();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/CharacterType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/CharacterType.java
new file mode 100644
index 0000000..22fa92c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/CharacterType.java
@@ -0,0 +1,143 @@
+/*
+ * 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.exp.type;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.charset.Charset;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.commons.codec.Charsets;
+import org.apache.commons.lang.text.StrBuilder;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/** */
+public class CharacterType extends BasicType implements CharacterDataType {
+    /** */
+    private static final int COERCIBLE_COLLATION = 0;
+
+    /** */
+    private static final int IMPLICIT_COLLATION = 1;
+
+    /** */
+    private static final int CUSTOM_COLLATION = 2;
+
+    /** */
+    private Charset charset;
+
+    /** */
+    private SqlCollation collation;
+
+    /** */
+    public CharacterType() {
+    }
+
+    /**
+     * @param type Source type.
+     */
+    public CharacterType(BasicSqlType type) {
+        super(type);
+
+        charset = type.getCharset();
+        collation = type.getCollation();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Charset charset() {
+        return charset;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlCollation collation() {
+        return collation;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType logicalType(IgniteTypeFactory factory) {
+        return factory.createTypeWithCharsetAndCollation(super.logicalType(factory), charset, collation);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> javaType(IgniteTypeFactory typeFactory) {
+        return (Class<?>)typeFactory.getJavaClass(logicalType(typeFactory));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        if (charset == null)
+            out.writeBoolean(false);
+        else {
+            out.writeBoolean(true);
+            out.writeUTF(charset.name());
+        }
+
+        if (collation == SqlCollation.COERCIBLE)
+            out.writeByte(COERCIBLE_COLLATION);
+        else if (collation == SqlCollation.IMPLICIT)
+            out.writeByte(IMPLICIT_COLLATION);
+        else {
+            out.writeByte(CUSTOM_COLLATION);
+            out.writeUTF(collation.getCollationName());
+            out.writeByte(collation.getCoercibility().ordinal());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        if (in.readBoolean())
+            charset = Charsets.toCharset(in.readUTF());
+
+        byte collationType = in.readByte();
+
+        if (collationType == COERCIBLE_COLLATION)
+            collation = SqlCollation.COERCIBLE;
+        else if (collationType == IMPLICIT_COLLATION)
+            collation = SqlCollation.IMPLICIT;
+        else if (collationType == CUSTOM_COLLATION)
+            collation = new SqlCollation(in.readUTF(), SqlCollation.Coercibility.values()[in.readByte()]);
+        else
+            throw new AssertionError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        StrBuilder sb = new StrBuilder(toString(false));
+
+        if (charset != null)
+            sb.append(" CHARACTER SET \"")
+                .append(charset.name())
+                .append("\"");
+        if (collation != null
+            && collation != SqlCollation.IMPLICIT
+            && collation != SqlCollation.COERCIBLE)
+            sb.append(" COLLATE \"")
+                .append(collation.getCollationName())
+                .append("\"");
+
+        if (!nullable())
+            sb.append(" NOT NULL");
+
+        return sb.toString();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/DataType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/DataType.java
new file mode 100644
index 0000000..21c0c9a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/DataType.java
@@ -0,0 +1,89 @@
+/*
+ * 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.exp.type;
+
+import java.io.Serializable;
+import java.util.LinkedHashMap;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/**
+ * Serializable RelDataType representation.
+ */
+public interface DataType extends Serializable {
+    /**
+     * Factory method to construct data type representation from RelDataType.
+     * @param type RelDataType.
+     * @return DataType.
+     */
+    static DataType fromType(RelDataType type) {
+        if (type.isStruct()) {
+            assert type.isStruct();
+
+            LinkedHashMap<String, DataType> fields = new LinkedHashMap<>();
+
+            for (RelDataTypeField field : type.getFieldList())
+                fields.put(field.getName(), fromType(field.getType()));
+
+            return new StructType(fields);
+        }
+
+        if (type instanceof RelDataTypeFactoryImpl.JavaType)
+            return new JavaType(((RelDataTypeFactoryImpl.JavaType) type).getJavaClass(), type.isNullable());
+
+        assert type instanceof BasicSqlType : type;
+
+        if (SqlTypeUtil.inCharFamily(type))
+            return new CharacterType((BasicSqlType) type);
+
+        return new BasicType((BasicSqlType) type);
+    }
+
+    /** */
+    SqlTypeName typeName();
+
+    /** */
+    int precision();
+
+    /** */
+    int scale();
+
+    /** */
+    boolean nullable();
+
+    /**
+     * Returns logical type.
+     *
+     * @param factory Type factory.
+     * @return Logical type.
+     */
+    RelDataType logicalType(IgniteTypeFactory factory);
+
+    /**
+     * Returns java type.
+     *
+     * @param factory Type factory.
+     * @return Java type.
+     */
+    Class<?> javaType(IgniteTypeFactory factory);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/JavaType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/JavaType.java
new file mode 100644
index 0000000..9b9af2e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/JavaType.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.exp.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
+import static org.apache.calcite.rel.type.RelDataType.SCALE_NOT_SPECIFIED;
+
+/** */
+public class JavaType implements DataType {
+    /** */
+    private final Class<?> clazz;
+
+    /** */
+    private final boolean nullable;
+
+    /** {@inheritDoc} */
+    @Override public SqlTypeName typeName() {
+        final SqlTypeName typeName =
+            JavaToSqlTypeConversionRules.instance().lookup(clazz);
+
+        return typeName == null ? SqlTypeName.OTHER : typeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int precision() {
+        return PRECISION_NOT_SPECIFIED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int scale() {
+        return SCALE_NOT_SPECIFIED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullable() {
+        return nullable;
+    }
+
+    /**
+     * @param clazz Value class.
+     * @param nullable nullable flag.
+     */
+    public JavaType(Class<?> clazz, boolean nullable) {
+        assert !nullable || !clazz.isPrimitive();
+
+        this.clazz = clazz;
+        this.nullable = nullable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType logicalType(IgniteTypeFactory factory) {
+        return factory.createJavaType(clazz);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> javaType(IgniteTypeFactory factory) {
+        return clazz;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(clazz);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/StructType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/StructType.java
new file mode 100644
index 0000000..f19c54e
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/type/StructType.java
@@ -0,0 +1,101 @@
+/*
+ * 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.exp.type;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.commons.lang.text.StrBuilder;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+import static org.apache.calcite.rel.type.RelDataType.PRECISION_NOT_SPECIFIED;
+import static org.apache.calcite.rel.type.RelDataType.SCALE_NOT_SPECIFIED;
+
+/** */
+public class StructType implements DataType {
+    /** */
+    private final LinkedHashMap<String, DataType> fields;
+
+    /**
+     * @param fields Fields.
+     */
+    public StructType(LinkedHashMap<String, DataType> fields) {
+        this.fields = fields;
+    }
+
+    public Map<String, DataType> fields() {
+        return fields;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SqlTypeName typeName() {
+        return SqlTypeName.ROW;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int precision() {
+        return PRECISION_NOT_SPECIFIED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int scale() {
+        return SCALE_NOT_SPECIFIED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType logicalType(IgniteTypeFactory factory) {
+        RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(factory);
+        fields.forEach((n,f) -> builder.add(n,f.logicalType(factory)));
+        return builder.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> javaType(IgniteTypeFactory typeFactory) {
+        return Object[].class; // TODO currently a row is mapped to an object array, will be changed in future.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        StrBuilder sb = new StrBuilder("Row(");
+
+        boolean first = true;
+
+        for (Map.Entry<String, DataType> field : fields.entrySet()) {
+            if(first)
+                first = false;
+            else
+                sb.append(", ");
+
+            sb.append("\"")
+                .append(field.getKey())
+                .append("\":")
+                .append(field.getValue());
+        }
+
+        sb.append(")");
+
+        return sb.toString();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
similarity index 59%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
index 8726333..7973753 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
@@ -15,74 +15,52 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
-import com.google.common.collect.ImmutableList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.NotNull;
 
 /**
  * Abstract node of execution tree.
  */
 public abstract class AbstractNode<T> implements Node<T> {
-    /** for debug purpose */
-    private volatile Thread thread;
+    /** */
+    protected static final int IN_BUFFER_SIZE = IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IN_BUFFER_SIZE", 512);
 
     /** */
-    private final ImmutableList<Node<T>> inputs;
+    protected static final int MODIFY_BATCH_SIZE = IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_BATCH_SIZE", 100);
 
     /** */
-    private Sink<T> target;
+    protected static final int IO_BATCH_SIZE = IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IO_BATCH_SIZE", 200);
+
+    /** */
+    protected static final int IO_BATCH_CNT = IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IO_BATCH_CNT", 50);
+
+    /** for debug purpose */
+    private volatile Thread thread;
 
     /**
      * {@link Inbox} node may not have proper context at creation time in case it
      * creates on first message received from a remote source. This case the context
      * sets in scope of {@link Inbox#init(ExecutionContext, Collection, Comparator)} method call.
      */
-    private ExecutionContext ctx;
+    protected ExecutionContext ctx;
 
-    /**
-     * @param ctx Execution context.
-     */
-    protected AbstractNode(ExecutionContext ctx) {
-        this(ctx, ImmutableList.of());
-    }
+    /** */
+    protected Downstream<T> downstream;
 
-    /**
-     * @param ctx Execution context.
-     */
-    protected AbstractNode(ExecutionContext ctx, @NotNull Node<T> input) {
-        this(ctx, ImmutableList.of(input));
-    }
+    /** */
+    protected List<Node<T>> sources;
 
     /**
      * @param ctx Execution context.
      */
-    protected AbstractNode(ExecutionContext ctx, @NotNull List<Node<T>> inputs) {
-        this.ctx = ctx;
-        this.inputs = ImmutableList.copyOf(inputs);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void target(Sink<T> sink) {
-        target = sink;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Sink<T> target() {
-        return target;
-    }
-
-    /** {@inheritDoc} */
-    @Override public List<Node<T>> inputs() {
-        return inputs;
-    }
-
-    /** */
-    protected void context(ExecutionContext ctx) {
+    protected AbstractNode(ExecutionContext ctx) {
         this.ctx = ctx;
     }
 
@@ -92,36 +70,32 @@ public abstract class AbstractNode<T> implements Node<T> {
     }
 
     /** {@inheritDoc} */
-    @Override public void request() {
-        checkThread();
+    @Override public void register(List<Node<T>> sources) {
+        this.sources = sources;
 
-        inputs().forEach(Node::request);
+        for (int i = 0; i < sources.size(); i++)
+            sources.get(i).onRegister(requestDownstream(i));
     }
 
     /** {@inheritDoc} */
-    @Override public void cancel() {
-        checkThread();
-
-        context().setCancelled();
-        inputs().forEach(Node::cancel);
+    @Override public void onRegister(Downstream<T> downstream) {
+        this.downstream = downstream;
     }
 
     /** {@inheritDoc} */
-    @Override public void reset() {
+    @Override public void cancel() {
         checkThread();
 
-        inputs().forEach(Node::reset);
-    }
+        context().markCancelled();
 
-    /**
-     * Links the node inputs to the node sinks.
-     */
-    protected void link() {
-        for (int i = 0; i < inputs.size(); i++)
-            inputs.get(i).target(sink(i));
+        if (!F.isEmpty(sources))
+            sources.forEach(Node::cancel);
     }
 
     /** */
+    protected abstract Downstream<T> requestDownstream(int idx);
+
+    /** */
     protected void checkThread() {
         if (!U.assertionsEnabled())
             return;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
similarity index 71%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
index 8d09c5f..9d1748c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
@@ -15,25 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 /**
  * Represents an abstract data consumer.
  *
- * <p/><b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request()}, {@link Node#cancel()},
- * {@link Node#reset()}, {@link Sink#push(Object)} and {@link Sink#end()} methods should be used from one single thread.
+ * <p/><b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request(int)}, {@link Node#cancel()},
+ * {@link Downstream#push(Object)} and {@link Downstream#end()} methods should be used from one single thread.
  */
-public interface Sink<T> {
+public interface Downstream<T> {
     /**
      * Pushes a row to consumer.
      * @param row Data row.
-     * @return {@code True} if a row consumes and processed, {@code false} otherwise. In case the row was not consumed,
-     *      the row has to be send once again as soon as a target consumer become able to process data.
      */
-    boolean push(T row);
+    void push(T row);
 
     /**
      * Signals that data is over.
      */
     void end();
+
+    /** */
+    void onError(Throwable e);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
new file mode 100644
index 0000000..01869c6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ *
+ */
+public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Downstream<Object[]> {
+    /** */
+    private final Predicate<Object[]> predicate;
+
+    /** */
+    private Deque<Object[]> inBuffer = new ArrayDeque<>(IN_BUFFER_SIZE);
+
+    /** */
+    private int requested;
+
+    /** */
+    private int waiting;
+
+    /** */
+    private boolean inLoop;
+
+    /**
+     * @param ctx Execution context.
+     * @param predicate Predicate.
+     */
+    public FilterNode(ExecutionContext ctx, Predicate<Object[]> predicate) {
+        super(ctx);
+
+        this.predicate = predicate;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCount) {
+        checkThread();
+
+        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert rowsCount > 0 && requested == 0;
+
+        requested = rowsCount;
+
+        if (!inLoop)
+            context().execute(this::flushFromBuffer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void push(Object[] row) {
+        checkThread();
+
+        assert downstream != null;
+        assert waiting > 0;
+
+        waiting--;
+
+        try {
+            if (predicate.test(row))
+                inBuffer.add(row);
+
+            flushFromBuffer();
+        }
+        catch (Exception e) {
+            downstream.onError(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void end() {
+        checkThread();
+
+        assert downstream != null;
+        assert waiting > 0;
+
+        waiting = -1;
+
+        try {
+            flushFromBuffer();
+        }
+        catch (Exception e) {
+            downstream.onError(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(Throwable e) {
+        checkThread();
+
+        assert downstream != null;
+
+        downstream.onError(e);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Object[]> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    /** */
+    public void flushFromBuffer() {
+        inLoop = true;
+        try {
+            while (requested > 0 && !inBuffer.isEmpty()) {
+                requested--;
+                downstream.push(inBuffer.remove());
+            }
+
+            if (inBuffer.isEmpty() && waiting == 0)
+                F.first(sources).request(waiting = IN_BUFFER_SIZE);
+
+            if (waiting == -1 && requested > 0) {
+                assert inBuffer.isEmpty();
+
+                downstream.end();
+                requested = 0;
+            }
+        }
+        finally {
+            inLoop = false;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
similarity index 69%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
index de2118f..a42818c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Inbox.java
@@ -15,19 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
-import java.util.ListIterator;
 import java.util.Map;
 import java.util.PriorityQueue;
 import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
 import org.apache.calcite.util.Pair;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.query.calcite.exec.EndMarker;
+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.jetbrains.annotations.NotNull;
 
 /**
@@ -50,7 +54,7 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
     private final Map<UUID, Buffer> perNodeBuffers;
 
     /** */
-    private Collection<UUID> sources;
+    private Collection<UUID> nodes;
 
     /** */
     private Comparator<T> comparator;
@@ -59,16 +63,19 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
     private List<Buffer> buffers;
 
     /** */
-    private boolean end;
+    private int requested;
+
+    /** */
+    private boolean inLoop;
 
     /**
+     * @param ctx Execution context.
      * @param exchange Exchange service.
      * @param registry Mailbox registry.
-     * @param ctx Execution context.
-     * @param sourceFragmentId Source fragment ID.
      * @param exchangeId Exchange ID.
+     * @param sourceFragmentId Source fragment ID.
      */
-    public Inbox(ExchangeService exchange, MailboxRegistry registry, ExecutionContext ctx, long sourceFragmentId, long exchangeId) {
+    public Inbox(ExecutionContext ctx, ExchangeService exchange, MailboxRegistry registry, long exchangeId, long sourceFragmentId) {
         super(ctx);
         this.exchange = exchange;
         this.registry = registry;
@@ -97,26 +104,39 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
      * Inits this Inbox.
      *
      * @param ctx Execution context.
-     * @param sources Source nodes.
+     * @param nodes Source nodes.
      * @param comparator Optional comparator for merge exchange.
      */
-    public void init(ExecutionContext ctx, Collection<UUID> sources, Comparator<T> comparator) {
+    public void init(ExecutionContext ctx, Collection<UUID> nodes, Comparator<T> comparator) {
+        this.ctx = ctx;
+        this.nodes = nodes;
         this.comparator = comparator;
-        this.sources = sources;
-
-        context(ctx);
     }
 
     /** {@inheritDoc} */
-    @Override public void request() {
-        prepare();
-        pushInternal();
+    @Override public void request(int rowsCount) {
+        checkThread();
+
+        assert nodes != null;
+        assert rowsCount > 0 && requested == 0;
+
+        requested = rowsCount;
+
+        if (buffers == null) {
+            nodes.forEach(this::getOrCreateBuffer);
+            buffers = new ArrayList<>(perNodeBuffers.values());
+
+            assert buffers.size() == nodes.size();
+        }
+
+        if (!inLoop)
+            context().execute(this::pushInternal);
     }
 
     /** {@inheritDoc} */
     @Override public void cancel() {
         checkThread();
-        context().setCancelled();
+        context().markCancelled();
         close();
     }
 
@@ -125,6 +145,16 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
         registry.unregister(this);
     }
 
+    /** {@inheritDoc} */
+    @Override protected Downstream<T> requestDownstream(int idx) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(List<Node<T>> sources) {
+        throw new UnsupportedOperationException();
+    }
+
     /**
      * Pushes a batch into a buffer.
      *
@@ -134,36 +164,43 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
      */
     public void onBatchReceived(UUID source, int batchId, List<?> rows) {
         checkThread();
+
         Buffer buffer = getOrCreateBuffer(source);
 
-        if (buffer.add(batchId, rows))
-            pushInternal();
-    }
+        boolean waitingBefore = buffer.check() == State.WAITING;
 
-    /** {@inheritDoc} */
-    @Override public Sink<T> sink(int idx) {
-        throw new UnsupportedOperationException();
+        buffer.offer(batchId, rows);
+
+        if (requested > 0 && waitingBefore && buffer.check() != State.WAITING)
+            pushInternal();
     }
 
     /** */
     private void pushInternal() {
-        checkThread();
+        assert downstream != null;
 
-        if (context().cancelled())
-            close();
-        else if (!end && ready()) {
+        inLoop = true;
+        try {
             if (comparator != null)
                 pushOrdered();
             else
                 pushUnordered();
         }
+        catch (Exception e) {
+            downstream.onError(e);
+            close();
+        }
+        finally {
+            inLoop = false;
+        }
     }
 
     /** */
-    private void pushOrdered() {
-        PriorityQueue<Pair<T, Buffer>> heap = new PriorityQueue<>(buffers.size(), Map.Entry.comparingByKey(comparator));
+    private void pushOrdered() throws IgniteCheckedException {
+         PriorityQueue<Pair<T, Buffer>> heap =
+            new PriorityQueue<>(buffers.size(), Map.Entry.comparingByKey(comparator));
 
-        ListIterator<Buffer> it = buffers.listIterator();
+        Iterator<Buffer> it = buffers.iterator();
 
         while (it.hasNext()) {
             Buffer buffer = it.next();
@@ -183,17 +220,14 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
             }
         }
 
-        Sink<T> target = target();
-
-        while (!heap.isEmpty()) {
-            Pair<T, Buffer> pair = heap.poll();
-
-            T row = pair.left; Buffer buffer = pair.right;
-
-            if (!target.push(row))
+        while (requested > 0 && !heap.isEmpty()) {
+            if (context().cancelled())
                 return;
 
-            buffer.remove();
+            Buffer buffer = heap.poll().right;
+
+            requested--;
+            downstream.push((T)buffer.remove());
 
             switch (buffer.check()) {
                 case END:
@@ -210,80 +244,59 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
             }
         }
 
-        end = true;
-        target.end();
-        close();
-    }
+        if (requested > 0 && heap.isEmpty()) {
+            assert buffers.isEmpty();
 
-    /** */
-    private void pushUnordered() {
-        int size = buffers.size();
+            downstream.end();
+            requested = 0;
 
-        if (size <= 0 && !end)
-            throw new AssertionError("size=" + size + ", end=" + end);
+            close();
+        }
+    }
 
-        int idx = ThreadLocalRandom.current().nextInt(size);
-        int noProgress = 0;
+    /** */
+    private void pushUnordered() throws IgniteCheckedException {
+        int idx = 0, noProgress = 0;
 
-        Sink<T> target = target();
+        while (requested > 0 && !buffers.isEmpty()) {
+            if (context().cancelled())
+                return;
 
-        while (size > 0) {
             Buffer buffer = buffers.get(idx);
 
             switch (buffer.check()) {
                 case END:
-                    buffers.remove(idx);
-
-                    if (idx == --size)
-                        idx = 0;
+                    buffers.remove(idx--);
 
-                    continue;
+                    break;
                 case READY:
-                    if (!target.push((T)buffer.peek()))
-                        return;
-
-                    buffer.remove();
                     noProgress = 0;
+                    requested--;
+                    downstream.push((T)buffer.remove());
 
                     break;
                 case WAITING:
-                    if (++noProgress >= size)
+                    if (++noProgress >= buffers.size())
                         return;
 
                     break;
             }
 
-            if (++idx == size)
+            if (++idx == buffers.size())
                 idx = 0;
         }
 
-        end = true;
-        target.end();
-        close();
-    }
-
-    /** */
-    private void acknowledge(UUID nodeId, int batchId) {
-        exchange.acknowledge(this, nodeId, queryId(), sourceFragmentId, exchangeId, batchId);
-    }
-
-    /** */
-    private void prepare() {
-        if (ready())
-            return;
-
-        assert sources != null;
-
-        sources.forEach(this::getOrCreateBuffer);
+        if (requested > 0 && buffers.isEmpty()) {
+            downstream.end();
+            requested = 0;
 
-        assert perNodeBuffers.size() == sources.size();
-
-        buffers = new ArrayList<>(perNodeBuffers.values());
+            close();
+        }
     }
 
     /** */
-    private boolean ready() {
-        return buffers != null;
+    private void acknowledge(UUID nodeId, int batchId) throws IgniteCheckedException {
+        exchange.acknowledge(nodeId, queryId(), sourceFragmentId, exchangeId, batchId);
     }
 
     /** */
@@ -359,7 +372,7 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
         private int lastEnqueued = -1;
 
         /** */
-        private final PriorityQueue<Batch> batches = new PriorityQueue<>(ExchangeService.PER_NODE_BATCH_COUNT);
+        private final PriorityQueue<Batch> batches = new PriorityQueue<>(IO_BATCH_CNT);
 
         /** */
         private Batch curr = WAITING;
@@ -371,10 +384,8 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
         }
 
         /** */
-        private boolean add(int id, List<?> rows) {
+        private void offer(int id, List<?> rows) {
             batches.offer(new Batch(id, rows));
-
-            return curr == WAITING && batches.peek().batchId == lastEnqueued + 1;
         }
 
         /** */
@@ -396,10 +407,7 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
             if (curr == END)
                 return State.END;
 
-            if (curr == WAITING)
-                curr = pollBatch();
-
-            if (curr == WAITING)
+            if (curr == WAITING && (curr = pollBatch()) == WAITING)
                 return State.WAITING;
 
             if (curr.rows.get(curr.idx) == EndMarker.INSTANCE) {
@@ -422,7 +430,7 @@ public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoClos
         }
 
         /** */
-        private Object remove() {
+        private Object remove() throws IgniteCheckedException {
             assert curr != null;
             assert curr != WAITING;
             assert curr != END;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinNode.java
new file mode 100644
index 0000000..3afd362
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/JoinNode.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * TODO remove buffers.
+ */
+public class JoinNode extends AbstractNode<Object[]> {
+    /** */
+    private final Predicate<Object[]> condition;
+
+    /** */
+    private int requested;
+
+    /** */
+    private int waitingLeft;
+
+    /** */
+    private int waitingRight;
+
+    /** */
+    private List<Object[]> rightMaterialized = new ArrayList<>(IN_BUFFER_SIZE);
+
+    /** */
+    private Deque<Object[]> leftInBuffer = new ArrayDeque<>(IN_BUFFER_SIZE);
+
+    /** */
+    private boolean inLoop;
+
+    /** */
+    private Object[] left;
+
+    /** */
+    private int rightIdx;
+
+    /**
+     * @param ctx Execution context.
+     * @param condition Join expression.
+     */
+    public JoinNode(ExecutionContext ctx, Predicate<Object[]> condition) {
+        super(ctx);
+
+        this.condition = condition;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCount) {
+        checkThread();
+
+        assert !F.isEmpty(sources) && sources.size() == 2;
+        assert rowsCount > 0 && requested == 0;
+
+        requested = rowsCount;
+
+        if (!inLoop)
+            context().execute(this::flushFromBuffer);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Object[]> requestDownstream(int idx) {
+        if (idx == 0)
+            return new Downstream<Object[]>() {
+                /** {@inheritDoc} */
+                @Override public void push(Object[] row) {
+                    pushLeft(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() {
+                    endLeft();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    JoinNode.this.onError(e);
+                }
+            };
+        else if (idx == 1)
+            return new Downstream<Object[]>() {
+                /** {@inheritDoc} */
+                @Override public void push(Object[] row) {
+                    pushRight(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() {
+                    endRight();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    JoinNode.this.onError(e);
+                }
+            };
+
+        throw new IndexOutOfBoundsException();
+    }
+
+    /** */
+    private void pushLeft(Object[] row) {
+        checkThread();
+
+        assert downstream != null;
+        assert waitingLeft > 0;
+
+        leftInBuffer.add(row);
+
+        flushFromBuffer();
+    }
+
+    /** */
+    private void pushRight(Object[] row) {
+        checkThread();
+
+        assert downstream != null;
+        assert waitingRight > 0;
+
+        waitingRight--;
+
+        rightMaterialized.add(row);
+
+        if (waitingRight == 0)
+            sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
+    }
+
+    /** */
+    private void endLeft() {
+        checkThread();
+
+        assert downstream != null;
+        assert waitingLeft > 0;
+
+        waitingLeft = -1;
+
+        flushFromBuffer();
+    }
+
+    /** */
+    private void endRight() {
+        checkThread();
+
+        assert downstream != null;
+        assert waitingRight > 0;
+
+        waitingRight = -1;
+
+        flushFromBuffer();
+    }
+
+    /** */
+    private void onError(Throwable e) {
+        checkThread();
+
+        assert downstream != null;
+
+        downstream.onError(e);
+    }
+
+    /** */
+    private void flushFromBuffer() {
+        inLoop = true;
+        try {
+            if (waitingRight == -1) {
+                while (requested > 0 && (left != null || !leftInBuffer.isEmpty())) {
+                    if (left == null)
+                        left = leftInBuffer.remove();
+
+                    while (requested > 0 && rightIdx < rightMaterialized.size()) {
+                        Object[] row = F.concat(left, rightMaterialized.get(rightIdx++));
+
+                        if (!condition.test(row))
+                            continue;
+
+                        requested--;
+                        downstream.push(row);
+                    }
+
+                    if (rightIdx == rightMaterialized.size()) {
+                        left = null;
+                        rightIdx = 0;
+                    }
+                }
+            }
+
+            if (waitingRight == 0)
+                sources.get(1).request(waitingRight = IN_BUFFER_SIZE);
+
+            if (waitingLeft == 0 && leftInBuffer.isEmpty())
+                sources.get(0).request(waitingLeft = IN_BUFFER_SIZE);
+
+            if (requested > 0 && waitingLeft == -1 && waitingRight == -1 && left == null && leftInBuffer.isEmpty()) {
+                downstream.end();
+                requested = 0;
+            }
+        }
+        catch (Exception e) {
+            downstream.onError(e);
+        }
+        finally {
+            inLoop = false;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ModifyNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java
similarity index 59%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ModifyNode.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java
index fa76d9d..976d1a0 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ModifyNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -28,6 +28,7 @@ import org.apache.calcite.rel.core.TableModify;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 import org.apache.ignite.internal.processors.query.calcite.schema.TableDescriptor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -38,10 +39,7 @@ import static org.apache.ignite.internal.processors.cache.query.IgniteQueryError
 /**
  *
  */
-public class ModifyNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
-    /** */
-    private static final int BATCH_SIZE = 100;
-
+public class ModifyNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Downstream<Object[]> {
     /** */
     protected final TableDescriptor desc;
 
@@ -52,137 +50,192 @@ public class ModifyNode extends AbstractNode<Object[]> implements SingleNode<Obj
     private final List<String> columns;
 
     /** */
-    private final Map<Object,Object> tuples = U.newLinkedHashMap(BATCH_SIZE);
+    private List<IgniteBiTuple<?, ?>> tuples = new ArrayList<>(MODIFY_BATCH_SIZE);
 
     /** */
     private long updatedRows;
 
     /** */
+    private int waiting;
+
+    /** */
+    private int requested;
+
+    /** */
+    private boolean inLoop;
+
+    /** */
     private State state = State.UPDATING;
 
     /**
      * @param ctx Execution context.
      * @param desc Table descriptor.
      * @param columns Update column list.
-     * @param input Input node.
      */
-    protected ModifyNode(ExecutionContext ctx, TableDescriptor desc, TableModify.Operation op, List<String> columns, Node<Object[]> input) {
-        super(ctx, input);
+    public ModifyNode(ExecutionContext ctx, TableDescriptor desc, TableModify.Operation op, List<String> columns) {
+        super(ctx);
 
         this.desc = desc;
         this.op = op;
         this.columns = columns;
-
-        link();
     }
 
     /** {@inheritDoc} */
-    @Override public Sink<Object[]> sink(int idx) {
-        if (idx != 0)
-            throw new IndexOutOfBoundsException();
+    @Override public void request(int rowsCount) {
+        checkThread();
 
-        return this;
+        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert rowsCount > 0 && requested == 0;
+
+        requested = rowsCount;
+
+        if (!inLoop)
+            tryEnd();
     }
 
     /** {@inheritDoc} */
-    @Override public void request() {
+    @Override public void push(Object[] row) {
         checkThread();
 
-        if (state == State.UPDATING)
-            input().request();
-        else if (state == State.UPDATED)
-            endInternal();
-        else
-            assert state == State.END;
-    }
+        assert downstream != null;
+        assert waiting > 0;
+        assert state == State.UPDATING;
 
-    /** {@inheritDoc} */
-    @Override public boolean push(Object[] row) {
-        if (state != State.UPDATING)
-            return false;
+        waiting--;
 
-        switch (op) {
-            case DELETE:
-            case UPDATE:
-            case INSERT:
-                addToBatch(row);
+        try {
+            switch (op) {
+                case DELETE:
+                case UPDATE:
+                case INSERT:
+                    addToBatch(row);
+
+                    break;
+                default:
+                    throw new UnsupportedOperationException(op.name());
+            }
 
-                break;
-            default:
-                throw new UnsupportedOperationException(op.name());
+            if (waiting == 0)
+                F.first(sources).request(waiting = MODIFY_BATCH_SIZE);
+        }
+        catch (Exception e) {
+            downstream.onError(e);
         }
-
-        return true;
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
+        checkThread();
+
+        assert downstream != null;
+        assert waiting > 0;
+
+        waiting = -1;
         state = State.UPDATED;
 
-        endInternal();
+        tryEnd();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(Throwable e) {
+        checkThread();
+
+        assert downstream != null;
+
+        downstream.onError(e);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Object[]> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    /** */
+    private void addToBatch(Object[] row) throws IgniteCheckedException {
+        tuples.add(desc.toTuple(context(), row, op, columns));
+
+        flush(false);
     }
 
     /** */
-    private void addToBatch(Object[] row) {
+    private void tryEnd() {
+        assert downstream != null;
+
+        inLoop = true;
         try {
-            IgniteBiTuple<?, ?> t = desc.toTuple(context(), row, op, columns);
-            tuples.put(t.getKey(), t.getValue());
-            flush(false);
+            if (state == State.UPDATING && waiting == 0)
+                F.first(sources).request(waiting = MODIFY_BATCH_SIZE);
+
+            if (state == State.UPDATED && requested > 0) {
+                flush(true);
+
+                state = State.END;
+
+                requested--;
+                downstream.push(new Object[]{updatedRows});
+            }
+
+            if (state == State.END && requested > 0) {
+                downstream.end();
+                requested = 0;
+            }
+        }
+        catch (Exception e) {
+            downstream.onError(e);
         }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        finally {
+            inLoop = false;
         }
     }
 
     /** */
     @SuppressWarnings({"rawtypes", "unchecked"})
-    private void flush(boolean force) {
-        if (F.isEmpty(tuples) || !force && tuples.size() < BATCH_SIZE)
+    private void flush(boolean force) throws IgniteCheckedException {
+        if (F.isEmpty(tuples) || !force && tuples.size() < MODIFY_BATCH_SIZE)
             return;
 
-        try {
-            Map<Object, EntryProcessorResult<Long>> res =
-                ((GridCacheAdapter) desc.cacheContext().cache()).invokeAll(invokeMap());
+        List<IgniteBiTuple<?, ?>> tuples = this.tuples;
+        this.tuples = new ArrayList<>(MODIFY_BATCH_SIZE);
 
-            long updated = res.values().stream().mapToLong(EntryProcessorResult::get).sum();
+        Map<Object, EntryProcessorResult<Long>> res =
+            ((GridCacheAdapter) desc.cacheContext().cache()).invokeAll(invokeMap(tuples));
 
-            if (op == TableModify.Operation.INSERT && updated != res.size()) {
-                List<Object> duplicates = new ArrayList<>(res.size());
+        long updated = res.values().stream().mapToLong(EntryProcessorResult::get).sum();
 
-                for (Map.Entry<Object, EntryProcessorResult<Long>> e : res.entrySet()) {
-                    if (e.getValue().get() == 0)
-                        duplicates.add(e.getKey());
-                }
+        if (op == TableModify.Operation.INSERT && updated != res.size()) {
+            List<Object> duplicates = new ArrayList<>(res.size());
 
-                throw duplicateKeysException(duplicates);
+            for (Map.Entry<Object, EntryProcessorResult<Long>> e : res.entrySet()) {
+                if (e.getValue().get() == 0)
+                    duplicates.add(e.getKey());
             }
 
-            updatedRows += updated;
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+            throw duplicateKeysException(duplicates);
         }
 
-        tuples.clear();
+        updatedRows += updated;
     }
 
     /** */
-    private Map<Object, EntryProcessor<Object, Object, Long>> invokeMap() {
-        Map<Object, EntryProcessor<Object, Object, Long>> procMap = U.newLinkedHashMap(BATCH_SIZE);
+    private Map<Object, EntryProcessor<Object, Object, Long>> invokeMap(List<IgniteBiTuple<?,?>> tuples) {
+        Map<Object, EntryProcessor<Object, Object, Long>> procMap = U.newLinkedHashMap(tuples.size());
 
         switch (op) {
             case INSERT:
-                for (Map.Entry<Object, Object> entry : tuples.entrySet())
+                for (IgniteBiTuple<?, ?> entry : tuples)
                     procMap.put(entry.getKey(), new InsertOperation(entry.getValue()));
 
                 break;
             case UPDATE:
-                for (Map.Entry<Object, Object> entry : tuples.entrySet())
+                for (IgniteBiTuple<?, ?> entry : tuples)
                     procMap.put(entry.getKey(), new UpdateOperation(entry.getValue()));
 
                 break;
             case DELETE:
-                for (Map.Entry<Object, Object> entry : tuples.entrySet())
+                for (IgniteBiTuple<?, ?> entry : tuples)
                     procMap.put(entry.getKey(), new DeleteOperation());
 
                 break;
@@ -200,16 +253,6 @@ public class ModifyNode extends AbstractNode<Object[]> implements SingleNode<Obj
     }
 
     /** */
-    private void endInternal() {
-        flush(true);
-
-        if (target().push(new Object[]{updatedRows})) {
-            state = State.END;
-            target().end();
-        }
-    }
-
-    /** */
     private enum State {
         UPDATING, UPDATED, END
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java
similarity index 56%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java
index f0e9506..32929ca 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Node.java
@@ -15,15 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 import java.util.List;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 
 /**
  * Represents a node of execution tree.
  *
- * <p/><b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request()}, {@link Node#cancel()},
- * {@link Node#reset()}, {@link Sink#push(Object)} and {@link Sink#end()} methods should be used from one single thread.
+ * <p/><b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request(int)}, {@link Node#cancel()},
+ * {@link Downstream#push(Object)} and {@link Downstream#end()} methods should be used from one single thread.
  */
 public interface Node<T> {
     /**
@@ -34,51 +35,26 @@ public interface Node<T> {
     ExecutionContext context();
 
     /**
-     * Requests a sink of the node. The sink is used to push data into the node by its children.
+     * Registers node sources.
      *
-     * @param idx Sink index.
-     * @return Sink object.
-     * @throws IndexOutOfBoundsException in case there is no Sink object associated with given index.
+     * @param sources Sources collection.
      */
-    Sink<T> sink(int idx);
+    void register(List<Node<T>> sources);
 
     /**
-     * Registers target sink.
+     * Registers downstream.
      *
-     * @param sink Target sink.
+     * @param downstream Downstream.
      */
-    void target(Sink<T> sink);
+    void onRegister(Downstream<T> downstream);
 
     /**
-     * @return Registered target.
+     * Requests next bunch of rows.
      */
-    Sink<T> target();
-
-    /**
-     * @return Node inputs collection.
-     */
-    List<Node<T>> inputs();
-
-    /**
-     * @param idx Input index.
-     * @return Node input.
-     */
-    default Node<T> input(int idx) {
-        return inputs().get(idx);
-    }
-
-    /**
-     * Signals that consumer is ready to consume data.
-     */
-    void request();
+    void request(int rowsCount);
 
     /**
      * Cancels execution.
      */
     void cancel();
-
-    /**
-     * Resets execution sub-tree to initial state.
-     */
-    void reset();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
similarity index 55%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
index 3840cb4..180e652 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Outbox.java
@@ -15,59 +15,70 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
+import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.query.calcite.exec.EndMarker;
+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.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  * A part of exchange.
  */
-public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>, AutoCloseable {
+public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Downstream<T>, AutoCloseable {
     /** */
     private final ExchangeService exchange;
 
     /** */
-    private final long exchangeId;
+    private final MailboxRegistry registry;
 
     /** */
-    private final MailboxRegistry registry;
+    private final long exchangeId;
 
     /** */
     private final long targetFragmentId;
 
     /** */
-    private final Map<UUID, Buffer> perNodeBuffers = new HashMap<>();
+    private final Destination destination;
 
     /** */
-    private final Destination destination;
+    private final Deque<T> inBuffer = new ArrayDeque<>(IN_BUFFER_SIZE);
+
+    /** */
+    private final Map<UUID, Buffer> nodeBuffers = new HashMap<>();
 
     /** */
     private boolean cancelled;
 
+    /** */
+    private int waiting;
+
     /**
+     * @param ctx Execution context.
      * @param exchange Exchange service.
      * @param registry Mailbox registry.
-     * @param ctx Execution context.
-     * @param targetFragmentId Target fragment ID.
      * @param exchangeId Exchange ID.
-     * @param input Input node.
+     * @param targetFragmentId Target fragment ID.
      * @param destination Destination.
      */
-    public Outbox(ExchangeService exchange, MailboxRegistry registry, ExecutionContext ctx, long targetFragmentId, long exchangeId, Node<T> input, Destination destination) {
-        super(ctx, input);
+    public Outbox(ExecutionContext ctx, ExchangeService exchange, MailboxRegistry registry, long exchangeId, long targetFragmentId, Destination destination) {
+        super(ctx);
         this.exchange = exchange;
         this.registry = registry;
         this.targetFragmentId = targetFragmentId;
         this.exchangeId = exchangeId;
         this.destination = destination;
-
-        link();
     }
 
     /**
@@ -91,104 +102,112 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
      * @param batchId Batch ID.
      */
     public void onAcknowledge(UUID nodeId, int batchId) {
-        perNodeBuffers.get(nodeId).onAcknowledge(batchId);
+        nodeBuffers.get(nodeId).onAcknowledge(batchId);
     }
 
-    /** {@inheritDoc} */
-    @Override public void request() {
+    /** */
+    public void init() {
         checkThread();
 
-        if (context().cancelled())
-            cancelInternal();
-        else
-            input().request();
+        flushFromBuffer();
     }
 
     /** {@inheritDoc} */
-    @Override public void cancel() {
+    @Override public void push(T row) {
         checkThread();
 
-        context().setCancelled();
-        cancelInternal();
+        assert waiting > 0;
+
+        waiting--;
+
+        inBuffer.add(row);
+
+        flushFromBuffer();
     }
 
-    /** */
-    private void cancelInternal() {
-        if (cancelled)
-            return;
+    /** {@inheritDoc} */
+    @Override public void end() {
+        checkThread();
+
+        assert waiting > 0;
+
+        waiting = -1;
 
         try {
-            perNodeBuffers.values().forEach(Buffer::cancel);
-            input().cancel();
-        }
-        finally {
-            cancelled = true;
+            for (UUID node : destination.targets())
+                getOrCreateBuffer(node).end();
+
             close();
         }
+        catch (Exception e) {
+            onError(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void target(Sink<T> sink) {
-        throw new AssertionError();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Sink<T> sink(int idx) {
-        if (idx != 0)
-            throw new IndexOutOfBoundsException();
-
-        return this;
+    @Override public void onError(Throwable e) {
+        cancel(); // TODO send cause to originator.
     }
 
     /** {@inheritDoc} */
-    @Override public boolean push(T row) {
-        List<UUID> nodes = destination.targets(row);
+    @Override public void cancel() {
+        checkThread();
 
-        assert !F.isEmpty(nodes);
+        context().markCancelled();
 
-        List<Buffer> buffers = new ArrayList<>(nodes.size());
+        if (cancelled)
+            return;
 
-        for (UUID node : nodes) {
-            Buffer dest = getOrCreateBuffer(node);
+        cancelled = true;
 
-            if (!dest.ready())
-                return false;
+        nodeBuffers.values().forEach(Buffer::cancel);
 
-            buffers.add(dest);
-        }
+        close();
 
-        for (Buffer dest : buffers)
-            dest.add(row);
+        super.cancel();
+    }
 
-        return true;
+    /** {@inheritDoc} */
+    @Override public void close() {
+        registry.unregister(this);
     }
 
     /** {@inheritDoc} */
-    @Override public void end() {
-        for (UUID node : destination.targets())
-            getOrCreateBuffer(node).end();
+    @Override public void request(int rowCount) {
+        throw new UnsupportedOperationException();
+    }
 
-        close();
+    /** {@inheritDoc} */
+    @Override public void onRegister(Downstream<T> downstream) {
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public void close() {
-        registry.unregister(this);
+    @Override protected Downstream<T> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
     }
 
     /** */
-    private void sendBatch(UUID nodeId, int batchId, List<?> rows) {
-        exchange.sendBatch(this, nodeId, queryId(), targetFragmentId, exchangeId, batchId, rows);
+    private void sendBatch(UUID nodeId, int batchId, List<?> rows) throws IgniteCheckedException {
+        exchange.sendBatch(nodeId, queryId(), targetFragmentId, exchangeId, batchId, rows);
     }
 
     /** */
     private void sendCancel(UUID nodeId, int batchId) {
-        exchange.cancel(this, nodeId, queryId(), targetFragmentId, exchangeId, batchId);
+        try {
+            exchange.cancel(nodeId, queryId(), targetFragmentId, exchangeId, batchId);
+        }
+        catch (IgniteCheckedException e) {
+            U.warn(context().parent().logger(), "Failed to send cancel message.", e);
+        }
     }
 
     /** */
     private Buffer getOrCreateBuffer(UUID nodeId) {
-        return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+        return nodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
     }
 
     /** */
@@ -197,6 +216,42 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
     }
 
     /** */
+    private void flushFromBuffer() {
+        try {
+            while (!inBuffer.isEmpty()) {
+                T row = inBuffer.remove();
+
+                List<UUID> nodes = destination.targets(row);
+
+                assert !F.isEmpty(nodes);
+
+                List<Buffer> buffers = new ArrayList<>(nodes.size());
+
+                for (UUID node : nodes) {
+                    Buffer dest = getOrCreateBuffer(node);
+
+                    if (dest.ready())
+                        buffers.add(dest);
+                    else {
+                        inBuffer.addFirst(row);
+
+                        return;
+                    }
+                }
+
+                for (Buffer dest : buffers)
+                    dest.add(row);
+            }
+
+            if (waiting == 0)
+                F.first(sources).request(waiting = IN_BUFFER_SIZE);
+        }
+        catch (Exception e) {
+            onError(e);
+        }
+    }
+
+    /** */
     private static final class Buffer {
         /** */
         private final Outbox<?> owner;
@@ -218,7 +273,7 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
             this.nodeId = nodeId;
             this.owner = owner;
 
-            curr = new ArrayList<>(ExchangeService.BATCH_SIZE + 1); // extra space for end marker;
+            curr = new ArrayList<>(IO_BATCH_SIZE + 1); // extra space for end marker;
         }
 
         /**
@@ -226,12 +281,13 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
          *
          * @param row Row.
          */
-        public void add(Object row) {
+        public void add(Object row) throws IgniteCheckedException {
             assert ready();
 
-            if (curr.size() == ExchangeService.BATCH_SIZE) {
+            if (curr.size() == IO_BATCH_SIZE) {
                 owner.sendBatch(nodeId, ++hwm, curr);
-                curr = new ArrayList<>(ExchangeService.BATCH_SIZE + 1); // extra space for end marker;
+
+                curr = new ArrayList<>(IO_BATCH_SIZE + 1); // extra space for end marker;
             }
 
             curr.add(row);
@@ -240,7 +296,10 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
         /**
          * Signals data is over.
          */
-        public void end() {
+        public void end() throws IgniteCheckedException {
+            if (hwm == Integer.MAX_VALUE)
+                return;
+
             int batchId = hwm + 1;
             hwm = Integer.MAX_VALUE;
 
@@ -252,6 +311,9 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
         }
 
         public void cancel() {
+            if (hwm == Integer.MAX_VALUE)
+                return;
+
             int batchId = hwm + 1;
             hwm = Integer.MAX_VALUE;
 
@@ -265,12 +327,10 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
          * @return {@code True} is it possible to add a row to a batch.
          */
         private boolean ready() {
-            if (curr == null)
-                throw new AssertionError();
+            if (hwm == Integer.MAX_VALUE)
+                return false;
 
-            return hwm != Integer.MAX_VALUE
-                && hwm - lwm < ExchangeService.PER_NODE_BATCH_COUNT
-                || curr.size() < ExchangeService.BATCH_SIZE;
+            return curr.size() < IO_BATCH_SIZE || hwm - lwm < IO_BATCH_CNT;
         }
 
         /**
@@ -282,12 +342,12 @@ public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>
             if (lwm > id)
                 return;
 
-            boolean request = hwm - lwm == ExchangeService.PER_NODE_BATCH_COUNT;
+            boolean readyBefore = ready();
 
             lwm = id;
 
-            if (request)
-                owner.request();
+            if (!readyBefore && ready())
+                owner.flushFromBuffer();
         }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java
similarity index 55%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java
index 796bacd..59bf28d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ProjectNode.java
@@ -15,14 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 import java.util.function.Function;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.util.typedef.F;
 
 /**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ *
  */
-public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
+public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Downstream<Object[]> {
     /** */
     private final Function<Object[], Object[]> projection;
 
@@ -30,29 +32,59 @@ public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Ob
      * @param ctx Execution context.
      * @param projection Projection.
      */
-    public ProjectNode(ExecutionContext ctx, Node<Object[]> input, Function<Object[], Object[]> projection) {
-        super(ctx, input);
+    public ProjectNode(ExecutionContext ctx, Function<Object[], Object[]> projection) {
+        super(ctx);
 
         this.projection = projection;
-
-        link();
     }
 
     /** {@inheritDoc} */
-    @Override public Sink<Object[]> sink(int idx) {
-        if (idx != 0)
-            throw new IndexOutOfBoundsException();
+    @Override public void request(int rowsCount) {
+        checkThread();
 
-        return this;
+        assert !F.isEmpty(sources) && sources.size() == 1;
+        assert rowsCount > 0;
+
+        F.first(sources).request(rowsCount);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean push(Object[] row) {
-        return target().push(projection.apply(row));
+    @Override public void push(Object[] row) {
+        checkThread();
+
+        assert downstream != null;
+
+        try {
+            downstream.push(projection.apply(row));
+        }
+        catch (Throwable e) {
+            downstream.onError(e);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        target().end();
+        checkThread();
+
+        assert downstream != null;
+
+        downstream.end();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(Throwable e) {
+        checkThread();
+
+        assert downstream != null;
+
+        downstream.onError(e);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Object[]> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
similarity index 60%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
index 19a474c..3eb6ffe 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/RootNode.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
 import java.util.ArrayDeque;
 import java.util.Iterator;
@@ -25,21 +25,18 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
 import org.apache.ignite.IgniteInterruptedException;
-import org.apache.ignite.cache.query.QueryCancelledException;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.calcite.exec.EndMarker;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.util.typedef.F;
 
 /**
  * Client iterator.
  */
-public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]>, AutoCloseable {
+public class RootNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Downstream<Object[]>, Iterator<Object[]>, AutoCloseable {
     /** */
-    public static final Consumer<ConsumerNode> NO_OP = c -> {};
-
-    /** */
-    private static final int DEFAULT_BUFFER_SIZE = 1000;
-
-    /** */
-    private enum State {
+    public enum State {
         RUNNING, CANCELLED, END
     }
 
@@ -50,61 +47,35 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
     private final Condition cond;
 
     /** */
-    private final int bufferSize;
-
-    /** */
     private final ArrayDeque<Object> buff;
 
     /** */
-    private final Consumer<ConsumerNode> onClose;
+    private final Consumer<RootNode> onClose;
 
     /** */
-    private Object cur;
+    private volatile State state = State.RUNNING;
 
     /** */
-    private boolean requested;
+    private volatile IgniteSQLException ex;
 
     /** */
-    private volatile State state = State.RUNNING;
+    private Object row;
 
-    /**
-     * @param ctx Execution context.
-     */
-    public ConsumerNode(ExecutionContext ctx, Node<Object[]> input) {
-        this(ctx, input, DEFAULT_BUFFER_SIZE);
-    }
-
-    /**
-     * @param ctx Execution context.
-     */
-    public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, Consumer<ConsumerNode> onClose) {
-        this(ctx, input, DEFAULT_BUFFER_SIZE, onClose);
-    }
-
-    /**
-     * @param ctx        Execution context.
-     * @param input      Input node.
-     * @param bufferSize Buffer size.
-     */
-    public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, int bufferSize) {
-        this(ctx, input, bufferSize, NO_OP);
-    }
+    /** */
+    private int waiting;
 
     /**
      * @param ctx Execution context.
      */
-    public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, int bufferSize, Consumer<ConsumerNode> onClose) {
-        super(ctx, input);
+    public RootNode(ExecutionContext ctx, Consumer<RootNode> onClose) {
+        super(ctx);
 
-        this.bufferSize = bufferSize;
         this.onClose = onClose;
 
         // extra space for possible END marker
-        buff = new ArrayDeque<>(bufferSize + 1);
+        buff = new ArrayDeque<>(IN_BUFFER_SIZE + 1);
         lock = new ReentrantLock();
         cond = lock.newCondition();
-
-        link();
     }
     
     public UUID queryId() {
@@ -112,27 +83,6 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
     }
 
     /** {@inheritDoc} */
-    @Override public Sink<Object[]> sink(int idx) {
-        if (idx != 0)
-            throw new IndexOutOfBoundsException();
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void request() {
-        context().execute(this::requestInternal);
-    }
-
-    /** */
-    private void requestInternal() {
-        checkThread();
-
-        if (state == State.RUNNING)
-            input().request();
-    }
-
-    /** {@inheritDoc} */
     @Override public void cancel() {
         if (state != State.RUNNING)
             return;
@@ -142,7 +92,7 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
             if (state != State.RUNNING)
                 return;
 
-            context().setCancelled();
+            context().markCancelled();
             state = State.CANCELLED;
             buff.clear();
             cond.signalAll();
@@ -151,15 +101,15 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
             lock.unlock();
         }
         
-        context().execute(input()::cancel);
+        context().execute(F.first(sources)::cancel);
         onClose.accept(this);
     }
 
     /**
-     * @return Cancelled flag.
+     * @return Execution state.
      */
-    boolean canceled() {
-        return state == State.CANCELLED;
+    public State state() {
+        return state;
     }
 
     /** {@inheritDoc} */
@@ -168,37 +118,43 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
     }
 
     /** {@inheritDoc} */
-    @Override public boolean push(Object[] row) {
-        if (state != State.RUNNING)
-            return false;
+    @Override public void push(Object[] row) {
+        checkThread();
+
+        int request = 0;
 
         lock.lock();
         try {
-            if (state != State.RUNNING)
-                return false;
+            assert waiting > 0;
 
-            if (buff.size() == bufferSize) {
-                requested = false;
-                return false;
-            }
+            waiting--;
+
+            if (state != State.RUNNING)
+                return;
 
             buff.offer(row);
+
+            if (waiting == 0)
+                waiting = request = IN_BUFFER_SIZE - buff.size();
+
             cond.signalAll();
         }
         finally {
             lock.unlock();
         }
 
-        return true;
+        if (request > 0)
+            F.first(sources).request(request);
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        if (state != State.RUNNING)
-            return;
-
         lock.lock();
         try {
+            assert waiting > 0;
+
+            waiting = -1;
+
             if (state != State.RUNNING)
                 return;
 
@@ -211,13 +167,22 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
     }
 
     /** {@inheritDoc} */
+    @Override public void onError(Throwable e) {
+        checkThread();
+
+        ex = new IgniteSQLException("An error occurred while query executing.", IgniteQueryErrorCode.UNKNOWN, e);
+
+        cancel();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasNext() {
-        if (cur != null)
+        if (row != null)
             return true;
         else if (state == State.END)
             return false;
         else
-            return (cur = take()) != null;
+            return (row = take()) != null;
     }
 
     /** {@inheritDoc} */
@@ -225,21 +190,35 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
         if (!hasNext())
             throw new NoSuchElementException();
 
-        Object cur0 = cur;
-        cur = null;
+        Object cur0 = row;
+        row = null;
 
         return (Object[]) cur0;
     }
 
+    /** {@inheritDoc} */
+    @Override protected Downstream<Object[]> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onRegister(Downstream<Object[]> downstream) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCount) {
+        throw new UnsupportedOperationException();
+    }
+
     /** */
     private Object take() {
-        if (state == State.CANCELLED)
-            throw U.convertException(new QueryCancelledException());
-
         lock.lock();
         try {
-            if (state == State.CANCELLED)
-                throw U.convertException(new QueryCancelledException());
+            checkCancelled();
 
             assert state == State.RUNNING;
 
@@ -248,8 +227,7 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
 
                 cond.await();
 
-                if (state == State.CANCELLED)
-                    throw U.convertException(new QueryCancelledException());
+                checkCancelled();
 
                 assert state == State.RUNNING;
             }
@@ -276,11 +254,27 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
     }
 
     /** */
+    private void checkCancelled() {
+        if (state == State.CANCELLED) {
+            if (ex != null)
+                throw ex;
+
+            throw new IgniteSQLException("The query was cancelled while executing.", IgniteQueryErrorCode.QUERY_CANCELED);
+        }
+    }
+
+    /** */
     private void requestIfNeeded() {
-        if (requested)
+        assert !F.isEmpty(sources) && sources.size() == 1;
+
+        assert lock.isHeldByCurrentThread();
+
+        if (waiting != 0)
             return;
 
-        request();
-        requested = true;
+        int request = waiting = IN_BUFFER_SIZE - buff.size();
+
+        if (request > 0)
+            context().execute(() -> F.first(sources).request(request));
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java
new file mode 100644
index 0000000..bcc838d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ScanNode.java
@@ -0,0 +1,131 @@
+/*
+ * 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.Iterator;
+import java.util.List;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * Scan node.
+ */
+public class ScanNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, AutoCloseable {
+    /** */
+    private final Iterable<Object[]> source;
+
+    /** */
+    private Iterator<Object[]> it;
+
+    /** */
+    private int requested;
+
+    /** */
+    private boolean inLoop;
+
+    /**
+     * @param ctx Execution context.
+     * @param source Source.
+     */
+    public ScanNode(ExecutionContext ctx, Iterable<Object[]> source) {
+        super(ctx);
+
+        this.source = source;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCount) {
+        checkThread();
+
+        assert rowsCount > 0 && requested == 0;
+
+        requested = rowsCount;
+
+        if (!inLoop)
+            context().execute(this::pushInternal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        checkThread();
+        context().markCancelled();
+        close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        Commons.closeQuiet(it);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(List<Node<Object[]>> sources) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Object[]> requestDownstream(int idx) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** */
+    private void pushInternal() {
+        inLoop = true;
+        try {
+            if (it == null)
+                it = source.iterator();
+
+            int processed = 0;
+
+            Thread thread = Thread.currentThread();
+
+            while (requested > 0 && it.hasNext()) {
+                if (context().cancelled())
+                    return;
+
+                if (thread.isInterrupted())
+                    throw new IgniteInterruptedCheckedException("Thread was interrupted.");
+
+                requested--;
+                downstream.push(it.next());
+
+                if (++processed == IN_BUFFER_SIZE && requested > 0) {
+                    // allow others to do their job
+                    context().execute(this::pushInternal);
+
+                    return;
+                }
+            }
+
+            if (requested > 0 && !it.hasNext()) {
+                downstream.end();
+                requested = 0;
+
+                close();
+            }
+        }
+        catch (Throwable e) {
+            close();
+
+            downstream.onError(e);
+        }
+        finally {
+            inLoop = false;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
similarity index 77%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
index cebb0d2..ea732be 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/SingleNode.java
@@ -15,25 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
-import java.util.Objects;
+import org.apache.ignite.internal.util.typedef.F;
 
 /**
  * A node with a single input
  */
 public interface SingleNode<T> extends Node<T> {
-    /**
-     * @return Single sink object.
-     */
-    default Sink<T> sink() {
-        return Objects.requireNonNull(sink(0));
-    }
-
-    /**
-     * @return Single input.
-     */
-    default Node<T> input() {
-        return Objects.requireNonNull(input(0));
+    /** */
+    default void register(Node<T> source) {
+        register(F.asList(source));
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
index c5c386d..5d7b675 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
@@ -22,7 +22,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
+import org.apache.ignite.internal.processors.query.calcite.serialize.PhysicalRel;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -48,10 +48,10 @@ public class QueryStartRequest implements MarshalableMessage {
 
     /** */
     @GridDirectTransient
-    private RelGraph plan;
+    private PhysicalRel root;
 
     /** */
-    private byte[] planBytes;
+    private byte[] rootBytes;
 
     /** */
     @GridDirectTransient
@@ -61,13 +61,13 @@ public class QueryStartRequest implements MarshalableMessage {
     private byte[] paramsBytes;
 
     /** */
-    public QueryStartRequest(UUID queryId, long fragmentId, String schema, RelGraph plan, AffinityTopologyVersion version, int[] partitions, Object[] params) {
+    public QueryStartRequest(UUID queryId, long fragmentId, String schema, PhysicalRel root, AffinityTopologyVersion version, int[] partitions, Object[] params) {
         this.schema = schema;
         this.queryId = queryId;
         this.fragmentId = fragmentId;
         this.partitions = partitions;
         this.version = version;
-        this.plan = plan;
+        this.root = root;
         this.params = params;
     }
 
@@ -114,8 +114,8 @@ public class QueryStartRequest implements MarshalableMessage {
     /**
      * @return Fragment plan.
      */
-    public RelGraph plan() {
-        return plan;
+    public PhysicalRel root() {
+        return root;
     }
 
     /**
@@ -127,8 +127,8 @@ public class QueryStartRequest implements MarshalableMessage {
 
     /** {@inheritDoc} */
     @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
-        if (planBytes == null && plan != null)
-            planBytes = marshaller.marshal(plan);
+        if (rootBytes == null && root != null)
+            rootBytes = marshaller.marshal(root);
 
         if (paramsBytes == null && params != null)
             paramsBytes = marshaller.marshal(params);
@@ -136,8 +136,8 @@ public class QueryStartRequest implements MarshalableMessage {
 
     /** {@inheritDoc} */
     @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
-        if (plan == null && planBytes != null)
-            plan = marshaller.unmarshal(planBytes, loader);
+        if (root == null && rootBytes != null)
+            root = marshaller.unmarshal(rootBytes, loader);
 
         if (params == null && paramsBytes != null)
             params = marshaller.unmarshal(paramsBytes, loader);
@@ -174,7 +174,7 @@ public class QueryStartRequest implements MarshalableMessage {
                 writer.incrementState();
 
             case 3:
-                if (!writer.writeByteArray("planBytes", planBytes))
+                if (!writer.writeByteArray("rootBytes", rootBytes))
                     return false;
 
                 writer.incrementState();
@@ -235,7 +235,7 @@ public class QueryStartRequest implements MarshalableMessage {
                 reader.incrementState();
 
             case 3:
-                planBytes = reader.readByteArray("planBytes");
+                rootBytes = reader.readByteArray("rootBytes");
 
                 if (!reader.isLastRead())
                     return false;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
index 3273d61..b651406 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
@@ -58,11 +58,6 @@ import org.apache.calcite.util.Pair;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.GraphToRelConverter;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 
 /**
@@ -153,13 +148,6 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
         validator = null;
     }
 
-    /**
-     * @return Planner context.
-     */
-    public PlanningContext context() {
-        return ctx;
-    }
-
     /** {@inheritDoc} */
     @Override public SqlNode parse(Reader reader) throws SqlParseException {
         SqlNodeList sqlNodes = SqlParser.create(reader, parserConfig).parseStmtList();
@@ -201,49 +189,17 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
 
     /** {@inheritDoc} */
     @Override public RelNode convert(SqlNode sql) {
-        return rel(sql).rel;
-    }
-
-    /**
-     * Converts intermediate relational nodes tree representation into a relational nodes tree, bounded to the planner.
-     *
-     * @param graph Relational nodes tree representation.
-     * @return Root node of relational tree.
-     */
-    public IgniteRel convert(RelGraph graph) {
-        RelOptCluster cluster = createCluster();
-        RelBuilder relBuilder = createRelBuilder(cluster, catalogReader);
-
-        return new GraphToRelConverter(this, relBuilder, operatorTable).convert(graph);
+        return rel(sql).project();
     }
 
     /** {@inheritDoc} */
     @Override public RelRoot rel(SqlNode sql) {
-        RelOptCluster cluster = createCluster();
-        SqlToRelConverter.Config config = SqlToRelConverter.configBuilder()
-            .withConfig(sqlToRelConverterConfig)
-            .withTrimUnusedFields(false)
-            .withConvertTableAccess(false)
-            .build();
-        SqlToRelConverter sqlToRelConverter =
-            new SqlToRelConverter(this, validator, catalogReader, cluster, convertletTable, config);
+        SqlToRelConverter sqlToRelConverter = new SqlToRelConverter(this,
+            validator, catalogReader, createCluster(), convertletTable, sqlToRelConverterConfig);
 
         return sqlToRelConverter.convertQuery(sql, false, true);
     }
 
-    /**
-     * Creates an intermediate relational nodes tree representation for a given relational nodes tree.
-     *
-     * @param rel Root node of relational tree.
-     * @return Relational nodes tree representation.
-     */
-    public RelGraph graph(RelNode rel) {
-        if (rel.getConvention() != IgniteConvention.INSTANCE)
-            throw new IllegalArgumentException("Physical node is required.");
-
-        return new RelToGraphConverter().go((IgniteRel) rel);
-    }
-
     /** {@inheritDoc} */
     @Override public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
         SqlParser parser = SqlParser.create(queryString, parserConfig);
@@ -260,15 +216,8 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
         SqlValidator validator = new IgniteSqlValidator(operatorTable, catalogReader, typeFactory, conformance);
         validator.setIdentifierExpansion(true);
 
-        RelOptCluster cluster = createCluster();
-        SqlToRelConverter.Config config = SqlToRelConverter
-            .configBuilder()
-            .withConfig(sqlToRelConverterConfig)
-            .withTrimUnusedFields(false)
-            .withConvertTableAccess(false)
-            .build();
-        SqlToRelConverter sqlToRelConverter =
-            new SqlToRelConverter(this, validator, catalogReader, cluster, convertletTable, config);
+        SqlToRelConverter sqlToRelConverter = new SqlToRelConverter(this,
+            validator, catalogReader, createCluster(), convertletTable, sqlToRelConverterConfig);
 
         return sqlToRelConverter.convertQuery(sqlNode, true, false);
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
index 04cf177..da20d13 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
@@ -25,10 +25,11 @@ import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.sql.SqlAccessEnum;
+import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDelete;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlOperatorTable;
@@ -40,12 +41,13 @@ import org.apache.calcite.sql.validate.SelectScope;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.calcite.schema.TableDescriptor;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.type.SystemType;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteResource;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -122,20 +124,27 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
     }
 
     /** {@inheritDoc} */
+    @Override public void validateCall(SqlCall call, SqlValidatorScope scope) {
+        if (call.getKind() == SqlKind.AS) {
+            final String alias = deriveAlias(call, 0);
+
+            if (isSystemFieldName(alias))
+                throw newValidationError(call, IgniteResource.INSTANCE.illegalAlias(alias));
+        }
+
+        super.validateCall(call, scope);
+    }
+
+    /** {@inheritDoc} */
     @Override protected void addToSelectList(List<SqlNode> list, Set<String> aliases,
         List<Map.Entry<String, RelDataType>> fieldList, SqlNode exp, SelectScope scope, boolean includeSystemVars) {
-        if (includeSystemVars || !isSystemType(deriveType(scope, exp)))
+        if (includeSystemVars || exp.getKind() != SqlKind.IDENTIFIER || !isSystemFieldName(deriveAlias(exp, 0)))
             super.addToSelectList(list, aliases, fieldList, exp, scope, includeSystemVars);
     }
 
     /** {@inheritDoc} */
     @Override public boolean isSystemField(RelDataTypeField field) {
-        return isSystemType(field.getType());
-    }
-
-    /** */
-    private boolean isSystemType(RelDataType type) {
-        return type instanceof SystemType;
+        return isSystemFieldName(field.getName());
     }
 
     /** */
@@ -190,7 +199,7 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
 
             if (!desc.isUpdateAllowed(relOptTable, target.getIndex()))
                 throw newValidationError(id,
-                    RESOURCE.accessNotAllowed(SqlAccessEnum.UPDATE.name(), id.toString()));
+                    IgniteResource.INSTANCE.cannotUpdateField(id.toString()));
         }
     }
 
@@ -221,4 +230,10 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
     private IgniteTypeFactory typeFactory() {
         return (IgniteTypeFactory) typeFactory;
     }
+
+    /** */
+    private boolean isSystemFieldName(String alias) {
+        return QueryUtils.KEY_FIELD_NAME.equalsIgnoreCase(alias)
+            || QueryUtils.VAL_FIELD_NAME.equalsIgnoreCase(alias);
+    }
 }
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 72da044..e4c3dc0 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
@@ -25,7 +25,6 @@ import org.apache.calcite.tools.RuleSets;
 import org.apache.ignite.internal.processors.query.calcite.rule.FilterConverter;
 import org.apache.ignite.internal.processors.query.calcite.rule.JoinConverter;
 import org.apache.ignite.internal.processors.query.calcite.rule.ProjectConverter;
-import org.apache.ignite.internal.processors.query.calcite.rule.TableConverter;
 import org.apache.ignite.internal.processors.query.calcite.rule.TableModifyConverter;
 import org.apache.ignite.internal.processors.query.calcite.rule.ValuesConverter;
 
@@ -43,7 +42,6 @@ public enum PlannerPhase {
         /** {@inheritDoc} */
         @Override public RuleSet getRules(PlanningContext ctx) {
             return RuleSets.ofList(
-                TableConverter.INSTANCE,
                 ValuesConverter.INSTANCE,
                 SubQueryRemoveRule.FILTER,
                 SubQueryRemoveRule.PROJECT,
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 6f52188..0824040 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
@@ -29,6 +29,8 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.ignite.IgniteLogger;
@@ -171,6 +173,20 @@ public final class PlanningContext implements Context {
     // Helper methods
 
     /**
+     * @return Sql operators table.
+     */
+    public SqlOperatorTable opTable() {
+        return config().getOperatorTable();
+    }
+
+    /**
+     * @return Sql conformance.
+     */
+    public SqlConformance conformance() {
+        return config.getParserConfig().conformance();
+    }
+
+    /**
      * @return Planner.
      */
     public IgnitePlanner planner() {
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java
index 8e7d477..b376c58 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Splitter.java
@@ -34,17 +34,16 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
-import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
 
 /**
  * Splits a query into a list of query fragments.
  */
-public class Splitter implements IgniteRelVisitor<IgniteRel>, RelOp<IgniteRel, List<Fragment>> {
+public class Splitter implements IgniteRelVisitor<IgniteRel> {
     /** */
     private List<Fragment> fragments;
 
-    /** {@inheritDoc} */
-    @Override public List<Fragment> go(IgniteRel root) {
+    /** */
+    public List<Fragment> go(IgniteRel root) {
         fragments = new ArrayList<>();
 
         try {
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableConverter.java
deleted file mode 100644
index fc4e8ff..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/TableConverter.java
+++ /dev/null
@@ -1,48 +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.rule;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.calcite.plan.ViewExpanders;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.logical.LogicalTableScan;
-import org.jetbrains.annotations.NotNull;
-
-/**
- *
- */
-public class TableConverter extends IgniteConverter {
-    /** */
-    public static final ConverterRule INSTANCE = new TableConverter();
-
-    /**
-     * Creates a ConverterRule.
-     */
-    public TableConverter() {
-        super(LogicalTableScan.class, "TableConverter");
-    }
-
-    /** {@inheritDoc} */
-    @Override protected List<RelNode> convert0(@NotNull RelNode rel) {
-        LogicalTableScan scan = (LogicalTableScan) rel;
-
-        return Collections.singletonList(scan.getTable().toRel(ViewExpanders.simpleContext(scan.getCluster())));
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
index fe30e75..d172480 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
@@ -100,8 +100,8 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory implem
         // A _key/_val fields is virtual in case there is an alias or a property(es) mapped to _key/_val object fields.
         BitSet virtualFlags = new BitSet();
 
-        descriptors.add(new KeyValDescriptor(QueryUtils.KEY_FIELD_NAME, typeDesc.keyClass(), true, true));
-        descriptors.add(new KeyValDescriptor(QueryUtils.VAL_FIELD_NAME, typeDesc.valueClass(), true, false));
+        descriptors.add(new KeyValDescriptor(QueryUtils.KEY_FIELD_NAME, typeDesc.keyClass(), true));
+        descriptors.add(new KeyValDescriptor(QueryUtils.VAL_FIELD_NAME, typeDesc.valueClass(), false));
 
         for (String field : fields) {
             if (Objects.equals(field, typeDesc.affinityKey()))
@@ -115,12 +115,12 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory implem
 
                 virtualFlags.set(0);
 
-                descriptors.add(new KeyValDescriptor(typeDesc.keyFieldAlias(), typeDesc.keyClass(), false, true));
+                descriptors.add(new KeyValDescriptor(typeDesc.keyFieldAlias(), typeDesc.keyClass(), true));
             }
             else if (Objects.equals(field, typeDesc.valueFieldAlias())) {
                 valField = descriptors.size();
 
-                descriptors.add(new KeyValDescriptor(typeDesc.valueFieldAlias(), typeDesc.valueClass(), false, false));
+                descriptors.add(new KeyValDescriptor(typeDesc.valueFieldAlias(), typeDesc.valueClass(), false));
 
                 virtualFlags.set(1);
             }
@@ -426,16 +426,12 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory implem
         private final Class<?> type;
 
         /** */
-        private final boolean isSystem;
-
-        /** */
         private final boolean isKey;
 
         /** */
-        private KeyValDescriptor(String name, Class<?> type, boolean isSystem, boolean isKey) {
+        private KeyValDescriptor(String name, Class<?> type, boolean isKey) {
             this.name = name;
             this.type = type;
-            this.isSystem = isSystem;
             this.isKey = isKey;
         }
 
@@ -466,7 +462,7 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory implem
 
         /** {@inheritDoc} */
         @Override public RelDataType logicalType(IgniteTypeFactory f) {
-            return isSystem ? f.createSystemType(javaType()) : f.createJavaType(javaType());
+            return f.createJavaType(javaType());
         }
 
         /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/FilterPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/FilterPhysicalRel.java
new file mode 100644
index 0000000..41346a6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/FilterPhysicalRel.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.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Expression;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+
+/**
+ * Describes {@link IgniteFilter}
+ */
+public class FilterPhysicalRel implements PhysicalRel {
+    /** */
+    private DataType rowType;
+
+    /** */
+    private Expression condition;
+
+    /** */
+    private PhysicalRel input;
+
+    /** */
+    public FilterPhysicalRel() {
+    }
+
+    /** */
+    public FilterPhysicalRel(DataType rowType, Expression condition, PhysicalRel input) {
+        this.rowType = rowType;
+        this.condition = condition;
+        this.input = input;
+    }
+
+    /** */
+    public DataType rowType() {
+        return rowType;
+    }
+
+    /** */
+    public Expression condition() {
+        return condition;
+    }
+
+    /** */
+    public PhysicalRel input() {
+        return input;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(rowType);
+        out.writeObject(condition);
+        out.writeObject(input);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        rowType = (DataType) in.readObject();
+        condition = (Expression) in.readObject();
+        input = (PhysicalRel) in.readObject();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/JoinPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/JoinPhysicalRel.java
new file mode 100644
index 0000000..e1dce81
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/JoinPhysicalRel.java
@@ -0,0 +1,94 @@
+/*
+ * 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.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Expression;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+
+/**
+ * Describes {@link IgniteJoin}.
+ */
+public class JoinPhysicalRel implements PhysicalRel {
+    /** */
+    private DataType rowType;
+
+    /** */
+    private PhysicalRel left;
+
+    /** */
+    private PhysicalRel right;
+
+    /** */
+    private Expression condition;
+
+    public JoinPhysicalRel() {
+    }
+
+    /** */
+    public JoinPhysicalRel(DataType rowType, PhysicalRel left, PhysicalRel right, Expression condition) {
+        this.rowType = rowType;
+        this.left = left;
+        this.right = right;
+        this.condition = condition;
+    }
+
+    /** */
+    public DataType rowType() {
+        return rowType;
+    }
+
+    /** */
+    public PhysicalRel left() {
+        return left;
+    }
+
+    /** */
+    public PhysicalRel right() {
+        return right;
+    }
+
+    /** */
+    public Expression condition() {
+        return condition;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(rowType);
+        out.writeObject(left);
+        out.writeObject(right);
+        out.writeObject(condition);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        rowType = (DataType) in.readObject();
+        left = (PhysicalRel) in.readObject();
+        right = (PhysicalRel) in.readObject();
+        condition = (Expression) in.readObject();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/PhysicalRel.java
similarity index 71%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/PhysicalRel.java
index 52819e3..ca0747d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/Expression.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/PhysicalRel.java
@@ -15,19 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.serialize;
 
-import java.io.Serializable;
+import java.io.Externalizable;
 
 /**
- * Describes {@link org.apache.calcite.rex.RexNode}
+ * A superclass of all relational nodes representations.
  */
-public interface Expression extends Serializable {
+public interface PhysicalRel extends Externalizable {
     /**
-     * Delegates implementation to given implementor using Visitor pattern.
+     * Accepts a visit from a visitor.
      *
-     * @param implementor Implementor.
-     * @return Expression implementation.
+     * @param visitor Visitor.
+     * @return Visit result.
      */
-    <T> T implement(ExpImplementor<T> implementor);
+    <T> T accept(PhysicalRelVisitor<T> visitor);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/PhysicalRelVisitor.java
similarity index 50%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/PhysicalRelVisitor.java
index b2cff09..ca12805 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpImplementor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/PhysicalRelVisitor.java
@@ -15,42 +15,57 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.expression;
+package org.apache.ignite.internal.processors.query.calcite.serialize;
 
 /**
- * Implements Expression tree recursively using Visitor pattern.
+ *
  */
-public interface ExpImplementor<T> {
+public interface PhysicalRelVisitor<T> {
+    /**
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
+     */
+    T visit(TableScanPhysicalRel rel);
+
+    /**
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
+     */
+    T visit(FilterPhysicalRel rel);
+
+    /**
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
+     */
+    T visit(ProjectPhysicalRel rel);
+
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
      */
-    T implement(CallExpression exp);
+    T visit(JoinPhysicalRel rel);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
      */
-    T implement(InputRefExpression exp);
+    T visit(ReceiverPhysicalRel rel);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
      */
-    T implement(LiteralExpression exp);
+    T visit(ValuesPhysicalRel rel);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
      */
-    T implement(LocalRefExpression exp);
+    T visit(SenderPhysicalRel rel);
 
     /**
-     * See {@link ExpImplementor#implement(Expression)}
+     * See {@link PhysicalRelVisitor#visit(PhysicalRel)}
      */
-    T implement(DynamicParamExpression exp);
+    T visit(TableModifyPhysicalRel rel);
 
     /**
-     * Implements given expression.
+     * Processes given physical rel.
      *
-     * @param exp Expression.
-     * @return Implementation result.
+     * @param rel Physical rel.
+     * @return Visit result.
      */
-    T implement(Expression exp);
+    T visit(PhysicalRel rel);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ProjectPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ProjectPhysicalRel.java
new file mode 100644
index 0000000..cb0b1fb
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ProjectPhysicalRel.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.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Expression;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+
+/**
+ * Describes {@link IgniteProject}.
+ */
+public class ProjectPhysicalRel implements PhysicalRel {
+    /** */
+    private DataType rowType;
+
+    /** */
+    private List<Expression> projects;
+
+    /** */
+    private PhysicalRel input;
+
+    public ProjectPhysicalRel() {
+    }
+
+    /** */
+    public ProjectPhysicalRel(DataType rowType, List<Expression> projects, PhysicalRel input) {
+        this.rowType = rowType;
+        this.projects = projects;
+        this.input = input;
+    }
+
+    /** */
+    public DataType rowType() {
+        return rowType;
+    }
+
+    /** */
+    public List<Expression> projects() {
+        return projects;
+    }
+
+    /** */
+    public PhysicalRel input() {
+        return input;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(rowType);
+
+        out.writeInt(projects.size());
+
+        for (Expression project : projects)
+            out.writeObject(project);
+
+        out.writeObject(input);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        rowType = (DataType) in.readObject();
+
+        int projectsSize = in.readInt();
+
+        List<Expression> projects = new ArrayList<>(projectsSize);
+
+        for (int i = 0; i < projectsSize; i++)
+            projects.add((Expression) in.readObject());
+
+        this.projects = projects;
+
+        input = (PhysicalRel) in.readObject();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ReceiverPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ReceiverPhysicalRel.java
new file mode 100644
index 0000000..113ef3c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ReceiverPhysicalRel.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+
+
+/**
+ * Describes {@link IgniteReceiver}.
+ */
+public class ReceiverPhysicalRel implements PhysicalRel {
+    /** */
+    private DataType rowType;
+
+    /** */
+    private long sourceFragmentId;
+
+    /** */
+    private List<UUID> sources;
+
+    /** */
+    private List<RelCollation> collations;
+
+    /** */
+    public ReceiverPhysicalRel() {
+    }
+
+    /** */
+    public ReceiverPhysicalRel(DataType rowType, long sourceFragmentId, List<UUID> sources, List<RelCollation> collations) {
+        this.rowType = rowType;
+        this.sourceFragmentId = sourceFragmentId;
+        this.sources = sources;
+        this.collations = collations;
+    }
+
+    /** */
+    public DataType rowType() {
+        return rowType;
+    }
+
+    /** */
+    public long sourceFragmentId() {
+        return sourceFragmentId;
+    }
+
+    /** */
+    public Collection<UUID> sources() {
+        return sources;
+    }
+
+    /** */
+    public List<RelCollation> collations() {
+        return collations;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(rowType);
+        out.writeLong(sourceFragmentId);
+
+        out.writeInt(sources.size());
+
+        for (UUID source : sources)
+            out.writeObject(source);
+
+        if (collations == null)
+            out.writeInt(-1);
+        else {
+            out.writeInt(collations.size());
+
+            for (RelCollation collation : collations) {
+                List<RelFieldCollation> fields = collation.getFieldCollations();
+
+                out.writeInt(fields.size());
+
+                for (RelFieldCollation field : fields) {
+                    out.writeInt(field.getFieldIndex());
+                    out.writeByte(field.direction.ordinal());
+                    out.writeByte(field.nullDirection.ordinal());
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        rowType = (DataType) in.readObject();
+        sourceFragmentId = in.readLong();
+
+        final int sourcesSize = in.readInt();
+
+        List<UUID> sources = new ArrayList<>(sourcesSize);
+
+        for (int i = 0; i < sourcesSize; i++)
+            sources.add((UUID) in.readObject());
+
+        int collationsSize = in.readInt();
+
+        if (collationsSize == -1)
+            return;
+
+        List<RelCollation> collations = new ArrayList<>(collationsSize);
+
+        for (int i = 0; i < collationsSize; i++) {
+            int fieldsSize = in.readInt();
+
+            List<RelFieldCollation> fields = new ArrayList<>(fieldsSize);
+
+            for (int j = 0; j < fieldsSize; j++) {
+                int fieldIndex = in.readInt();
+                RelFieldCollation.Direction direction = RelFieldCollation.Direction.values()[in.readByte()];
+                RelFieldCollation.NullDirection nullDirection = RelFieldCollation.NullDirection.values()[in.readByte()];
+
+                fields.add(new RelFieldCollation(fieldIndex, direction, nullDirection));
+            }
+            collations.add(RelCollationTraitDef.INSTANCE.canonize(RelCollations.of(fields)));
+        }
+
+        this.collations = collations;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/RelToPhysicalConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/RelToPhysicalConverter.java
new file mode 100644
index 0000000..c11b793
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/RelToPhysicalConverter.java
@@ -0,0 +1,132 @@
+/*
+ * 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.serialize;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Expression;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.type.DataType;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ * Converts RelNode tree to physical rel tree.
+ */
+public class RelToPhysicalConverter implements IgniteRelVisitor<PhysicalRel> {
+    /** */
+    private final RexToExpTranslator rexTranslator;
+
+    /** */
+    private final IgniteTypeFactory typeFactory;
+
+    /** */
+    public RelToPhysicalConverter(IgniteTypeFactory typeFactory) {
+        this.typeFactory = typeFactory;
+
+        rexTranslator = new RexToExpTranslator(typeFactory);
+    }
+
+    /** */
+    public PhysicalRel go(IgniteRel root) {
+        return visit(root);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteSender rel) {
+        long targetFragmentId = rel.target().fragmentId();
+        NodesMapping targetMapping = rel.target().mapping();
+        DistributionFunction targetFunction = rel.target().distribution().function();
+        ImmutableIntList distributionKeys = rel.target().distribution().getKeys();
+
+        return new SenderPhysicalRel(targetFragmentId, targetMapping, targetFunction, distributionKeys, visit((IgniteRel) rel.getInput()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteFilter rel) {
+        return new FilterPhysicalRel(DataType.fromType(rel.getRowType()),
+            rexTranslator.translate(rel.getCondition()), visit((IgniteRel) rel.getInput()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteProject rel) {
+        return new ProjectPhysicalRel(DataType.fromType(rel.getInput().getRowType()),
+            rexTranslator.translate(rel.getProjects()), visit((IgniteRel) rel.getInput()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteJoin rel) {
+        DataType dataType = DataType.fromType(
+            Commons.combinedRowType(
+                typeFactory, rel.getLeft().getRowType(), rel.getRight().getRowType()));
+
+        return new JoinPhysicalRel(dataType, visit((IgniteRel) rel.getLeft()), visit((IgniteRel) rel.getRight()), rexTranslator.translate(rel.getCondition()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteTableScan rel) {
+        return new TableScanPhysicalRel(rel.getTable().getQualifiedName());
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteReceiver rel) {
+        return new ReceiverPhysicalRel(DataType.fromType(rel.getRowType()),
+            rel.source().fragmentId(), rel.source().mapping().nodes(), rel.collations());
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteTableModify rel) {
+        return new TableModifyPhysicalRel(rel.getTable().getQualifiedName(),
+            rel.getOperation(), rel.getUpdateColumnList(), visit((IgniteRel) rel.getInput()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteValues rel) {
+        return new ValuesPhysicalRel(toValues(rel.getTuples()), rel.getRowType().getFieldCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteExchange rel) {
+        throw new AssertionError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public PhysicalRel visit(IgniteRel rel) {
+        return rel.accept(this);
+    }
+
+    /** */
+    private List<Expression> toValues(ImmutableList<ImmutableList<RexLiteral>> tuples) {
+        return rexTranslator.translate(Commons.flat(Commons.cast(tuples)));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/SenderPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/SenderPhysicalRel.java
new file mode 100644
index 0000000..22bc5be
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/SenderPhysicalRel.java
@@ -0,0 +1,98 @@
+/*
+ * 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.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
+
+/**
+ * Describes {@link IgniteSender}.
+ */
+public class SenderPhysicalRel implements PhysicalRel {
+    /** */
+    private long targetFragmentId;
+
+    /** */
+    private NodesMapping mapping;
+
+    /** */
+    private DistributionFunction function;
+
+    /** */
+    private ImmutableIntList distributionKeys;
+
+    /** */
+    private PhysicalRel input;
+
+    public SenderPhysicalRel() {
+    }
+
+    /** */
+    public SenderPhysicalRel(long targetFragmentId, NodesMapping mapping, DistributionFunction function, ImmutableIntList distributionKeys, PhysicalRel input) {
+        this.targetFragmentId = targetFragmentId;
+        this.mapping = mapping;
+        this.function = function;
+        this.distributionKeys = distributionKeys;
+        this.input = input;
+    }
+
+    public long targetFragmentId() {
+        return targetFragmentId;
+    }
+
+    public NodesMapping mapping() {
+        return mapping;
+    }
+
+    public DistributionFunction distributionFunction() {
+        return function;
+    }
+
+    public ImmutableIntList distributionKeys() {
+        return distributionKeys;
+    }
+
+    public PhysicalRel input() {
+        return input;
+    }
+
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(targetFragmentId);
+        out.writeObject(mapping);
+        out.writeObject(function);
+        out.writeObject(distributionKeys.toIntArray());
+        out.writeObject(input);
+    }
+
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        targetFragmentId = in.readLong();
+        mapping = (NodesMapping) in.readObject();
+        function = (DistributionFunction) in.readObject();
+        distributionKeys = ImmutableIntList.of((int[]) in.readObject());
+        input = (PhysicalRel) in.readObject();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/TableModifyPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/TableModifyPhysicalRel.java
new file mode 100644
index 0000000..651ff7b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/TableModifyPhysicalRel.java
@@ -0,0 +1,115 @@
+/*
+ * 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.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.rel.core.TableModify;
+
+/**
+ *
+ */
+public class TableModifyPhysicalRel implements PhysicalRel {
+    /** */
+    private List<String> tableName;
+
+    /** */
+    private TableModify.Operation operation;
+
+    /** */
+    private List<String> updateColumnList;
+
+    /** */
+    private PhysicalRel input;
+
+    public TableModifyPhysicalRel() {
+    }
+
+    /***/
+    public TableModifyPhysicalRel(List<String> tableName, TableModify.Operation operation, List<String> updateColumnList, PhysicalRel input) {
+        this.tableName = tableName;
+        this.operation = operation;
+        this.updateColumnList = updateColumnList;
+        this.input = input;
+    }
+
+    public List<String> tableName() {
+        return tableName;
+    }
+
+    public TableModify.Operation operation() {
+        return operation;
+    }
+
+    public List<String> updateColumnList() {
+        return updateColumnList;
+    }
+
+    public PhysicalRel input() {
+        return input;
+    }
+
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(tableName.size());
+
+        for (String name : tableName)
+            out.writeUTF(name);
+
+        out.writeByte(operation.ordinal());
+
+        if (updateColumnList == null)
+            out.writeInt(-1);
+        else {
+            out.writeInt(updateColumnList.size());
+
+            for (String column : updateColumnList)
+                out.writeUTF(column);
+        }
+
+        out.writeObject(input);
+    }
+
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        int tableNameSize = in.readInt();
+
+        List<String> tableName = new ArrayList<>(tableNameSize);
+
+        for (int i = 0; i < tableNameSize; i++)
+            tableName.add(in.readUTF());
+
+        operation = TableModify.Operation.values()[in.readByte()];
+
+        int columnsSize = in.readInt();
+
+        if (columnsSize != -1) {
+            List<String> updateColumnList = new ArrayList<>(columnsSize);
+
+            for (int i = 0; i < columnsSize; i++)
+                updateColumnList.add(in.readUTF());
+        }
+
+        input = (PhysicalRel) in.readObject();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/TableScanPhysicalRel.java
similarity index 52%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/TableScanPhysicalRel.java
index a37e2a2..92f5f1d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableScanNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/TableScanPhysicalRel.java
@@ -15,47 +15,59 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize.relation;
+package org.apache.ignite.internal.processors.query.calcite.serialize;
 
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
 import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 
 /**
  * Describes {@link IgniteTableScan}.
  */
-public class TableScanNode extends RelGraphNode {
+public class TableScanPhysicalRel implements PhysicalRel {
     /** */
-    private final List<String> tableName;
+    private List<String> tableName;
+
+    /** */
+    public TableScanPhysicalRel() {
+    }
 
     /**
-     * @param traits   Traits of this relational expression
      * @param tableName Qualified table name
      */
-    private TableScanNode(RelTraitSet traits, List<String> tableName) {
-        super(traits);
+    public TableScanPhysicalRel(List<String> tableName) {
         this.tableName = tableName;
     }
 
-    /**
-     * Factory method.
-     *
-     * @param rel TableScan rel.
-     * @return TableScanNode.
-     */
-    public static TableScanNode create(IgniteTableScan rel) {
-        return new TableScanNode(rel.getTraitSet(), rel.getTable().getQualifiedName());
+    public List<String> tableName() {
+        return tableName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        RelOptCluster cluster = ctx.getCluster();
-        RelOptTable table = ctx.getSchema().getTableForMember(tableName);
-        RelTraitSet traits = traitSet(cluster);
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(tableName.size());
 
-        return new IgniteTableScan(cluster, traits, table);
+        for (String name : tableName)
+            out.writeUTF(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        int tableNameSize = in.readInt();
+
+        List<String> tableName = new ArrayList<>(tableNameSize);
+
+        for (int i = 0; i < tableNameSize; i++)
+            tableName.add(in.readUTF());
+
+        this.tableName = tableName;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ValuesPhysicalRel.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ValuesPhysicalRel.java
new file mode 100644
index 0000000..7ad80be
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/ValuesPhysicalRel.java
@@ -0,0 +1,79 @@
+/*
+ * 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.serialize;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Expression;
+
+/**
+ *
+ */
+public class ValuesPhysicalRel implements PhysicalRel {
+    /** */
+    private List<Expression> values;
+
+    /** */
+    private int rowLen;
+
+    public ValuesPhysicalRel() {
+    }
+
+    /** */
+    public ValuesPhysicalRel(List<Expression> values, int rowLen) {
+        this.values = values;
+        this.rowLen = rowLen;
+    }
+
+    public List<Expression> values() {
+        return values;
+    }
+
+    public int rowLength() {
+        return rowLen;
+    }
+
+    @Override public <T> T accept(PhysicalRelVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(rowLen);
+
+        out.writeInt(values.size());
+
+        for (Expression value : values)
+            out.writeObject(value);
+    }
+
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        rowLen = in.readInt();
+
+        int valuesSize = in.readInt();
+
+        List<Expression> values = new ArrayList<>(valuesSize);
+
+        for (int i = 0; i < valuesSize; i++)
+            values.add((Expression) in.readObject());
+
+        this.values = values;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
deleted file mode 100644
index 124df13..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
+++ /dev/null
@@ -1,87 +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.serialize.expression;
-
-import java.util.List;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSyntax;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
-
-/**
- * Describes {@link org.apache.calcite.rex.RexCall}.
- */
-public class CallExpression implements Expression {
-    /** */
-    private final String opName;
-
-    /** */
-    private final SqlSyntax opSyntax;
-
-    /** */
-    private DataType type;
-
-    /** */
-    private final List<Expression> operands;
-
-    /**
-     * @param op Sql operation.
-     * @param type Result type.
-     * @param operands Operands.
-     */
-    public CallExpression(SqlOperator op, RelDataType type, List<Expression> operands) {
-        opName = op.getName();
-        this.type = Types.fromType(type);
-        this.operands = operands;
-        opSyntax = op.getSyntax();
-    }
-
-    /**
-     * @return Operation name.
-     */
-    public String name() {
-        return opName;
-    }
-
-    /**
-     * @return Operation syntax;
-     */
-    public SqlSyntax syntax() {
-        return opSyntax;
-    }
-
-    /**
-     * @return Data type.
-     */
-    public DataType dataType() {
-        return type;
-    }
-
-    /**
-     * @return Operands.
-     */
-    public List<Expression> operands() {
-        return operands;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T implement(ExpImplementor<T> implementor) {
-        return implementor.implement(this);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java
deleted file mode 100644
index aaea04e..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/ExpToRexTranslator.java
+++ /dev/null
@@ -1,117 +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.serialize.expression;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexLocalRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.util.Pair;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-/**
- * A translator of Expressions into Rex nodes.
- */
-public class ExpToRexTranslator implements ExpImplementor<RexNode> {
-    /** */
-    private final RexBuilder builder;
-
-    /** */
-    private final Map<Pair<String, SqlSyntax>, SqlOperator> ops;
-
-    /**
-     * Creates a Translator.
-     *
-     * @param builder Rex builder.
-     * @param opTable Operators table.
-     */
-    public ExpToRexTranslator(RexBuilder builder, SqlOperatorTable opTable) {
-        this.builder = builder;
-
-        List<SqlOperator> opList = opTable.getOperatorList();
-
-        HashMap<Pair<String, SqlSyntax>, SqlOperator> ops = U.newHashMap(opList.size());
-
-        for (SqlOperator op : opList)
-            ops.put(Pair.of(op.getName(), op.getSyntax()), op);
-
-        this.ops = ops;
-    }
-
-    /**
-     * Translates a list of expressions into a list of Rex nodes.
-     *
-     * @param exps List of expressions.
-     * @return List of Rex nodes.
-     */
-    public List<RexNode> translate(List<Expression> exps) {
-        return Commons.transform(exps, this::translate);
-    }
-
-    /**
-     * Translates an expression into a RexNode.
-     *
-     * @param exp Expression.
-     * @return RexNode.
-     */
-    public RexNode translate(Expression exp) {
-        return exp.implement(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public RexNode implement(CallExpression exp) {
-        return builder.makeCall(exp.dataType().toRelDataType((IgniteTypeFactory) builder.getTypeFactory()), op(exp.name(), exp.syntax()), translate(exp.operands()));
-    }
-
-    /** {@inheritDoc} */
-    @Override public RexNode implement(InputRefExpression exp) {
-        return builder.makeInputRef(exp.dataType().toRelDataType((IgniteTypeFactory) builder.getTypeFactory()), exp.index());
-    }
-
-    /** {@inheritDoc} */
-    @Override public RexNode implement(LiteralExpression exp) {
-        return builder.makeLiteral(exp.value(), exp.dataType().toRelDataType((IgniteTypeFactory) builder.getTypeFactory()), false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public RexNode implement(LocalRefExpression exp) {
-        return new RexLocalRef(exp.index(), exp.dataType().toRelDataType((IgniteTypeFactory) builder.getTypeFactory()));
-    }
-
-    /** {@inheritDoc} */
-    @Override public RexNode implement(DynamicParamExpression exp) {
-        return builder.makeDynamicParam(exp.dataType().toRelDataType((IgniteTypeFactory) builder.getTypeFactory()), exp.index());
-    }
-
-    /** {@inheritDoc} */
-    @Override public RexNode implement(Expression exp) {
-        return exp.implement(this);
-    }
-
-    /** */
-    private SqlOperator op(String name, SqlSyntax syntax) {
-        return ops.get(Pair.of(name, syntax));
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java
deleted file mode 100644
index 521a3229..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/LocalRefExpression.java
+++ /dev/null
@@ -1,61 +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.serialize.expression;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
-
-/**
- * Describes {@link org.apache.calcite.rex.RexLocalRef}.
- */
-public class LocalRefExpression implements Expression {
-    /** */
-    private final DataType type;
-
-    /** */
-    private final int index;
-
-    /**
-     * @param type Data type.
-     * @param index Index.
-     */
-    public LocalRefExpression(RelDataType type, int index) {
-        this.type = Types.fromType(type);
-        this.index = index;
-    }
-
-    /**
-     * @return Data type.
-     */
-    public DataType dataType() {
-        return type;
-    }
-
-    /**
-     * @return Index.
-     */
-    public int index() {
-        return index;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T implement(ExpImplementor<T> implementor) {
-        return implementor.implement(this);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java
deleted file mode 100644
index 9b6a673..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ConversionContext.java
+++ /dev/null
@@ -1,49 +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.serialize.relation;
-
-import org.apache.calcite.plan.RelOptSchema;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-
-/**
- * Provides context to complete RelGraph to RelNode tree conversion.
- */
-public interface ConversionContext extends RelOptTable.ToRelContext {
-    /**
-     * @return Type factory.
-     */
-    IgniteTypeFactory getTypeFactory();
-
-    /**
-     * @return Schema.
-     */
-    RelOptSchema getSchema();
-
-    /**
-     * @return Planner context.
-     */
-    PlanningContext getContext();
-
-    /**
-     * @return Expression translator.
-     */
-    ExpToRexTranslator getExpressionTranslator();
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java
deleted file mode 100644
index 96fabfe..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/FilterNode.java
+++ /dev/null
@@ -1,66 +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.serialize.relation;
-
-import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rex.RexNode;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Describes {@link IgniteFilter}
- */
-public class FilterNode extends RelGraphNode {
-    /** */
-    private final Expression condition;
-
-    /**
-     * @param traits   Traits of this relational expression.
-     * @param condition Condition.
-     */
-    private FilterNode(RelTraitSet traits, Expression condition) {
-        super(traits);
-        this.condition = condition;
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Filter rel.
-     * @param expTranslator Expression translator.
-     * @return FilterNode.
-     */
-    public static FilterNode create(IgniteFilter rel, RexToExpTranslator expTranslator) {
-        return new FilterNode(rel.getTraitSet(), expTranslator.translate(rel.getCondition()));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        RelNode input = F.first(children);
-        RelOptCluster cluster = input.getCluster();
-        RexNode condition = this.condition.implement(ctx.getExpressionTranslator());
-
-        return new IgniteFilter(cluster, traitSet(cluster), input, condition);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java
deleted file mode 100644
index e121683..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/GraphToRelConverter.java
+++ /dev/null
@@ -1,111 +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.serialize.relation;
-
-import com.google.common.collect.ImmutableList;
-import java.util.List;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptSchema;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.rel.RelRoot;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Converts RelGraph to RelNode tree.
- */
-public class GraphToRelConverter implements ConversionContext {
-    /** */
-    private final RelOptTable.ViewExpander viewExpander;
-
-    /** */
-    private final RelBuilder relBuilder;
-
-    /** */
-    private final ExpToRexTranslator expTranslator;
-
-    /**
-     * @param viewExpander View expander.
-     * @param relBuilder Rel builder.
-     * @param opTable Operations table.
-     */
-    public GraphToRelConverter(RelOptTable.ViewExpander viewExpander, RelBuilder relBuilder, SqlOperatorTable opTable) {
-        this.viewExpander = viewExpander;
-        this.relBuilder = relBuilder;
-
-        expTranslator = new ExpToRexTranslator(relBuilder.getRexBuilder(), opTable);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteTypeFactory getTypeFactory() {
-        return (IgniteTypeFactory) getCluster().getTypeFactory();
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelOptSchema getSchema() {
-        return relBuilder.getRelOptSchema();
-    }
-
-    /** {@inheritDoc} */
-    @Override public PlanningContext getContext() {
-        return Commons.context(getCluster().getPlanner().getContext());
-    }
-
-    /** {@inheritDoc} */
-    @Override public ExpToRexTranslator getExpressionTranslator() {
-        return expTranslator;
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelOptCluster getCluster() {
-        return relBuilder.getCluster();
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
-        return viewExpander.expandView(rowType, queryString, schemaPath, viewPath);
-    }
-
-    /**
-     * Converts RelGraph to RelNode tree.
-     *
-     * @param graph RelGraph.
-     * @return RelNode tree.
-     */
-    public IgniteRel convert(RelGraph graph) {
-        return F.first(convertRecursive(this, graph, graph.nodes().subList(0, 1)));
-    }
-
-    /** */
-    private List<IgniteRel> convertRecursive(ConversionContext ctx, RelGraph graph, List<Ord<RelGraphNode>> src) {
-        ImmutableList.Builder<IgniteRel> b = ImmutableList.builder();
-
-        for (Ord<RelGraphNode> node : src)
-            b.add(node.e.toRel(ctx, convertRecursive(ctx, graph, graph.children(node.i))));
-
-        return b.build();
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java
deleted file mode 100644
index 9f6f2cb..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/JoinNode.java
+++ /dev/null
@@ -1,87 +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.serialize.relation;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
-
-/**
- * Describes {@link IgniteJoin}.
- */
-public class JoinNode extends RelGraphNode {
-    /** */
-    private final Expression condition;
-
-    /** */
-    private final int[] variables;
-
-    /** */
-    private final JoinRelType joinType;
-
-    /**
-     * @param traits   Traits of this relational expression.
-     * @param condition Condition.
-     * @param variables Variables set. See {@link IgniteJoin#getVariablesSet()}.
-     * @param joinType Join type.
-     */
-    private JoinNode(RelTraitSet traits, Expression condition, int[] variables, JoinRelType joinType) {
-        super(traits);
-        this.condition = condition;
-        this.variables = variables;
-        this.joinType = joinType;
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Join rel.
-     * @param expTranslator Expression translator.
-     * @return JoinNode.
-     */
-    public static JoinNode create(IgniteJoin rel, RexToExpTranslator expTranslator) {
-        return new JoinNode(rel.getTraitSet(),
-            expTranslator.translate(rel.getCondition()),
-            rel.getVariablesSet().stream().mapToInt(CorrelationId::getId).toArray(),
-            rel.getJoinType());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        assert children.size() == 2;
-
-        RelNode left = children.get(0);
-        RelNode right = children.get(1);
-
-        return new IgniteJoin(ctx.getCluster(),
-            traitSet(ctx.getCluster()),
-            left,
-            right,
-            ctx.getExpressionTranslator().translate(condition),
-            Arrays.stream(variables).mapToObj(CorrelationId::new).collect(Collectors.toSet()),
-            joinType);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java
deleted file mode 100644
index e7cc801..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ProjectNode.java
+++ /dev/null
@@ -1,74 +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.serialize.relation;
-
-import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rex.RexNode;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Describes {@link IgniteProject}.
- */
-public class ProjectNode extends RelGraphNode {
-    /** */
-    private final List<Expression> projects;
-
-    /** */
-    private final DataType dataType;
-
-    /**
-     * @param traits   Traits of this relational expression.
-     * @param projects Projects.
-     * @param dataType Output row type
-     */
-    private ProjectNode(RelTraitSet traits, List<Expression> projects, DataType dataType) {
-        super(traits);
-        this.projects = projects;
-        this.dataType = dataType;
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Project rel.
-     * @param rexTranslator Expression translator.
-     * @return ProjectNode.
-     */
-    public static ProjectNode create(IgniteProject rel, RexToExpTranslator rexTranslator) {
-        return new ProjectNode(rel.getTraitSet(), rexTranslator.translate(rel.getProjects()),
-            Types.fromType(rel.getRowType()));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        RelNode input = F.first(children);
-        RelOptCluster cluster = input.getCluster();
-        List<RexNode> projects = ctx.getExpressionTranslator().translate(this.projects);
-
-        return new IgniteProject(cluster, traitSet(cluster), input, projects, dataType.toRelDataType(ctx.getTypeFactory()));
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java
deleted file mode 100644
index 9eb71b1..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ReceiverNode.java
+++ /dev/null
@@ -1,68 +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.serialize.relation;
-
-import java.util.List;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.ignite.internal.processors.query.calcite.prepare.RelSource;
-import org.apache.ignite.internal.processors.query.calcite.prepare.RelSourceImpl;
-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.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
-
-
-/**
- * Describes {@link IgniteReceiver}.
- */
-public class ReceiverNode extends RelGraphNode {
-    /** */
-    private final DataType dataType;
-
-    /** */
-    private final RelSource source;
-
-    /**
-     * @param traits   Traits of this relational expression
-     * @param dataType Output row type
-     * @param source   Remote sources information.
-     */
-    private ReceiverNode(RelTraitSet traits, DataType dataType, RelSource source) {
-        super(traits);
-        this.dataType = dataType;
-        this.source = source;
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Receiver rel.
-     * @return ReceiverNode.
-     */
-    public static ReceiverNode create(IgniteReceiver rel) {
-        RelSource source = new RelSourceImpl(rel.source().fragmentId(), rel.source().mapping());
-
-        return new ReceiverNode(rel.getTraitSet(), Types.fromType(rel.getRowType()), source);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        return new IgniteReceiver(ctx.getCluster(), traitSet(ctx.getCluster()),
-            dataType.toRelDataType(ctx.getTypeFactory()), source);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java
deleted file mode 100644
index 62aa08f..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraph.java
+++ /dev/null
@@ -1,114 +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.serialize.relation;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.util.ImmutableIntList;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.GridIntList;
-
-/**
- * RelNode tree intermediate representation. Used by communication layer to send a query plan to remote node.
- */
-public class RelGraph implements Serializable {
-    /** */
-    private final List<RelGraphNode> nodes = new ArrayList<>();
-
-    /** */
-    private final List<GridIntList> edges = new ArrayList<>();
-
-    /**
-     * @return Pairs of node and its ID.
-     */
-    public List<Ord<RelGraphNode>> nodes() {
-        return Ord.zip(nodes);
-    }
-
-    /**
-     * @return Edges.
-     */
-    public List<ImmutableIntList> edges() {
-        return Commons.transform(edges, l -> ImmutableIntList.of(l.array()));
-    }
-
-    /**
-     * Adds a new node and link it as a child of node with given ID.
-     *
-     * @param parentId Parent node ID.
-     * @param node Node.
-     * @return New node ID.
-     */
-    public int addNode(int parentId, RelGraphNode node) {
-        int id = addNode(node);
-
-        addEdge(parentId, id);
-
-        return id;
-    }
-
-    /**
-     * Adds a new node.
-     *
-     * @param node Node.
-     * @return New node ID.
-     */
-    public int addNode(RelGraphNode node) {
-        assert nodes.size() == edges.size();
-
-        int id = nodes.size();
-
-        nodes.add(node);
-        edges.add(new GridIntList());
-
-        return id;
-    }
-
-    /**
-     * Adds a new edge.
-     *
-     * @param parentId Parent node ID.
-     * @param childId Child node ID.
-     */
-    public void addEdge(int parentId, int childId) {
-        assert parentId == -1 || (parentId >= 0 && parentId < edges.size());
-        assert nodes.size() == edges.size();
-
-        if (parentId != -1)
-            edges.get(parentId).add(childId);
-    }
-
-    /**
-     * Returns children of a node with given ID.
-     *
-     * @param parentId Parent node ID.
-     * @return Pairs of child node and its ID.
-     */
-    public List<Ord<RelGraphNode>> children(int parentId) {
-        GridIntList children = edges.get(parentId);
-
-        ArrayList<Ord<RelGraphNode>> ords = new ArrayList<>(children.size());
-
-        for (int i = 0; i < children.size(); i++)
-            ords.add(Ord.of(children.get(i), nodes.get(children.get(i))));
-
-        return ords;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java
deleted file mode 100644
index d508500..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelGraphNode.java
+++ /dev/null
@@ -1,91 +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.serialize.relation;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-
-/**
- * A superclass of all relational nodes representations.
- */
-public abstract class RelGraphNode implements Serializable {
-    /** */
-    private final List<Serializable> traits;
-
-    /**
-     * @param traits Traits of this relational expression.
-     */
-    protected RelGraphNode(RelTraitSet traits) {
-        assert traits.contains(IgniteConvention.INSTANCE);
-
-        List<Serializable> list = new ArrayList<>(traits.size() - 1);
-
-        for (RelTrait trait : traits) {
-            if (trait != IgniteConvention.INSTANCE)
-                list.add(toSerializable(trait));
-        }
-
-        this.traits = list;
-    }
-
-    /**
-     * Perform back conversion of serializable traits representation to trait set.
-     *
-     * @param cluster Cluster.
-     * @return Trait set.
-     */
-    protected RelTraitSet traitSet(RelOptCluster cluster) {
-        RelTraitSet traits = cluster.traitSetOf(IgniteConvention.INSTANCE);
-
-        for (Serializable trait : this.traits)
-            traits = traits.replace(fromSerializable(trait));
-
-        return traits.simplify();
-    }
-
-    /** Converts a trait to its serializable representation. */
-    private Serializable toSerializable(RelTrait trait) {
-        if (trait instanceof Serializable)
-            return (Serializable) trait;
-
-        throw new AssertionError();
-    }
-
-    /** Converts a serializable representation of a trait to a trait itself. */
-    private RelTrait fromSerializable(Serializable trait) {
-        if (trait instanceof RelTrait)
-            return (RelTrait) trait;
-
-        throw new AssertionError();
-    }
-
-    /**
-     * Converts representation to particular IgniteRel.
-     *
-     * @param ctx Conversion context.
-     * @param children Input rels.
-     * @return RelNode.
-     */
-    public abstract IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children);
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java
deleted file mode 100644
index 34fe967..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/RelToGraphConverter.java
+++ /dev/null
@@ -1,142 +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.serialize.relation;
-
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.List;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteJoin;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
-import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Converts RelNode tree to RelGraph.
- */
-public class RelToGraphConverter implements RelOp<IgniteRel, RelGraph> {
-    /** */
-    private final RexToExpTranslator rexTranslator = new RexToExpTranslator();
-
-    /** */
-    private RelGraph graph;
-
-    /** Parent ID of currently processed node. */
-    private int curParent;
-
-    /** */
-    private static final class Item {
-        /** */
-        private final int parentId;
-
-        /** */
-        private final List<IgniteRel> children;
-
-        /** */
-        private Item(int parentId, List<IgniteRel> children) {
-            this.parentId = parentId;
-            this.children = children;
-        }
-    }
-
-    /**
-     * Adds a representation of input node to the graph and returns node id in the graph and its children for future processing.
-     */
-    private final class ItemTranslator implements IgniteRelVisitor<Item> {
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteFilter rel) {
-            return new Item(graph.addNode(curParent, FilterNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteJoin rel) {
-            return new Item(graph.addNode(curParent, JoinNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteProject rel) {
-            return new Item(graph.addNode(curParent, ProjectNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteTableScan rel) {
-            return new Item(graph.addNode(curParent, TableScanNode.create(rel)), Commons.cast(rel.getInputs()));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteReceiver rel) {
-            return new Item(graph.addNode(curParent, ReceiverNode.create(rel)), Collections.emptyList());
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteSender rel) {
-            return new Item(graph.addNode(curParent, SenderNode.create(rel)), Commons.cast(rel.getInputs()));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteTableModify rel) {
-            return new Item(graph.addNode(curParent, TableModifyNode.create(rel, rexTranslator)), Commons.cast(rel.getInputs()));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteValues rel) {
-            return new Item(graph.addNode(curParent, ValuesNode.create(rel, rexTranslator)), Collections.emptyList());
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteRel rel) {
-            return rel.accept(this);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Item visit(IgniteExchange rel) {
-            throw new AssertionError("Unexpected node: " + rel);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelGraph go(IgniteRel root) {
-        graph = new RelGraph();
-
-        ItemTranslator itemTranslator = new ItemTranslator();
-        Deque<Item> stack = new ArrayDeque<>();
-        stack.push(new Item(-1, F.asList(root)));
-
-        while (!stack.isEmpty()) {
-            Item item = stack.pop();
-
-            curParent = item.parentId;
-
-            for (IgniteRel child : item.children)
-                stack.push(itemTranslator.visit(child));
-        }
-
-        return graph;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java
deleted file mode 100644
index e8c161f..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/SenderNode.java
+++ /dev/null
@@ -1,63 +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.serialize.relation;
-
-import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.ignite.internal.processors.query.calcite.prepare.RelTarget;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Describes {@link IgniteSender}.
- */
-public class SenderNode extends RelGraphNode {
-    /** */
-    private final RelTarget target;
-
-    /**
-     *
-     * @param traits   Traits of this relational expression
-     * @param target   Remote targets information
-     */
-    private SenderNode(RelTraitSet traits, RelTarget target) {
-        super(traits);
-        this.target = target;
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Sender rel.
-     * @return SenderNode.
-     */
-    public static SenderNode create(IgniteSender rel) {
-        return new SenderNode(rel.getTraitSet(), rel.target());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        RelNode input = F.first(children);
-        RelOptCluster cluster = input.getCluster();
-
-        return new IgniteSender(cluster, traitSet(cluster), input, target);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableModifyNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableModifyNode.java
deleted file mode 100644
index 99eb2a8..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/TableModifyNode.java
+++ /dev/null
@@ -1,94 +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.serialize.relation;
-
-import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- *
- */
-public class TableModifyNode extends RelGraphNode {
-    /** */
-    private final List<String> tableName;
-
-    /** */
-    private final TableModify.Operation operation;
-
-    /** */
-    private final List<String> updateColumnList;
-
-    /** */
-    private final List<Expression> sourceExpressionList;
-
-    /** */
-    private final boolean flattened;
-
-    /**
-     * @param traits Traits of this relational expression.
-     * @param tableName Table name.
-     * @param operation Operation.
-     * @param updateColumnList Update column list.
-     * @param sourceExpressionList Source expression list.
-     * @param flattened Flattened flag.
-     */
-    private TableModifyNode(RelTraitSet traits, List<String> tableName, TableModify.Operation operation, List<String> updateColumnList, List<Expression> sourceExpressionList, boolean flattened) {
-        super(traits);
-
-        this.tableName = tableName;
-        this.operation = operation;
-        this.updateColumnList = updateColumnList;
-        this.sourceExpressionList = sourceExpressionList;
-        this.flattened = flattened;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        RelOptCluster cluster = ctx.getCluster();
-        RelNode input = F.first(children);
-        RelOptTable table = ctx.getSchema().getTableForMember(tableName);
-        RelTraitSet traits = traitSet(cluster);
-        CalciteCatalogReader catalogReader = (CalciteCatalogReader) ctx.getSchema();
-        List<RexNode> sourceExpressionList = ctx.getExpressionTranslator().translate(this.sourceExpressionList);
-
-        return new IgniteTableModify(cluster, traits, table, catalogReader, input, operation, updateColumnList, sourceExpressionList, flattened);
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Table modify rel.
-     * @param rexTranslator Expression translator.
-     * @return ProjectNode.
-     */
-    public static TableModifyNode create(IgniteTableModify rel, RexToExpTranslator rexTranslator) {
-        return new TableModifyNode(rel.getTraitSet(), rel.getTable().getQualifiedName(), rel.getOperation(), rel.getUpdateColumnList(),
-            rexTranslator.translate(rel.getSourceExpressionList()), rel.isFlattened());
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ValuesNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ValuesNode.java
deleted file mode 100644
index ce56f8c..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/relation/ValuesNode.java
+++ /dev/null
@@ -1,93 +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.serialize.relation;
-
-import com.google.common.collect.ImmutableList;
-import java.util.List;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteValues;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.ExpToRexTranslator;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.Expression;
-import org.apache.ignite.internal.processors.query.calcite.serialize.expression.RexToExpTranslator;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.DataType;
-import org.apache.ignite.internal.processors.query.calcite.serialize.type.Types;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-
-/**
- *
- */
-public class ValuesNode extends RelGraphNode {
-    /** */
-    private final DataType dataType;
-
-    /** */
-    private final List<List<Expression>> tuples;
-
-
-    /**
-     * @param traits Traits of this relational expression.
-     */
-    private ValuesNode(RelTraitSet traits, DataType dataType, List<List<Expression>> tuples) {
-        super(traits);
-        this.dataType = dataType;
-        this.tuples = tuples;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteRel toRel(ConversionContext ctx, List<IgniteRel> children) {
-        RelOptCluster cluster = ctx.getCluster();
-        RelTraitSet traits = traitSet(cluster);
-        RelDataType rowType = dataType.toRelDataType((IgniteTypeFactory) ctx.getTypeFactory());
-        ImmutableList<ImmutableList<RexLiteral>> tuples = translate(this.tuples, ctx.getExpressionTranslator());
-
-        return new IgniteValues(cluster, rowType, tuples, traits);
-    }
-
-    /**
-     * Factory method.
-     *
-     * @param rel Values rel.
-     * @param rexTranslator Expression translator.
-     * @return ProjectNode.
-     */
-    public static ValuesNode create(IgniteValues rel, RexToExpTranslator rexTranslator) {
-        return new ValuesNode(rel.getTraitSet(), Types.fromType(rel.getRowType()),
-            Commons.transform(rel.getTuples(), tuple -> rexTranslator.translate(Commons.cast(tuple))));
-    }
-
-    /** */
-    private static ImmutableList<ImmutableList<RexLiteral>> translate(List<List<Expression>> tuples, ExpToRexTranslator translator) {
-        ImmutableList.Builder<ImmutableList<RexLiteral>> b = ImmutableList.builder();
-
-        for (List<Expression> tuple : tuples) {
-            ImmutableList.Builder<RexLiteral> b1 = ImmutableList.builder();
-
-            for (Expression exp : tuple)
-                b1.add((RexLiteral) translator.translate(exp));
-
-            b.add(b1.build());
-        }
-
-        return b.build();
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/Types.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/Types.java
deleted file mode 100644
index 32d07d5..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/type/Types.java
+++ /dev/null
@@ -1,187 +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.serialize.type;
-
-import java.nio.charset.Charset;
-import java.util.LinkedHashMap;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.type.SqlTypeUtil;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.type.SystemType;
-
-/**
- *
- */
-public final class Types {
-    /** */
-    private Types() {
-        // No-op.
-    }
-
-    /**
-     * Factory method to construct data type representation from RelDataType.
-     * @param type RelDataType.
-     * @return DataType.
-     */
-    public static DataType fromType(RelDataType type) {
-        if (type.isStruct()) {
-            assert type.isStruct();
-
-            LinkedHashMap<String, DataType> fields = new LinkedHashMap<>();
-
-            for (RelDataTypeField field : type.getFieldList())
-                fields.put(field.getName(), fromType(field.getType()));
-
-            return new StructType(fields);
-        }
-
-        if (type instanceof RelDataTypeFactoryImpl.JavaType)
-            return new JavaType(((RelDataTypeFactoryImpl.JavaType) type).getJavaClass(), type instanceof SystemType);
-
-        assert type instanceof org.apache.calcite.sql.type.BasicSqlType : type;
-
-        if (SqlTypeUtil.inCharFamily(type))
-            return new CharacterSqlType(type.getSqlTypeName(),
-                type.getCharset(),
-                type.getCollation(),
-                type.getPrecision(),
-                type.getScale(),
-                type.isNullable()
-            );
-
-        return new BasicSqlType(type.getSqlTypeName(), type.getPrecision(), type.getScale(), type.isNullable());
-    }
-
-    /** */
-    private static class JavaType implements DataType {
-        /** */
-        private final Class<?> clazz;
-
-        /** */
-        private final boolean system;
-
-        /**
-         * @param clazz Value class.
-         * @param system System type flag.
-         */
-        private JavaType(Class<?> clazz, boolean system) {
-            this.clazz = clazz;
-            this.system = system;
-        }
-
-        /** {@inheritDoc} */
-        @Override public RelDataType toRelDataType(IgniteTypeFactory factory) {
-            return system ? factory.createSystemType(clazz) : factory.createJavaType(clazz);
-        }
-    }
-
-    /** */
-    private static class BasicSqlType implements DataType {
-        /** */
-        private final SqlTypeName typeName;
-
-        /** */
-        private final int precision;
-
-        /** */
-        private final int scale;
-
-        /** */
-        private final boolean nullable;
-
-        /**
-         * @param typeName Type name
-         * @param precision Precision.
-         * @param scale Scale.
-         * @param nullable Nullable flag.
-         */
-        private BasicSqlType(SqlTypeName typeName, int precision, int scale, boolean nullable) {
-            this.typeName = typeName;
-            this.precision = precision;
-            this.scale = scale;
-            this.nullable = nullable;
-        }
-
-        /** {@inheritDoc} */
-        @Override public RelDataType toRelDataType(IgniteTypeFactory factory) {
-            RelDataType type;
-
-            if (!typeName.allowsPrec())
-                type = factory.createSqlType(typeName);
-            else if (!typeName.allowsScale())
-                type = factory.createSqlType(typeName, precision);
-            else
-                type = factory.createSqlType(typeName, precision, scale);
-
-            return nullable ? factory.createTypeWithNullability(type, nullable) : type;
-        }
-    }
-
-    /** */
-    private static class CharacterSqlType extends BasicSqlType {
-        /** */
-        private final Charset charset;
-
-        /** */
-        private final SqlCollation collation;
-
-        /**
-         * @param typeName  Type name
-         * @param charset Charset.
-         * @param collation Collation.
-         * @param precision Precision.
-         * @param scale     Scale.
-         * @param nullable Nullable flag.
-         */
-        private CharacterSqlType(SqlTypeName typeName, Charset charset, SqlCollation collation, int precision, int scale, boolean nullable) {
-            super(typeName, precision, scale, nullable);
-            this.charset = charset;
-            this.collation = collation;
-        }
-
-        /** {@inheritDoc} */
-        @Override public RelDataType toRelDataType(IgniteTypeFactory factory) {
-            return factory.createTypeWithCharsetAndCollation(super.toRelDataType(factory), charset, collation);
-        }
-    }
-
-    /** */
-    private static class StructType implements DataType {
-        /** */
-        private final LinkedHashMap<String, DataType> fields;
-
-        /**
-         * @param fields Fields.
-         */
-        private StructType(LinkedHashMap<String, DataType> fields) {
-            this.fields = fields;
-        }
-
-        /** {@inheritDoc} */
-        @Override public RelDataType toRelDataType(IgniteTypeFactory factory) {
-            RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(factory);
-            fields.forEach((n,f) -> builder.add(n,f.toRelDataType(factory)));
-            return builder.build();
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DistributionTrait.java
index cf6c0f9..d85dd91 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
@@ -18,11 +18,6 @@
 package org.apache.ignite.internal.processors.query.calcite.trait;
 
 import com.google.common.collect.Ordering;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.ObjectStreamException;
-import java.io.Serializable;
 import java.util.Objects;
 import org.apache.calcite.plan.RelMultipleTrait;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -40,7 +35,7 @@ import static org.apache.calcite.rel.RelDistribution.Type.SINGLETON;
 /**
  * Description of the physical distribution of a relational expression.
  */
-public final class DistributionTrait implements IgniteDistribution, Serializable {
+public final class DistributionTrait implements IgniteDistribution {
     /** */
     private static final Ordering<Iterable<Integer>> ORDERING =
         Ordering.<Integer>natural().lexicographical();
@@ -142,6 +137,7 @@ public final class DistributionTrait implements IgniteDistribution, Serializable
         return other.getType() == SINGLETON && getType() == BROADCAST_DISTRIBUTED;
     }
 
+    /** {@inheritDoc} */
     @Override public IgniteDistribution apply(Mappings.TargetMapping mapping) {
         if (keys.isEmpty())
             return this;
@@ -153,10 +149,12 @@ public final class DistributionTrait implements IgniteDistribution, Serializable
         return newKeys.isEmpty() ? IgniteDistributions.random() : IgniteDistributions.hash(newKeys, function);
     }
 
+    /** {@inheritDoc} */
     @Override public boolean isTop() {
         return getType() == Type.ANY;
     }
 
+    /** {@inheritDoc} */
     @Override public int compareTo(RelMultipleTrait o) {
         // TODO is this method really needed??
 
@@ -173,21 +171,4 @@ public final class DistributionTrait implements IgniteDistribution, Serializable
 
         return getType().compareTo(distribution.getType());
     }
-
-    /** */
-    private void writeObject(ObjectOutputStream out) throws IOException {
-        out.writeObject(keys.toIntArray());
-        out.writeObject(function);
-    }
-
-    /** */
-    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-        keys = ImmutableIntList.of((int[])in.readObject());
-        function = (DistributionFunction) in.readObject();
-    }
-
-    /** */
-    private Object readResolve() throws ObjectStreamException {
-        return IgniteDistributions.canonize(this);
-    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
index 507bd59..d2f2110 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
@@ -17,13 +17,8 @@
 
 package org.apache.ignite.internal.processors.query.calcite.type;
 
-import java.nio.charset.Charset;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.linq4j.tree.Primitive;
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.type.SqlTypeUtil;
 
 /**
  * Ignite type factory.
@@ -40,74 +35,4 @@ public class IgniteTypeFactory extends JavaTypeFactoryImpl {
     public IgniteTypeFactory(RelDataTypeSystem typeSystem) {
         super(typeSystem);
     }
-
-    /**
-     * Creates a system type.
-     */
-    public RelDataType createSystemType(Class<?> clazz) {
-        final SystemJavaType javaType = clazz != String.class ? new SystemJavaType(clazz)
-            : new SystemJavaType(clazz, true, getDefaultCharset(), SqlCollation.IMPLICIT);
-
-        return canonize(javaType);
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelDataType createTypeWithCharsetAndCollation(RelDataType type, Charset charset, SqlCollation collation) {
-        if (type instanceof SystemJavaType)
-            return canonize(((SystemJavaType) type).copyWithCharsetAndCollation(charset, collation));
-
-        return super.createTypeWithCharsetAndCollation(type, charset, collation);
-    }
-
-    /** {@inheritDoc} */
-    @Override public RelDataType createTypeWithNullability(RelDataType type, boolean nullable) {
-        if (type instanceof SystemJavaType)
-            return canonize(((SystemJavaType) type).copyWithNullability(nullable));
-
-        return super.createTypeWithNullability(type, nullable);
-    }
-
-    /** */
-    public class SystemJavaType extends JavaType implements SystemType {
-        /**
-         * @param clazz Java type.
-         */
-        public SystemJavaType(Class<?> clazz) {
-            super(clazz);
-        }
-
-        /**
-         * @param clazz Java type.
-         * @param nullable Nullability.
-         */
-        public SystemJavaType(Class<?> clazz, boolean nullable) {
-            super(clazz, nullable);
-        }
-
-        /**
-         * @param clazz Java type.
-         * @param nullable Nullability.
-         * @param charset Charset.
-         * @param collation Collation.
-         */
-        public SystemJavaType(Class<?> clazz, boolean nullable, Charset charset, SqlCollation collation) {
-            super(clazz, nullable, charset, collation);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void generateTypeString(StringBuilder sb, boolean withDetail) {
-            sb.append("SystemJavaType(").append(getJavaClass()).append(")");
-        }
-
-        /** */
-        private SystemJavaType copyWithCharsetAndCollation(Charset charset, SqlCollation collation) {
-            return new SystemJavaType(getJavaClass(), isNullable(), charset, collation);
-        }
-
-        /** */
-        private SystemJavaType copyWithNullability(boolean nullable) {
-            return SqlTypeUtil.inCharFamily(this) ? new SystemJavaType(getJavaClass(), nullable, getCharset(), getCollation())
-                : new SystemJavaType(nullable ? Primitive.box(getJavaClass()) : Primitive.unbox(getJavaClass()), nullable);
-        }
-    }
 }
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 4772fe3..c6b07e5 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
@@ -25,14 +25,19 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.query.QueryContext;
 import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
@@ -193,4 +198,26 @@ public final class Commons {
         else
             U.closeQuiet((AutoCloseable) o);
     }
+
+    /** */
+    public static RelDataType combinedRowType(IgniteTypeFactory typeFactory, RelDataType... types) {
+        final RelDataTypeFactory.Builder builder = new RelDataTypeFactory.Builder(typeFactory);
+
+        for (RelDataType type : types)
+            builder.addAll(type.getFieldList());
+
+        return builder.build();
+    }
+
+    /** */
+    public static Class<?> boxType(Class<?> type) {
+        Primitive primitive = Primitive.of(type);
+
+        return primitive == null ? type : primitive.boxClass;
+    }
+
+    /** */
+    public static <T> List<T> flat(List<List<? extends T>> src) {
+        return src.stream().flatMap(List::stream).collect(Collectors.toList());
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java
index c71f6a0..ff8f2bd 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteMethod.java
@@ -19,6 +19,9 @@ package org.apache.ignite.internal.processors.query.calcite.util;
 
 import java.lang.reflect.Method;
 import org.apache.calcite.linq4j.tree.Types;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.CallOperation;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Scalar;
 import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.DerivedDistribution;
 import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata.FragmentMetadata;
 
@@ -26,6 +29,8 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetada
  * Contains methods used in metadata definitions.
  */
 public enum IgniteMethod {
+    CALL_APPLY(CallOperation.class, "apply", Object[].class),
+    SCALAR_EXECUTE(Scalar.class, "execute", ExecutionContext.class, Object[].class, Object[].class),
     DERIVED_DISTRIBUTIONS(DerivedDistribution.class, "deriveDistributions"),
     FRAGMENT_INFO(FragmentMetadata.class, "fragmentInfo");
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteResource.java
similarity index 55%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteResource.java
index cebb0d2..e8a5ade 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/IgniteResource.java
@@ -15,25 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exec;
+package org.apache.ignite.internal.processors.query.calcite.util;
 
-import java.util.Objects;
+import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.validate.SqlValidatorException;
 
 /**
- * A node with a single input
+ *
  */
-public interface SingleNode<T> extends Node<T> {
-    /**
-     * @return Single sink object.
-     */
-    default Sink<T> sink() {
-        return Objects.requireNonNull(sink(0));
-    }
+public interface IgniteResource {
+    /** */
+    IgniteResource INSTANCE = Resources.create(IgniteResource.class);
+
+    /** */
+    @Resources.BaseMessage("Illegal alias. {0} is reserved name.")
+    Resources.ExInst<SqlValidatorException> illegalAlias(String a0);
 
-    /**
-     * @return Single input.
-     */
-    default Node<T> input() {
-        return Objects.requireNonNull(input(0));
-    }
+    /** */
+    @Resources.BaseMessage("Cannot update field \"{0}\". You cannot update key, key fields or val field in case the val is a complex type.")
+    Resources.ExInst<SqlValidatorException> cannotUpdateField(String field);
 }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
index 1789eeb..c54eec9 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.query.QueryEngine;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.After;
 import org.junit.Before;
@@ -38,6 +39,7 @@ import org.junit.Test;
 /**
  *
  */
+@WithSystemProperty(key = "calcite.debug", value = "true")
 public class CalciteQueryProcessorTest extends GridCommonAbstractTest {
     /** */
     private IgniteEx ignite;
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/PlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/PlannerTest.java
index a356493..62ce15a 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/PlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/PlannerTest.java
@@ -49,13 +49,14 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
-import org.apache.ignite.internal.processors.query.calcite.exec.ConsumerNode;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeServiceImpl;
 import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
-import org.apache.ignite.internal.processors.query.calcite.exec.Implementor;
+import org.apache.ignite.internal.processors.query.calcite.exec.LogicalRelImplementor;
 import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistryImpl;
-import org.apache.ignite.internal.processors.query.calcite.exec.Node;
 import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutorImpl;
+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.CalciteMessage;
 import org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl;
 import org.apache.ignite.internal.processors.query.calcite.message.TestIoManager;
@@ -73,8 +74,6 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.schema.DistributedTable;
 import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
 import org.apache.ignite.internal.processors.query.calcite.schema.SortedTable;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelGraph;
-import org.apache.ignite.internal.processors.query.calcite.serialize.relation.RelToGraphConverter;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
@@ -82,7 +81,6 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactor
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -591,7 +589,7 @@ public class PlannerTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     @Test
-    public void testPlanSerializationDeserialization() throws Exception {
+    public void testSplitterCollocatedPartitionedPartitioned() throws Exception {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
         TestTable developer = new TestTable(
@@ -670,7 +668,9 @@ public class PlannerTest extends GridCommonAbstractTest {
             .topologyVersion(AffinityTopologyVersion.NONE)
             .build();
 
-        byte[] convertedBytes;
+        assertNotNull(ctx);
+
+        RelRoot relRoot;
 
         try (IgnitePlanner planner = ctx.planner()){
             assertNotNull(planner);
@@ -686,7 +686,9 @@ public class PlannerTest extends GridCommonAbstractTest {
             sqlNode = planner.validate(sqlNode);
 
             // Convert to Relational operators graph
-            RelNode rel = planner.convert(sqlNode);
+            relRoot = planner.rel(sqlNode);
+
+            RelNode rel = relRoot.rel;
 
             // Transformation chain
             rel = planner.transform(PlannerPhase.HEURISTIC_OPTIMIZATION, rel.getTraitSet(), rel);
@@ -698,41 +700,29 @@ public class PlannerTest extends GridCommonAbstractTest {
 
             rel = planner.transform(PlannerPhase.OPTIMIZATION, desired, rel);
 
-            assertNotNull(rel);
-
-            MultiStepPlan plan = new MultiStepQueryPlan(new Splitter().go((IgniteRel) rel));
-
-            assertNotNull(plan);
-
-            assertTrue(plan.fragments().size() == 2);
-
-            plan.init(this::intermediateMapping, ctx);
-
-            RelGraph graph = new RelToGraphConverter().go(plan.fragments().get(1).root());
-
-            convertedBytes = new JdkMarshaller().marshal(graph);
-
-            assertNotNull(convertedBytes);
+            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
         }
 
-        try (IgnitePlanner planner = ctx.planner()) {
-            assertNotNull(planner);
+        assertNotNull(relRoot);
+
+        MultiStepPlan plan = new MultiStepQueryPlan(new Splitter().go((IgniteRel) relRoot.rel));
 
-            RelGraph graph = new JdkMarshaller().unmarshal(convertedBytes, getClass().getClassLoader());
+        assertNotNull(plan);
 
-            assertNotNull(graph);
+        plan.init(this::intermediateMapping, ctx);
 
-            RelNode rel = planner.convert(graph);
+        assertNotNull(plan);
 
-            assertNotNull(rel);
-        }
+        assertEquals(2, plan.fragments().size());
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testSplitterCollocatedPartitionedPartitioned() throws Exception {
+    public void testPhysicalPlan() throws Exception {
+        executors = new ArrayList<>();
+
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
         TestTable developer = new TestTable(
@@ -741,18 +731,19 @@ public class PlannerTest extends GridCommonAbstractTest {
                 .add("NAME", f.createJavaType(String.class))
                 .add("PROJECTID", f.createJavaType(Integer.class))
                 .build()) {
+            @Override public Enumerable<Object[]> scan(DataContext root) {
+                return Linq4j.asEnumerable(Arrays.asList(
+                    new Object[]{0, "Igor", 0},
+                    new Object[]{1, "Roman", 0}
+                ));
+            }
+
             @Override public NodesMapping mapping(PlanningContext ctx) {
-                return new NodesMapping(null, Arrays.asList(
-                    select(nodes, 0, 1),
-                    select(nodes, 1, 2),
-                    select(nodes, 2, 0),
-                    select(nodes, 0, 1),
-                    select(nodes, 1, 2)
-                ), NodesMapping.HAS_PARTITIONED_CACHES);
+                return new NodesMapping(select(nodes, 1), null, (byte) (HAS_REPLICATED_CACHES | PARTIALLY_REPLICATED));
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Developer", "hash");
+                return IgniteDistributions.broadcast();
             }
         };
 
@@ -762,18 +753,19 @@ public class PlannerTest extends GridCommonAbstractTest {
                 .add("NAME", f.createJavaType(String.class))
                 .add("VER", f.createJavaType(Integer.class))
                 .build()) {
+            @Override public Enumerable<Object[]> scan(DataContext root) {
+                return Linq4j.asEnumerable(Arrays.asList(
+                    new Object[]{0, "Calcite", 1},
+                    new Object[]{1, "Ignite", 1}
+                ));
+            }
+
             @Override public NodesMapping mapping(PlanningContext ctx) {
-                return new NodesMapping(null, Arrays.asList(
-                    select(nodes, 0, 1),
-                    select(nodes, 1, 2),
-                    select(nodes, 2, 0),
-                    select(nodes, 0, 1),
-                    select(nodes, 1, 2)
-                ), NodesMapping.HAS_PARTITIONED_CACHES);
+                return new NodesMapping(select(nodes, 1), null, (byte) (HAS_REPLICATED_CACHES | PARTIALLY_REPLICATED));
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Project", "hash");
+                return IgniteDistributions.broadcast();
             }
         };
 
@@ -785,11 +777,11 @@ public class PlannerTest extends GridCommonAbstractTest {
         SchemaPlus schema = createRootSchema(false)
             .add("PUBLIC", publicSchema);
 
-        String sql = "SELECT d.id, d.name, d.projectId, p.id0, p.ver0 " +
+        String sql = "SELECT d.id, d.name, d.projectId, p.name0, p.ver0 " +
             "FROM PUBLIC.Developer d JOIN (" +
-            "SELECT pp.id as id0, pp.ver as ver0 FROM PUBLIC.Project pp" +
+            "SELECT pp.id as id0, pp.name as name0, pp.ver as ver0 FROM PUBLIC.Project pp" +
             ") p " +
-            "ON d.id = p.id0 " +
+            "ON d.projectId = p.id0 " +
             "WHERE (d.projectId + 1) > ?";
 
         RelTraitDef<?>[] traitDefs = {
@@ -807,14 +799,10 @@ public class PlannerTest extends GridCommonAbstractTest {
                 .build())
             .logger(log)
             .query(sql)
-            .parameters(new Object[]{2})
+            .parameters(new Object[]{-10})
             .topologyVersion(AffinityTopologyVersion.NONE)
             .build();
 
-        assertNotNull(ctx);
-
-        RelRoot relRoot;
-
         try (IgnitePlanner planner = ctx.planner()){
             assertNotNull(planner);
 
@@ -829,77 +817,175 @@ public class PlannerTest extends GridCommonAbstractTest {
             sqlNode = planner.validate(sqlNode);
 
             // Convert to Relational operators graph
-            relRoot = planner.rel(sqlNode);
+            RelRoot relRoot = planner.rel(sqlNode);
 
             RelNode rel = relRoot.rel;
 
             // Transformation chain
             rel = planner.transform(PlannerPhase.HEURISTIC_OPTIMIZATION, rel.getTraitSet(), rel);
 
-            RelTraitSet desired = rel.getCluster().traitSet()
-                .replace(IgniteConvention.INSTANCE)
-                .replace(IgniteDistributions.single())
-                .simplify();
+            RelTraitSet desired = rel.getCluster()
+                .traitSetOf(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single());
 
-            rel = planner.transform(PlannerPhase.OPTIMIZATION, desired, rel);
+            RelNode phys = planner.transform(PlannerPhase.OPTIMIZATION, desired, rel);
 
-            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
-        }
+            assertNotNull(phys);
 
-        assertNotNull(relRoot);
+            MultiStepPlan plan = new MultiStepQueryPlan(new Splitter().go((IgniteRel) phys));
 
-        MultiStepPlan plan = new MultiStepQueryPlan(new Splitter().go((IgniteRel) relRoot.rel));
+            assertNotNull(plan);
 
-        assertNotNull(plan);
+            plan.init(this::intermediateMapping, ctx);
 
-        plan.init(this::intermediateMapping, ctx);
+            List<Fragment> fragments = plan.fragments();
+            assertEquals(2, fragments.size());
 
-        assertNotNull(plan);
+            UUID queryId = UUID.randomUUID();
 
-        assertEquals(2, plan.fragments().size());
+            TestIoManager mgr = new TestIoManager();
+            GridTestKernalContext kernal;
+            QueryTaskExecutorImpl taskExecutor;
+            MessageServiceImpl messageService;
+            MailboxRegistryImpl mailboxRegistry;
+            ExchangeServiceImpl exchangeService;
+            ExecutionContext ectx;
+            Node<Object[]> exec;
+
+            //// Local part
+
+            Fragment fragment = fragments.get(0);
+            assert fragment.local();
+
+            kernal = newContext();
+
+            taskExecutor = new QueryTaskExecutorImpl(kernal);
+            taskExecutor.stripedThreadPoolExecutor(new IgniteStripedThreadPoolExecutor(
+                kernal.config().getQueryThreadPoolSize(),
+                kernal.igniteInstanceName(),
+                "calciteQry",
+                (t,ex) -> {
+                    log().error(ex.getMessage(), ex);
+                    lastException = ex;
+                },
+                true,
+                DFLT_THREAD_KEEP_ALIVE_TIME
+            ));
+            executors.add(taskExecutor);
+
+            messageService = new TestMessageServiceImpl(kernal, mgr);
+
+            messageService.localNodeId(nodes.get(0));
+            messageService.taskExecutor(taskExecutor);
+            mgr.register(messageService);
+
+            mailboxRegistry = new MailboxRegistryImpl(kernal);
+
+            exchangeService = new ExchangeServiceImpl(kernal);
+            exchangeService.taskExecutor(taskExecutor);
+            exchangeService.messageService(messageService);
+            exchangeService.mailboxRegistry(mailboxRegistry);
+            exchangeService.init();
+
+            ectx = new ExecutionContext(taskExecutor, ctx, queryId, fragment.fragmentId(), null, Commons.parametersMap(ctx.parameters()));
+
+            exec = new LogicalRelImplementor(ectx, c1 -> r1 -> 0, mailboxRegistry, exchangeService,
+                new TestFailureProcessor(kernal)).go(fragment.root());
+
+            RootNode consumer = new RootNode(ectx, r -> {});
+            consumer.register(exec);
+
+            //// Remote part
+
+            fragment = fragments.get(1);
+
+            assert !fragment.local();
+
+            kernal = newContext();
+
+            taskExecutor = new QueryTaskExecutorImpl(kernal);
+            taskExecutor.stripedThreadPoolExecutor(new IgniteStripedThreadPoolExecutor(
+                kernal.config().getQueryThreadPoolSize(),
+                kernal.igniteInstanceName(),
+                "calciteQry",
+                (t,ex) -> {
+                    log().error(ex.getMessage(), ex);
+                    lastException = ex;
+                },
+                true,
+                DFLT_THREAD_KEEP_ALIVE_TIME
+            ));
+            executors.add(taskExecutor);
+
+            messageService = new TestMessageServiceImpl(kernal, mgr);
+            messageService.localNodeId(nodes.get(1));
+            messageService.taskExecutor(taskExecutor);
+            mgr.register(messageService);
+
+            mailboxRegistry = new MailboxRegistryImpl(kernal);
+
+            exchangeService = new ExchangeServiceImpl(kernal);
+            exchangeService.taskExecutor(taskExecutor);
+            exchangeService.messageService(messageService);
+            exchangeService.mailboxRegistry(mailboxRegistry);
+            exchangeService.init();
+
+            ectx = new ExecutionContext(
+                taskExecutor,
+                PlanningContext.builder()
+                    .localNodeId(nodes.get(1))
+                    .originatingNodeId(nodes.get(0))
+                    .parentContext(Contexts.empty())
+                    .frameworkConfig(newConfigBuilder(FRAMEWORK_CONFIG)
+                        .defaultSchema(schema)
+                        .traitDefs(traitDefs)
+                        .build())
+                    .logger(log)
+                    .build(),
+                queryId,
+                fragment.fragmentId(),
+                null,
+                Commons.parametersMap(ctx.parameters()));
+
+            exec = new LogicalRelImplementor(ectx, c -> r -> 0, mailboxRegistry, exchangeService,
+                new TestFailureProcessor(kernal)).go(fragment.root());
+
+            //// Start execution
+
+            assert exec instanceof Outbox;
+
+            exec.context().execute(((Outbox<Object[]>) exec)::init);
+
+            ArrayList<Object[]> res = new ArrayList<>();
+
+            while (consumer.hasNext())
+                res.add(consumer.next());
+
+            assertFalse(res.isEmpty());
+
+            Assert.assertArrayEquals(new Object[]{0, "Igor", 0, "Calcite", 1}, res.get(0));
+            Assert.assertArrayEquals(new Object[]{1, "Roman", 0, "Calcite", 1}, res.get(1));
+        }
     }
 
     /**
      * @throws Exception If failed.
      */
     @Test
-    public void testPhysicalPlan() throws Exception {
+    public void testPhysicalPlan2() throws Exception {
         executors = new ArrayList<>();
 
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        TestTable developer = new TestTable(
-            new RelDataTypeFactory.Builder(f)
-                .add("ID", f.createJavaType(Integer.class))
-                .add("NAME", f.createJavaType(String.class))
-                .add("PROJECTID", f.createJavaType(Integer.class))
-                .build()) {
-            @Override public Enumerable<Object[]> scan(DataContext root) {
-                return Linq4j.asEnumerable(Arrays.asList(
-                    new Object[]{0, "Igor", 0},
-                    new Object[]{1, "Roman", 0}
-                ));
-            }
-
-            @Override public NodesMapping mapping(PlanningContext ctx) {
-                return new NodesMapping(select(nodes, 1), null, (byte) (HAS_REPLICATED_CACHES | PARTIALLY_REPLICATED));
-            }
-
-            @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.broadcast();
-            }
-        };
-
-        TestTable project = new TestTable(
+        TestTable testTable = new TestTable(
             new RelDataTypeFactory.Builder(f)
-                .add("ID", f.createJavaType(Integer.class))
-                .add("NAME", f.createJavaType(String.class))
-                .add("VER", f.createJavaType(Integer.class))
+                .add("ID0", f.createJavaType(Integer.class))
+                .add("ID1", f.createJavaType(Integer.class))
                 .build()) {
             @Override public Enumerable<Object[]> scan(DataContext root) {
                 return Linq4j.asEnumerable(Arrays.asList(
-                    new Object[]{0, "Calcite", 1},
-                    new Object[]{1, "Ignite", 1}
+                    new Object[]{0, 1},
+                    new Object[]{1, 2}
                 ));
             }
 
@@ -914,18 +1000,12 @@ public class PlannerTest extends GridCommonAbstractTest {
 
         IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
 
-        publicSchema.addTable("DEVELOPER", developer);
-        publicSchema.addTable("PROJECT", project);
+        publicSchema.addTable("TEST_TABLE", testTable);
 
         SchemaPlus schema = createRootSchema(false)
             .add("PUBLIC", publicSchema);
 
-        String sql = "SELECT d.id, d.name, d.projectId, p.name0, p.ver0 " +
-            "FROM PUBLIC.Developer d JOIN (" +
-            "SELECT pp.id as id0, pp.name as name0, pp.ver as ver0 FROM PUBLIC.Project pp" +
-            ") p " +
-            "ON d.projectId = p.id0 " +
-            "WHERE (d.projectId + 1) > ?";
+        String sql = "SELECT (ID0 + ID1) AS RES FROM PUBLIC.TEST_TABLE";
 
         RelTraitDef<?>[] traitDefs = {
             DistributionTraitDef.INSTANCE,
@@ -1032,10 +1112,11 @@ public class PlannerTest extends GridCommonAbstractTest {
 
             ectx = new ExecutionContext(taskExecutor, ctx, queryId, fragment.fragmentId(), null, Commons.parametersMap(ctx.parameters()));
 
-            exec = new Implementor(c1 -> r1 -> 0, mailboxRegistry, exchangeService,
-                new TestFailureProcessor(kernal), ectx, log()).go(fragment.root());
+            exec = new LogicalRelImplementor(ectx, c1 -> r1 -> 0, mailboxRegistry, exchangeService,
+                new TestFailureProcessor(kernal)).go(fragment.root());
 
-            ConsumerNode consumer = new ConsumerNode(ectx, exec);
+            RootNode consumer = new RootNode(ectx, r -> {});
+            consumer.register(exec);
 
             //// Remote part
 
@@ -1089,12 +1170,14 @@ public class PlannerTest extends GridCommonAbstractTest {
                 null,
                 Commons.parametersMap(ctx.parameters()));
 
-            exec = new Implementor(c -> r -> 0, mailboxRegistry, exchangeService,
-                new TestFailureProcessor(kernal), ectx, log()).go(fragment.root());
+            exec = new LogicalRelImplementor(ectx, c -> r -> 0, mailboxRegistry, exchangeService,
+                new TestFailureProcessor(kernal)).go(fragment.root());
 
             //// Start execution
 
-            exec.context().execute(exec::request);
+            assert exec instanceof Outbox;
+
+            exec.context().execute(((Outbox<Object[]>) exec)::init);
 
             ArrayList<Object[]> res = new ArrayList<>();
 
@@ -1103,8 +1186,8 @@ public class PlannerTest extends GridCommonAbstractTest {
 
             assertFalse(res.isEmpty());
 
-            Assert.assertArrayEquals(new Object[]{0, "Igor", 0, "Calcite", 1}, res.get(0));
-            Assert.assertArrayEquals(new Object[]{1, "Roman", 0, "Calcite", 1}, res.get(1));
+            Assert.assertArrayEquals(new Object[]{1}, res.get(0));
+            Assert.assertArrayEquals(new Object[]{3}, res.get(1));
         }
     }
 
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolderTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolderTest.java
index a9e62d9..d1587d0 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolderTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ClosableIteratorsHolderTest.java
@@ -23,6 +23,7 @@ import java.util.NoSuchElementException;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.After;
 import org.junit.Before;
@@ -63,6 +64,8 @@ public class ClosableIteratorsHolderTest extends GridCommonAbstractTest {
 
         System.gc();
 
+        GridTestUtils.waitForCondition(() -> iterators.size() < GENERATED, 10_000);
+
         assertTrue(iterators.size() < GENERATED);
     }
 
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ExpressionEvaluationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ExpressionEvaluationTest.java
new file mode 100644
index 0000000..d385f6e
--- /dev/null
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/ExpressionEvaluationTest.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.UUID;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.Expression;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.RexToExpTranslator;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class ExpressionEvaluationTest extends GridCommonAbstractTest {
+    /** */
+    private static final PlanningContext PLANNING_CONTEXT = PlanningContext.EMPTY;
+
+    /** */
+    private static final QueryTaskExecutor THROWING_EXECUTOR = (qid, fId, t) -> {
+        throw new AssertionError();
+    };
+
+    /** */
+    public static final IgniteTypeFactory TYPE_FACTORY = PLANNING_CONTEXT.typeFactory();
+
+    @Test
+    public void testCallEvaluation() throws Exception {
+
+        RexBuilder builder = new RexBuilder(TYPE_FACTORY);
+
+        RexNode call = builder.makeCall(
+            SqlStdOperatorTable.PLUS,
+            builder.makeInputRef(TYPE_FACTORY.createTypeWithNullability(TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), true), 0),
+            builder.makeInputRef(TYPE_FACTORY.createTypeWithNullability(TYPE_FACTORY.createSqlType(SqlTypeName.INTEGER), false), 1));
+
+        RexToExpTranslator translator = new RexToExpTranslator(TYPE_FACTORY);
+
+        Expression expression = translator.translate(call);
+
+        ExecutionContext ctx = context();
+
+        assertEquals(3, expression.<Object>evaluate(ctx, 1, 2));
+        assertEquals(4, expression.<Object>evaluate(ctx, 2, 2));
+        assertEquals(1, expression.<Object>evaluate(ctx, -1, 2));
+        assertEquals(null, expression.<Object>evaluate(ctx, null, 2));
+    }
+
+    public ExecutionContext context(Object... params) {
+        return new ExecutionContext(THROWING_EXECUTOR, PLANNING_CONTEXT, UUID.randomUUID(), 0, null, Commons.parametersMap(params));
+    }
+}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/OutboxTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/OutboxTest.java
deleted file mode 100644
index aa5dbe6..0000000
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/OutboxTest.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.query.calcite.exec;
-
-import com.google.common.collect.ImmutableMap;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
-import org.apache.ignite.internal.processors.query.calcite.trait.AllNodes;
-import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- *
- */
-public class OutboxTest extends GridCommonAbstractTest {
-    /** */
-    private static UUID nodeId;
-
-    /** */
-    private static Destination destination;
-
-    /** */
-    private Outbox<Object[]> outbox;
-
-    /** */
-    private TestNode input;
-
-    /** */
-    private TestExchangeService exch;
-
-    /** */
-    @BeforeClass
-    public static void setupClass() {
-        destination = new AllNodes(Collections.singletonList(nodeId = UUID.randomUUID()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Before
-    public void setUp() throws Exception {
-        exch = new TestExchangeService();
-
-        PlanningContext ctx = PlanningContext.builder()
-            .localNodeId(nodeId)
-            .build();
-
-        ExecutionContext ectx = new ExecutionContext(null, ctx, UUID.randomUUID(), 0, null, ImmutableMap.of());
-
-        input = new TestNode(ectx);
-        outbox = new Outbox<>(exch, new MailboxRegistryImpl(newContext()), ectx, 0, ectx.fragmentId(), input, destination);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testBasicOps() throws Exception {
-        outbox.request();
-
-        assertTrue(input.signal);
-
-        input.signal = false;
-
-        int maxRows = ExchangeService.BATCH_SIZE * (ExchangeService.PER_NODE_BATCH_COUNT + 1);
-        int rows = 0;
-
-        while (input.push(new Object[]{new Object()})) {
-            rows++;
-
-            assertFalse(rows > maxRows);
-        }
-
-        assertEquals(maxRows, rows);
-
-        assertFalse(exch.ids.isEmpty());
-
-        assertEquals(ExchangeService.PER_NODE_BATCH_COUNT, exch.ids.size());
-
-        assertFalse(input.push(new Object[]{new Object()}));
-
-        assertFalse(input.signal);
-
-        outbox.onAcknowledge(nodeId, exch.ids.remove(0));
-
-        assertTrue(input.signal);
-
-        input.signal = false;
-
-        outbox.onAcknowledge(nodeId, exch.ids.remove(0));
-
-        assertFalse(input.signal);
-
-        assertTrue(input.push(new Object[]{new Object()}));
-
-        input.end();
-
-        assertEquals(EndMarker.INSTANCE, F.last(exch.lastBatch));
-    }
-
-    /** */
-    private static class TestExchangeService implements ExchangeService {
-        /** */
-        private List<Integer> ids = new ArrayList<>();
-
-        /** */
-        private List<?> lastBatch;
-
-        /** {@inheritDoc} */
-        @Override public void sendBatch(Outbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) {
-            ids.add(batchId);
-
-            lastBatch = rows;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void acknowledge(Inbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
-            throw new AssertionError();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel(Outbox<?> caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
-            throw new AssertionError();
-        }
-    }
-
-    /** */
-    private static class TestNode extends AbstractNode<Object[]> {
-        /** */
-        private boolean signal;
-
-        /** */
-        private TestNode(ExecutionContext ctx) {
-            super(ctx);
... 230 lines suppressed ...