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/01/29 13:46:48 UTC

[ignite] branch ignite-12248 updated: IGNITE-12448: Calcite integration. Communication protocol. IGNITE-12449: Calcite integration. Execution flow. This closes #7272

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 095804d  IGNITE-12448: Calcite integration. Communication protocol. IGNITE-12449: Calcite integration. Execution flow. This closes #7272
095804d is described below

commit 095804d5dd4c170ab2dd23f7b060047fef3b4b17
Author: Igor Seliverstov <gv...@gmail.com>
AuthorDate: Wed Jan 29 16:46:30 2020 +0300

    IGNITE-12448: Calcite integration. Communication protocol.
    IGNITE-12449: Calcite integration. Execution flow.
    This closes #7272
---
 .../query/calcite/CalciteQueryProcessor.java       |  275 ++-
 .../query/calcite/cluster/MappingServiceImpl.java  |  158 --
 .../processors/query/calcite/exchange/Inbox.java   |  105 --
 .../processors/query/calcite/exchange/Outbox.java  |  202 ---
 .../query/calcite/exec/AbstractNode.java           |  108 +-
 .../query/calcite/exec/ConsumerNode.java           |  238 ++-
 .../calcite/{exchange => exec}/EndMarker.java      |    2 +-
 .../ExchangeService.java}                          |   61 +-
 .../query/calcite/exec/ExchangeServiceImpl.java    |  190 ++
 .../query/calcite/exec/ExecutionContext.java       |  154 ++
 .../query/calcite/exec/ExecutionService.java}      |   32 +-
 .../query/calcite/exec/ExecutionServiceImpl.java   |  825 +++++++++
 .../processors/query/calcite/exec/FilterNode.java  |   14 +-
 .../processors/query/calcite/exec/Implementor.java |  152 +-
 .../processors/query/calcite/exec/Inbox.java       |  442 +++++
 .../processors/query/calcite/exec/JoinNode.java    |   42 +-
 .../query/calcite/exec/MailboxRegistry.java        |   92 +
 .../query/calcite/exec/MailboxRegistryImpl.java    |  132 ++
 .../processors/query/calcite/exec/Node.java        |   53 +-
 .../processors/query/calcite/exec/Outbox.java      |  293 ++++
 .../processors/query/calcite/exec/ProjectNode.java |   12 +-
 .../exec/{Source.java => QueryTaskExecutor.java}   |   15 +-
 .../query/calcite/exec/QueryTaskExecutorImpl.java  |   74 +
 .../query/calcite/exec/ScalarFactory.java          |  175 +-
 .../processors/query/calcite/exec/ScanNode.java    |   79 +-
 .../processors/query/calcite/exec/SingleNode.java  |    9 +-
 .../processors/query/calcite/exec/Sink.java        |   22 +-
 .../EndMarker.java => message/CalciteMessage.java} |   27 +-
 .../calcite/message/CalciteMessageFactory.java     |   53 +
 .../ExecutionContextAware.java}                    |   24 +-
 .../query/calcite/message/GenericRowMessage.java   |  118 ++
 .../query/calcite/message/InboxCancelMessage.java  |  171 ++
 .../MarshalableMessage.java}                       |   24 +-
 .../MessageListener.java}                          |   13 +-
 .../MessageService.java}                           |   32 +-
 .../query/calcite/message/MessageServiceImpl.java  |  287 ++++
 .../query/calcite/message/MessageType.java         |   90 +
 .../message/QueryBatchAcknowledgeMessage.java      |  173 ++
 .../query/calcite/message/QueryBatchMessage.java   |  246 +++
 .../query/calcite/message/QueryCancelRequest.java  |  100 ++
 .../query/calcite/message/QueryStartRequest.java   |  283 +++
 .../query/calcite/message/QueryStartResponse.java  |  174 ++
 .../query/calcite/metadata/FragmentInfo.java       |   10 +-
 .../calcite/metadata/IgniteMdDistribution.java     |    7 +-
 .../calcite/metadata/IgniteMdFragmentInfo.java     |    8 +-
 .../query/calcite/metadata/MappingService.java     |   25 +-
 .../query/calcite/metadata/MappingServiceImpl.java |   78 +
 .../query/calcite/metadata/NodesMapping.java       |   16 +-
 .../metadata/OptimisticPlanningException.java      |   10 +
 .../PartitionService.java}                         |   17 +-
 .../calcite/metadata/PartitionServiceImpl.java     |   63 +
 .../query/calcite/metadata/RelMetadataQueryEx.java |    2 +-
 .../processors/query/calcite/prepare/CacheKey.java |   78 +
 .../query/calcite/prepare/ContextValue.java        |   59 -
 .../query/calcite/prepare/DataContextImpl.java     |   67 -
 .../calcite/prepare/DistributedExecution.java      |  110 --
 .../query/calcite/prepare/IgnitePlanner.java       |  115 +-
 .../query/calcite/prepare/IgniteSqlValidator.java  |   37 +-
 .../query/calcite/prepare/PlannerContext.java      |  345 ----
 .../query/calcite/prepare/PlannerPhase.java        |    6 +-
 .../query/calcite/prepare/PlanningContext.java     |  342 ++++
 .../processors/query/calcite/prepare/Query.java    |   93 -
 .../{QueryExecution.java => QueryPlanCache.java}   |   17 +-
 .../query/calcite/prepare/QueryPlanCacheImpl.java  |  115 ++
 .../{QueryExecution.java => QueryPlanFactory.java} |   14 +-
 .../query/calcite/rel/IgniteReceiver.java          |    9 +
 .../DistributedTable.java}                         |   25 +-
 .../query/calcite/schema/IgniteSchema.java         |   31 +-
 .../query/calcite/schema/IgniteTable.java          |  195 ++-
 .../GraphNode.java => schema/SchemaHolder.java}    |   13 +-
 ...citeSchemaHolder.java => SchemaHolderImpl.java} |   69 +-
 .../GraphNode.java => schema/SortedTable.java}     |   13 +-
 .../query/calcite/schema/TableDescriptor.java      |   65 +
 .../query/calcite/schema/TableDescriptorImpl.java  |  131 ++
 .../processors/query/calcite/serialize/Graph.java  |  114 --
 .../serialize/expression/CallExpression.java       |   19 +-
 .../expression/DynamicParamExpression.java         |    3 +-
 .../serialize/expression/ExpToRexTranslator.java   |    2 +-
 .../serialize/expression/InputRefExpression.java   |    3 +-
 .../serialize/expression/LiteralExpression.java    |   10 +-
 .../serialize/expression/LocalRefExpression.java   |    3 +-
 .../serialize/expression/RexToExpTranslator.java   |    2 +-
 .../serialize/relation/ConversionContext.java      |    4 +-
 .../calcite/serialize/relation/FilterNode.java     |    5 +-
 .../serialize/relation/GraphToRelConverter.java    |   14 +-
 .../query/calcite/serialize/relation/JoinNode.java |    5 +-
 .../calcite/serialize/relation/ProjectNode.java    |    8 +-
 .../calcite/serialize/relation/ReceiverNode.java   |   11 +-
 .../query/calcite/serialize/relation/RelGraph.java |   92 +-
 .../calcite/serialize/relation/RelGraphNode.java   |   58 +-
 .../calcite/serialize/relation/SenderNode.java     |    5 +-
 .../serialize/relation/SerializedTraits.java       |   81 -
 .../calcite/serialize/relation/TableScanNode.java  |   14 +-
 .../query/calcite/serialize/type/DataType.java     |   16 -
 .../query/calcite/serialize/type/StructType.java   |    2 +-
 .../serialize/type/{DataType.java => Types.java}   |   21 +-
 .../splitter/{Splitter.java => Cloner.java}        |  114 +-
 .../query/calcite/splitter/Fragment.java           |  101 +-
 .../query/calcite/splitter/QueryPlan.java          |   37 +-
 .../query/calcite/splitter/RelSource.java          |   17 +-
 .../query/calcite/splitter/RelSourceImpl.java      |   12 +-
 .../query/calcite/splitter/RelTarget.java          |   10 +-
 .../query/calcite/splitter/RelTargetImpl.java      |   15 +-
 .../query/calcite/splitter/Splitter.java           |   62 +-
 .../{DestinationFunction.java => AllNodes.java}    |   31 +-
 .../{DestinationFunction.java => Destination.java} |    6 +-
 .../query/calcite/trait/DistributionFunction.java  |  158 +-
 .../query/calcite/trait/DistributionTrait.java     |    7 +-
 .../query/calcite/trait/IgniteDistributions.java   |   24 +-
 .../AbstractNode.java => trait/Partitioned.java}   |   43 +-
 .../{DestinationFunction.java => RandomNode.java}  |   40 +-
 .../query/calcite/type/IgniteTypeFactory.java      |   77 +-
 .../query/calcite/type/IgniteTypeSystem.java       |    2 +-
 .../processors/query/calcite/type/RowType.java     |  213 ---
 .../GraphNode.java => type/SystemType.java}        |    8 +-
 .../query/calcite/util/AbstractService.java}       |   32 +-
 .../processors/query/calcite/util/Commons.java     |  111 +-
 .../LifecycleAware.java}                           |   20 +-
 .../query/calcite/util/ListFieldsQueryCursor.java  |   37 +-
 .../processors/query/calcite/util/NoOpPlanner.java |  228 +++
 .../{exec/Source.java => util/Service.java}        |   19 +-
 .../processors/query/calcite/util/TableScan.java   |  242 +++
 .../query/calcite/util/TableScanIterator.java      |  146 --
 .../query/calcite/CalciteQueryProcessorTest.java   | 1272 +-------------
 .../processors/query/calcite/PlannerTest.java      | 1815 ++++++++++++++++++++
 .../query/calcite/exchange/OutboxTest.java         |  185 --
 .../query/calcite/exec/AbstractExecutionTest.java  |  175 ++
 .../calcite/exec/ContinuousExecutionTest.java      |  146 ++
 .../query/calcite/exec/ExecutionTest.java          |   40 +-
 .../processors/query/calcite/exec/OutboxTest.java  |  179 ++
 .../query/calcite/message/TestIoManager.java}      |   25 +-
 .../ignite/testsuites/IgniteCalciteTestSuite.java  |   10 +-
 modules/codegen/pom.xml                            |    6 +
 .../ignite/codegen/MessageCodeGenerator.java       |    3 +
 .../ignite/internal/GridKernalContextImpl.java     |    4 +-
 .../ignite/internal/IgniteComponentType.java       |    3 +-
 .../org/apache/ignite/internal/IgniteKernal.java   |    9 +-
 .../managers/communication/GridIoManager.java      |    2 +
 .../managers/communication/GridIoPolicy.java       |    3 +
 .../internal/processors/pool/PoolProcessor.java    |    4 +
 .../processors/query/GridQueryProcessor.java       |   31 +-
 .../internal/processors/query/NoOpQueryEngine.java |   16 +-
 .../internal/processors/query/QueryContext.java    |    2 +-
 .../internal/processors/query/QueryEngine.java     |   16 +-
 .../query/schema/SchemaChangeListener.java         |   29 +-
 .../apache/ignite/thread/SameThreadExecutor.java}  |   28 +-
 146 files changed, 9876 insertions(+), 4387 deletions(-)

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 9ab992b..fc87608 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
@@ -19,60 +19,53 @@ package org.apache.ignite.internal.processors.query.calcite;
 
 import java.util.Collections;
 import java.util.List;
-import java.util.function.BiFunction;
 import org.apache.calcite.config.Lex;
-import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.sql.fun.SqlLibrary;
 import org.apache.calcite.sql.fun.SqlLibraryOperatorTableFactory;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryContext;
 import org.apache.ignite.internal.processors.query.QueryEngine;
-import org.apache.ignite.internal.processors.query.calcite.cluster.MappingServiceImpl;
-import org.apache.ignite.internal.processors.query.calcite.prepare.DistributedExecution;
-import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
-import org.apache.ignite.internal.processors.query.calcite.prepare.Query;
-import org.apache.ignite.internal.processors.query.calcite.prepare.QueryExecution;
-import org.apache.ignite.internal.processors.query.calcite.schema.CalciteSchemaHolder;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExchangeServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionService;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistry;
+import org.apache.ignite.internal.processors.query.calcite.exec.MailboxRegistryImpl;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutorImpl;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionServiceImpl;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCache;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCacheImpl;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolder;
+import org.apache.ignite.internal.processors.query.calcite.schema.SchemaHolderImpl;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
-import org.apache.ignite.resources.LoggerResource;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.processors.query.calcite.util.LifecycleAware;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
-public class CalciteQueryProcessor implements QueryEngine {
+public class CalciteQueryProcessor extends GridProcessorAdapter implements QueryEngine {
     /** */
-    private final CalciteSchemaHolder schemaHolder = new CalciteSchemaHolder();
-
-    /** */
-    private final FrameworkConfig config;
-
-    /** */
-    private IgniteLogger log;
-
-    /** */
-    private GridKernalContext kernalContext;
-
-    /** */
-    public CalciteQueryProcessor() {
-        config = Frameworks.newConfigBuilder()
+    public static final FrameworkConfig FRAMEWORK_CONFIG = Frameworks.newConfigBuilder()
             .parserConfig(SqlParser.configBuilder()
                 // Lexical configuration defines how identifiers are quoted, whether they are converted to upper or lower
                 // case when they are read, and whether identifiers are matched case-sensitively.
-                .setLex(Lex.MYSQL)
+                .setLex(Lex.ORACLE)
                 .build())
             // Dialects support.
             .operatorTable(SqlLibraryOperatorTableFactory.INSTANCE
@@ -80,91 +73,201 @@ public class CalciteQueryProcessor implements QueryEngine {
                     SqlLibrary.STANDARD,
                     SqlLibrary.MYSQL))
             // Context provides a way to store data within the planner session that can be accessed in planner rules.
-            .context(Contexts.of(this))
+            .context(Contexts.empty())
             // Custom cost factory to use during optimization
             .costFactory(null)
-            .typeSystem(IgniteTypeSystem.DEFAULT)
+            .typeSystem(IgniteTypeSystem.INSTANCE)
             .build();
+
+    /** */
+    private final QueryPlanCache queryPlanCache;
+
+    /** */
+    private final QueryTaskExecutor taskExecutor;
+
+    /** */
+    private final FailureProcessor failureProcessor;
+
+    /** */
+    private final PartitionService partitionService;
+
+    /** */
+    private final SchemaHolder schemaHolder;
+
+    /** */
+    private final MessageService messageService;
+
+    /** */
+    private final ExchangeService exchangeService;
+
+    /** */
+    private final MappingService mappingService;
+    
+    /** */
+    private final MailboxRegistry mailboxRegistry;
+    
+    /** */
+    private final ExecutionService executionService;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public CalciteQueryProcessor(GridKernalContext ctx) {
+        this(
+            ctx,
+            ctx.failure(),
+            new SchemaHolderImpl(ctx),
+            new QueryPlanCacheImpl(ctx),
+            new MailboxRegistryImpl(ctx),
+            new QueryTaskExecutorImpl(ctx),
+            new ExecutionServiceImpl(ctx),
+            new PartitionServiceImpl(ctx),
+            new MessageServiceImpl(ctx),
+            new MappingServiceImpl(ctx),
+            new ExchangeServiceImpl(ctx));
     }
 
     /**
-     * @param log Logger.
+     * For tests purpose.
+     * @param ctx Kernal context.
+     * @param failureProcessor Failure processor.
+     * @param schemaHolder Schema holder.
+     * @param queryPlanCache Query cache;
+     * @param mailboxRegistry Mailbox registry.
+     * @param taskExecutor Task executor.
+     * @param executionService Execution service.
+     * @param partitionService Affinity service.
+     * @param messageService Message service.
+     * @param mappingService Mapping service.
+     * @param exchangeService Exchange service.
      */
-    @LoggerResource
-    public void setLogger(IgniteLogger log) {
-        this.log = log;
+    CalciteQueryProcessor(GridKernalContext ctx, FailureProcessor failureProcessor, SchemaHolder schemaHolder, QueryPlanCache queryPlanCache, MailboxRegistry mailboxRegistry, QueryTaskExecutor taskExecutor, ExecutionService executionService, PartitionService partitionService, MessageService messageService,
+        MappingService mappingService, ExchangeService exchangeService) {
+        super(ctx);
+
+        this.failureProcessor = failureProcessor;
+        this.schemaHolder = schemaHolder;
+        this.queryPlanCache = queryPlanCache;
+        this.mailboxRegistry = mailboxRegistry;
+        this.taskExecutor = taskExecutor;
+        this.executionService = executionService;
+        this.partitionService = partitionService;
+        this.messageService = messageService;
+        this.mappingService = mappingService;
+        this.exchangeService = exchangeService;
     }
 
-    /** {@inheritDoc} */
-    @Override public void start(@NotNull GridKernalContext ctx) {
-        kernalContext = ctx;
+    public PartitionService affinityService() {
+        return partitionService;
+    }
 
-        GridInternalSubscriptionProcessor prc = ctx.internalSubscriptionProcessor();
+    /**
+     * @return Query cache.
+     */
+    public QueryPlanCache queryPlanCache() {
+        return queryPlanCache;
+    }
 
-        if (prc != null) // Stubbed context doesn't have such processor
-            prc.registerSchemaChangeListener(schemaHolder);
+    /**
+     * @return Task executor.
+     */
+    public QueryTaskExecutor taskExecutor() {
+        return taskExecutor;
     }
 
-    /** {@inheritDoc} */
-    @Override public void stop() {
+    /**
+     * @return Schema holder.
+     */
+    public SchemaHolder schemaHolder() {
+        return schemaHolder;
     }
 
-    /** {@inheritDoc} */
-    @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String query, Object... params) throws IgniteSQLException {
-        PlannerContext context = context(Commons.convert(ctx), query, params, this::buildContext);
-        QueryExecution execution = prepare(context);
-        FieldsQueryCursor<List<?>> cur = execution.execute();
-        return Collections.singletonList(cur);
+    /**
+     * @return Message service.
+     */
+    public MessageService messageService() {
+        return messageService;
     }
 
     /**
-     * Creates a planner.
-     *
-     * @param traitDefs Trait definitions.
-     * @param ctx Planner context.
-     * @return Ignite planner.
+     * @return Mapping service.
      */
-    public IgnitePlanner planner(RelTraitDef[] traitDefs, PlannerContext ctx) {
-        FrameworkConfig cfg = Frameworks.newConfigBuilder(config)
-                .defaultSchema(ctx.schema())
-                .traitDefs(traitDefs)
-                .context(ctx)
-                .build();
+    public MappingService mappingService() {
+        return mappingService;
+    }
 
-        return new IgnitePlanner(cfg);
+    /**
+     * @return Exchange service.
+     */
+    public ExchangeService exchangeService() {
+        return exchangeService;
     }
 
     /**
-     * @param ctx External context.
-     * @param query Query string.
-     * @param params Query parameters.
-     * @return Query execution context.
+     * @return Mailbox registry.
      */
-    PlannerContext context(@NotNull Context ctx, String query, Object[] params, BiFunction<Context, Query, PlannerContext> clo) { // Package private visibility for tests.
-        return clo.apply(Contexts.chain(ctx, config.getContext()), new Query(query, params));
+    public MailboxRegistry mailboxRegistry() {
+        return mailboxRegistry;
     }
 
-    /** */
-    private PlannerContext buildContext(@NotNull Context parent, @NotNull Query query) {
-        return PlannerContext.builder()
-            .logger(log)
-            .kernalContext(kernalContext)
-            .queryProcessor(this)
-            .parentContext(parent)
-            .query(query)
-            .schema(schemaHolder.schema())
-            .topologyVersion(readyAffinityVersion())
-            .mappingService(new MappingServiceImpl(kernalContext))
-            .build();
+    /**
+     * @return Failure processor.
+     */
+    public FailureProcessor failureProcessor() {
+        return failureProcessor;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        onStart(ctx,
+            executionService,
+            mailboxRegistry,
+            partitionService,
+            schemaHolder,
+            messageService,
+            taskExecutor,
+            mappingService,
+            queryPlanCache,
+            exchangeService
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) {
+        onStop(
+            executionService,
+            mailboxRegistry,
+            partitionService,
+            schemaHolder,
+            messageService,
+            taskExecutor,
+            mappingService,
+            queryPlanCache,
+            exchangeService
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext qryCtx, @Nullable String schemaName,
+        String query, Object... params) throws IgniteSQLException {
+        
+        // TODO multiline query.
+        return Collections.singletonList(executionService.executeQuery(qryCtx, schemaName, query, params));
     }
 
     /** */
-    private QueryExecution prepare(PlannerContext ctx) {
-        return new DistributedExecution(ctx);
+    private void onStart(GridKernalContext ctx, Service... services) {
+        for (Service service : services) {
+            if (service instanceof LifecycleAware)
+                ((LifecycleAware) service).onStart(ctx);
+        }
     }
 
     /** */
-    private AffinityTopologyVersion readyAffinityVersion() {
-        return kernalContext.cache().context().exchange().readyAffinityVersion();
+    private void onStop(Service... services) {
+        for (Service service : services) {
+            if (service instanceof LifecycleAware)
+                ((LifecycleAware) service).onStop();
+        }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
deleted file mode 100644
index d420540..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/cluster/MappingServiceImpl.java
+++ /dev/null
@@ -1,158 +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.cluster;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
-import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.typedef.F;
-
-import static org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping.DEDUPLICATED;
-
-/**
- *
- */
-public class MappingServiceImpl implements MappingService {
-    /** */
-    private final GridKernalContext ctx;
-
-    /**
-     * @param ctx Grid kernal context.
-     */
-    public MappingServiceImpl(GridKernalContext ctx) {
-        this.ctx = ctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public NodesMapping local() {
-        return new NodesMapping(Collections.singletonList(ctx.discovery().localNode().id()), null, DEDUPLICATED);
-    }
-
-    /** {@inheritDoc} */
-    @Override public NodesMapping random(AffinityTopologyVersion topVer) {
-        List<ClusterNode> nodes = ctx.discovery().discoCache(topVer).serverNodes();
-
-        return new NodesMapping(Commons.transform(nodes, ClusterNode::id), null, DEDUPLICATED);
-    }
-
-    /** {@inheritDoc} */
-    @Override public NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer) {
-        GridCacheContext<?,?> cctx = ctx.cache().context().cacheContext(cacheId);
-
-        return cctx.isReplicated() ? replicatedLocation(cctx, topVer) : partitionedLocation(cctx, topVer);
-    }
-
-    /**
-     * @param cctx Cache context.
-     * @param topVer Topology version.
-     * @return Node mapping, describing location of interested data.
-     */
-    private NodesMapping partitionedLocation(GridCacheContext<?,?> cctx, AffinityTopologyVersion topVer) {
-        byte flags = NodesMapping.HAS_PARTITIONED_CACHES;
-
-        List<List<ClusterNode>> assignments = cctx.affinity().assignments(topVer);
-        List<List<UUID>> res;
-
-        if (cctx.config().getWriteSynchronizationMode() == CacheWriteSynchronizationMode.PRIMARY_SYNC) {
-            res = new ArrayList<>(assignments.size());
-
-            for (List<ClusterNode> partNodes : assignments)
-                res.add(F.isEmpty(partNodes) ? Collections.emptyList() : Collections.singletonList(F.first(partNodes).id()));
-        }
-        else if (!cctx.topology().rebalanceFinished(topVer)) {
-            res = new ArrayList<>(assignments.size());
-
-            flags |= NodesMapping.HAS_MOVING_PARTITIONS;
-
-            for (int part = 0; part < assignments.size(); part++) {
-                List<ClusterNode> partNodes = assignments.get(part);
-                List<UUID> partIds = new ArrayList<>(partNodes.size());
-
-                for (ClusterNode node : partNodes) {
-                    if (cctx.topology().partitionState(node.id(), part) == GridDhtPartitionState.OWNING)
-                        partIds.add(node.id());
-                }
-
-                res.add(partIds);
-            }
-        }
-        else
-            res = Commons.transform(assignments, nodes -> Commons.transform(nodes, ClusterNode::id));
-
-        return new NodesMapping(null, res, flags);
-    }
-
-    /**
-     * @param cctx Cache context.
-     * @param topVer Topology version.
-     * @return Node mapping, describing location of interested data.
-     */
-    private NodesMapping replicatedLocation(GridCacheContext<?,?> cctx, AffinityTopologyVersion topVer) {
-        byte flags = NodesMapping.HAS_REPLICATED_CACHES;
-
-        if (cctx.config().getNodeFilter() != null)
-            flags |= NodesMapping.PARTIALLY_REPLICATED;
-
-        GridDhtPartitionTopology topology = cctx.topology();
-
-        List<ClusterNode> nodes = cctx.discovery().discoCache(topVer).cacheGroupAffinityNodes(cctx.cacheId());
-        List<UUID> res;
-
-        if (!topology.rebalanceFinished(topVer)) {
-            flags |= NodesMapping.PARTIALLY_REPLICATED;
-
-            res = new ArrayList<>(nodes.size());
-
-            int parts = topology.partitions();
-
-            for (ClusterNode node : nodes) {
-                if (isOwner(node.id(), topology, parts))
-                    res.add(node.id());
-            }
-        }
-        else
-            res = Commons.transform(nodes, ClusterNode::id);
-
-        return new NodesMapping(res, null, flags);
-    }
-
-    /**
-     * @param nodeId Node ID.
-     * @param topology Topology version.
-     * @param parts partitions count.
-     * @return {@code True} if all partitions are in {@link GridDhtPartitionState#OWNING} state on the given node.
-     */
-    private boolean isOwner(UUID nodeId, GridDhtPartitionTopology topology, int parts) {
-        for (int p = 0; p < parts; p++) {
-            if (topology.partitionState(nodeId, p) != GridDhtPartitionState.OWNING)
-                return false;
-        }
-        return true;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
deleted file mode 100644
index fd0cb12..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Inbox.java
+++ /dev/null
@@ -1,105 +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.exchange;
-
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.query.calcite.exec.SingleNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.Sink;
-import org.apache.ignite.internal.processors.query.calcite.exec.Source;
-
-/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12448
- */
-public class Inbox<T> implements SingleNode<T> {
-    /** */
-    private final GridCacheVersion queryId;
-
-    /** */
-    private final long exchangeId;
-
-    /** */
-    private Sink<T> target;
-
-    /** */
-    private Collection<UUID> sources;
-
-    /** */
-    private Comparator<T> comparator;
-
-    /** */
-    private ExchangeProcessor srvc;
-
-    /**
-     * @param queryId Query ID.
-     * @param exchangeId Exchange ID.
-     */
-    public Inbox(GridCacheVersion queryId, long exchangeId) {
-        this.queryId = queryId;
-        this.exchangeId = exchangeId;
-    }
-
-    /**
-     * Binds this Inbox to the given target.
-     *
-     * @param target Target sink.
-     * @param sources Source nodes.
-     * @param comparator Optional comparator for merge exchange.
-     */
-    public void bind(Sink<T> target, Collection<UUID> sources, Comparator<T> comparator) {
-        this.target = target;
-        this.sources = sources;
-        this.comparator = comparator;
-    }
-
-    /**
-     * @param srvc Exchange service.
-     */
-    void init(ExchangeProcessor srvc) {
-        this.srvc = srvc;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void signal() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sources(List<Source> sources) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Sink<T> sink(int idx) {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * Pushes a batch into a buffer.
-     *
-     * @param source Source node.
-     * @param batchId Batch ID.
-     * @param rows Rows.
-     */
-    public void push(UUID source, int batchId, List<?> rows) {
-
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
deleted file mode 100644
index b64573ff..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/Outbox.java
+++ /dev/null
@@ -1,202 +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.exchange;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.query.calcite.exec.AbstractNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.SingleNode;
-import org.apache.ignite.internal.processors.query.calcite.exec.Sink;
-import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12448
- */
-public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T> {
-    /** */
-    private final Map<UUID, Destination> perNode = new HashMap<>();
-
-    /** */
-    private final GridCacheVersion queryId;
-
-    /** */
-    private final long exchangeId;
-
-    /** */
-    private final DestinationFunction function;
-
-    /** */
-    private ExchangeProcessor srvc;
-
-    /**
-     *
-     * @param queryId Query ID.
-     * @param exchangeId Exchange ID.
-     * @param function Destination function.
-     */
-    public Outbox(GridCacheVersion queryId, long exchangeId, DestinationFunction function) {
-        super(Sink.noOp());
-        this.queryId = queryId;
-        this.exchangeId = exchangeId;
-        this.function = function;
-    }
-
-    public void init(ExchangeProcessor srvc) {
-        this.srvc = srvc;
-
-        srvc.register(this);
-
-        signal();
-    }
-
-    public void acknowledge(UUID nodeId, int batchId) {
-        perNode.get(nodeId).onAcknowledge(batchId);
-    }
-
-    @Override public Sink<T> sink(int idx) {
-        if (idx != 0)
-            throw new IndexOutOfBoundsException();
-
-        return this;
-    }
-
-    @Override public boolean push(T row) {
-        List<UUID> nodes = function.destination(row);
-
-        if (F.isEmpty(nodes))
-            return true;
-
-        List<Destination> destinations = new ArrayList<>(nodes.size());
-
-        for (UUID node : nodes) {
-            Destination dest = perNode.computeIfAbsent(node, Destination::new);
-
-            if (!dest.ready()) {
-                dest.needSignal();
-
-                return false;
-            }
-
-            destinations.add(dest);
-        }
-
-        for (Destination dest : destinations)
-            dest.add(row);
-
-        return true;
-    }
-
-    @Override public void end() {
-        for (UUID node : function.targets())
-            perNode.computeIfAbsent(node, Destination::new).end();
-
-        srvc.unregister(this);
-    }
-
-    /** */
-    private final class Destination {
-        /** */
-        private final UUID nodeId;
-
-        /** */
-        private int hwm = -1;
-
-        /** */
-        private int lwm = -1;
-
-        /** */
-        private ArrayList<Object> curr = new ArrayList<>(ExchangeProcessor.BATCH_SIZE + 1); // extra space for end marker;
-
-        /** */
-        private boolean needSignal;
-
-        /** */
-        private Destination(UUID nodeId) {
-            this.nodeId = nodeId;
-        }
-
-        /**
-         * Adds a row to current batch.
-         *
-         * @param row Row.
-         */
-        public void add(T row) {
-            if (curr.size() == ExchangeProcessor.BATCH_SIZE) {
-                assert ready() && srvc != null;
-
-                srvc.send(queryId, exchangeId, nodeId, ++hwm, curr);
-
-                curr = new ArrayList<>(ExchangeProcessor.BATCH_SIZE);
-            }
-
-            curr.add(row);
-        }
-
-        /**
-         * Signals data is over.
-         */
-        public void end() {
-            curr.add(EndMarker.INSTANCE);
-
-            assert srvc != null;
-
-            srvc.send(queryId, exchangeId, nodeId, hwm, curr);
-
-            curr = null;
-            hwm = Integer.MAX_VALUE;
-        }
-
-        /**
-         * Checks whether there is a place for a new row.
-         *
-         * @return {@code True} is it possible to add a row to a batch.
-         */
-        boolean ready() {
-            return hwm - lwm < ExchangeProcessor.PER_NODE_BATCH_COUNT || curr.size() < ExchangeProcessor.BATCH_SIZE;
-        }
-
-        /**
-         * Callback method.
-         *
-         * @param id batch ID.
-         */
-        void onAcknowledge(int id) {
-            if (lwm < id) {
-                lwm = id;
-
-                if (needSignal) {
-                    needSignal = false;
-
-                    signal();
-                }
-            }
-        }
-
-        /**
-         * Sets "needSignal" flag.
-         */
-        public void needSignal() {
-            needSignal = true;
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractNode.java
index 65fb965..8726333 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/AbstractNode.java
@@ -17,40 +17,118 @@
 
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
-import java.util.Collections;
+import com.google.common.collect.ImmutableList;
+import java.util.Collection;
+import java.util.Comparator;
 import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
 
 /**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Abstract node of execution tree.
  */
 public abstract class AbstractNode<T> implements Node<T> {
+    /** for debug purpose */
+    private volatile Thread thread;
+
     /** */
-    protected final Sink<T> target;
+    private final ImmutableList<Node<T>> inputs;
 
     /** */
-    protected List<Source> sources;
+    private Sink<T> target;
 
     /**
-     * @param target Target.
+     * {@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.
      */
-    protected AbstractNode(Sink<T> target) {
-        this.target = target;
+    private ExecutionContext ctx;
+
+    /**
+     * @param ctx Execution context.
+     */
+    protected AbstractNode(ExecutionContext ctx) {
+        this(ctx, ImmutableList.of());
     }
 
-    /** {@inheritDoc} */
-    @Override public void sources(List<Source> sources) {
-        this.sources = Collections.unmodifiableList(sources);
+    /**
+     * @param ctx Execution context.
+     */
+    protected AbstractNode(ExecutionContext ctx, @NotNull Node<T> input) {
+        this(ctx, ImmutableList.of(input));
     }
 
     /**
-     * @param idx Index of a source to signal/
+     * @param ctx Execution context.
      */
-    public void signal(int idx) {
-        sources.get(idx).signal();
+    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 void signal() {
-        sources.forEach(Source::signal);
+    @Override public Sink<T> target() {
+        return target;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Node<T>> inputs() {
+        return inputs;
+    }
+
+    /** */
+    protected void context(ExecutionContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutionContext context() {
+        return ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request() {
+        checkThread();
+
+        inputs().forEach(Node::request);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        checkThread();
+
+        context().setCancelled();
+        inputs().forEach(Node::cancel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        checkThread();
+
+        inputs().forEach(Node::reset);
+    }
+
+    /**
+     * 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));
+    }
+
+    /** */
+    protected void checkThread() {
+        if (!U.assertionsEnabled())
+            return;
+
+        if (thread == null)
+            thread = Thread.currentThread();
+        else
+            assert thread == Thread.currentThread();
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ConsumerNode.java
index 453dee7..a686866 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/ConsumerNode.java
@@ -20,26 +20,95 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
 import java.util.ArrayDeque;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
-import java.util.Objects;
+import java.util.UUID;
+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;
 
 /**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Client iterator.
  */
-public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]> {
+public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]>, Iterator<Object[]>, AutoCloseable {
+    /** */
+    public static final Consumer<ConsumerNode> NO_OP = c -> {};
+
     /** */
     private static final int DEFAULT_BUFFER_SIZE = 1000;
 
     /** */
-    private static final Object[] END = new Object[0];
+    private enum State {
+        RUNNING, CANCELLED, END
+    }
+
+    /** */
+    private final ReentrantLock lock;
+
+    /** */
+    private final Condition cond;
+
+    /** */
+    private final int bufferSize;
 
     /** */
-    private ArrayDeque<Object[]> buff;
+    private final ArrayDeque<Object> buff;
 
     /** */
-    public ConsumerNode() {
-        super(Sink.noOp());
+    private final Consumer<ConsumerNode> onClose;
+
+    /** */
+    private Object cur;
+
+    /** */
+    private boolean requested;
+
+    /** */
+    private volatile State state = State.RUNNING;
+
+    /**
+     * @param ctx Execution context.
+     */
+    public ConsumerNode(ExecutionContext ctx, Node<Object[]> input) {
+        this(ctx, input, DEFAULT_BUFFER_SIZE);
+    }
 
-        buff = new ArrayDeque<>(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);
+    }
+
+    /**
+     * @param ctx Execution context.
+     */
+    public ConsumerNode(ExecutionContext ctx, Node<Object[]> input, int bufferSize, Consumer<ConsumerNode> onClose) {
+        super(ctx, input);
+
+        this.bufferSize = bufferSize;
+        this.onClose = onClose;
+
+        // extra space for possible END marker
+        buff = new ArrayDeque<>(bufferSize + 1);
+        lock = new ReentrantLock();
+        cond = lock.newCondition();
+
+        link();
+    }
+    
+    public UUID queryId() {
+        return context().queryId();
     }
 
     /** {@inheritDoc} */
@@ -51,36 +120,165 @@ public class ConsumerNode extends AbstractNode<Object[]> implements SingleNode<O
     }
 
     /** {@inheritDoc} */
+    @Override public void request() {
+        context().execute(this::requestInternal);
+    }
+
+    /** */
+    private void requestInternal() {
+        checkThread();
+
+        if (state == State.RUNNING)
+            input().request();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        context().setCancelled();
+
+        if (state != State.RUNNING)
+            return;
+
+        lock.lock();
+        try {
+            if (state != State.RUNNING)
+                return;
+
+            state = State.CANCELLED;
+            buff.clear();
+            cond.signalAll();
+        }
+        finally {
+            lock.unlock();
+        }
+        
+        context().execute(input()::cancel);
+        onClose.accept(this);
+    }
+    
+    public boolean canceled() {
+        return state == State.CANCELLED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        cancel();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean push(Object[] row) {
-        if (buff.size() == DEFAULT_BUFFER_SIZE)
+        if (state != State.RUNNING)
             return false;
 
-        buff.add(row);
+        lock.lock();
+        try {
+            if (state != State.RUNNING)
+                return false;
+
+            if (buff.size() == bufferSize) {
+                requested = false;
+                return false;
+            }
+
+            buff.offer(row);
+            cond.signalAll();
+        }
+        finally {
+            lock.unlock();
+        }
 
         return true;
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        buff.add(END);
+        if (state != State.RUNNING)
+            return;
+
+        lock.lock();
+        try {
+            if (state != State.RUNNING)
+                return;
+
+            buff.offer(EndMarker.INSTANCE);
+            cond.signalAll();
+        }
+        finally {
+            lock.unlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public boolean hasNext() {
-        if (buff.isEmpty())
-            signal();
-
-        return buff.peek() != END;
+        if (cur != null)
+            return true;
+        else if (state == State.END)
+            return false;
+        else
+            return (cur = take()) != null;
     }
 
     /** {@inheritDoc} */
     @Override public Object[] next() {
-        if (buff.isEmpty())
-            signal();
-
-        if(!hasNext())
+        if (!hasNext())
             throw new NoSuchElementException();
 
-        return Objects.requireNonNull(buff.poll());
+        Object cur0 = cur;
+        cur = null;
+
+        return (Object[]) cur0;
+    }
+
+    /** */
+    private Object take() {
+        if (state == State.CANCELLED)
+            throw U.convertException(new QueryCancelledException());
+
+        lock.lock();
+        try {
+            if (state == State.CANCELLED)
+                throw U.convertException(new QueryCancelledException());
+
+            assert state == State.RUNNING;
+
+            while (buff.isEmpty()) {
+                requestIfNeeded();
+
+                cond.await();
+
+                if (state == State.CANCELLED)
+                    throw U.convertException(new QueryCancelledException());
+
+                assert state == State.RUNNING;
+            }
+
+            Object row = buff.poll();
+
+            if (row != EndMarker.INSTANCE)
+                return row;
+
+            state = State.END;
+        }
+        catch (InterruptedException e) {
+            throw new IgniteInterruptedException(e);
+        }
+        finally {
+            lock.unlock();
+        }
+
+        assert state == State.END;
+
+        onClose.accept(this);
+        
+        return null;
+    }
+
+    /** */
+    private void requestIfNeeded() {
+        if (requested)
+            return;
+
+        request();
+        requested = true;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/EndMarker.java
similarity index 94%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/EndMarker.java
index d4368c7..377e116 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/EndMarker.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exchange;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import java.io.ObjectStreamException;
 import java.io.Serializable;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
similarity index 53%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
index 1fb2beb..24b5383 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/ExchangeProcessor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeService.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exchange;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import java.util.List;
 import java.util.UUID;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
  *
  */
-public interface ExchangeProcessor {
+public interface ExchangeService extends Service {
     /** A number of rows in a single batch. */
     int BATCH_SIZE = 200;
 
@@ -32,50 +32,39 @@ public interface ExchangeProcessor {
     int PER_NODE_BATCH_COUNT = 10;
 
     /**
-     * Registers an outbox in Exchange service.
-     * Registered outbox will be notified each time a consumer processed a batch of rows.
-     * @param outbox Outbox.
-     * @return Registered outbox.
-     */
-    <T> Outbox<T> register(Outbox<T> outbox);
-
-    /**
-     * Unregisters an outbox in Exchange service.
-     * @param outbox Outbox to unregister.
-     */
-    <T> void unregister(Outbox<T> outbox);
-
-    /**
-     * Registers an inbox in Exchange service.
-     * Registered inbox starts consuming data from remote sources.
-     * In case an inbox with the same [queryId, exchangeId] pair is already registered, previously registered inbox is return.
-     * @param inbox Inbox.
-     * @return Registered inbox.
-     */
-    <T> Inbox<T> register(Inbox<T> inbox);
-
-    /**
-     * Unregisters an inbox in Exchange service.
-     * @param inbox Inbox to unregister.
-     */
-    <T> void unregister(Inbox<T> inbox);
-
-    /**
      * 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.
      * @param exchangeId Exchange ID.
-     * @param nodeId Target node ID.
      * @param batchId Batch ID.
      * @param rows Data rows.
      */
-    void send(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId, List<?> rows);
+    void sendBatch(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows);
 
     /**
      * 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(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId);
+
+    /**
+     * 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 nodeId Node ID to notify.
      * @param batchId Batch ID.
      */
-    void acknowledge(GridCacheVersion queryId, long exchangeId, UUID nodeId, int batchId);
+    void cancel(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
new file mode 100644
index 0000000..7559f06
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java
@@ -0,0 +1,190 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+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;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryBatchAcknowledgeMessage;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryBatchMessage;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+
+/**
+ *
+ */
+public class ExchangeServiceImpl extends AbstractService implements ExchangeService {
+    /** */
+    private QueryTaskExecutor taskExecutor;
+
+    /** */
+    private MailboxRegistry mailboxRegistry;
+
+    /** */
+    private MessageService messageService;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public ExchangeServiceImpl(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /**
+     * @param taskExecutor Task executor.
+     */
+    public void taskExecutor(QueryTaskExecutor taskExecutor) {
+        this.taskExecutor = taskExecutor;
+    }
+
+    /**
+     * @return Task executor.
+     */
+    public QueryTaskExecutor taskExecutor() {
+        return taskExecutor;
+    }
+
+    /**
+     * @param mailboxRegistry Mailbox registry.
+     */
+    public void mailboxRegistry(MailboxRegistry mailboxRegistry) {
+        this.mailboxRegistry = mailboxRegistry;
+    }
+
+    /**
+     * @return  Mailbox registry.
+     */
+    public MailboxRegistry mailboxRegistry() {
+        return mailboxRegistry;
+    }
+
+    /**
+     * @param messageService Message service.
+     */
+    public void messageService(MessageService messageService) {
+        this.messageService = messageService;
+    }
+
+    /**
+     * @return  Message service.
+     */
+    public MessageService messageService() {
+        return messageService;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendBatch(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) {
+        messageService().send(nodeId, new QueryBatchMessage(queryId, fragmentId, exchangeId, batchId, rows));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void acknowledge(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
+        messageService().send(nodeId, new QueryBatchAcknowledgeMessage(queryId, fragmentId, exchangeId, batchId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel(Object caller, UUID nodeId, UUID queryId, long fragmentId, long exchangeId, int batchId) {
+        messageService().send(nodeId, new InboxCancelMessage(queryId, fragmentId, exchangeId, batchId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        CalciteQueryProcessor proc = Objects.requireNonNull(Commons.lookupComponent(ctx, CalciteQueryProcessor.class));
+
+        taskExecutor(proc.taskExecutor());
+        mailboxRegistry(proc.mailboxRegistry());
+        messageService(proc.messageService());
+
+        init();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init() {
+        messageService().register((n, m) -> onMessage(n, (InboxCancelMessage) m), MessageType.QUERY_INBOX_CANCEL_MESSAGE);
+        messageService().register((n, m) -> onMessage(n, (QueryBatchAcknowledgeMessage) m), MessageType.QUERY_ACKNOWLEDGE_MESSAGE);
+        messageService().register((n, m) -> onMessage(n, (QueryBatchMessage) m), MessageType.QUERY_BATCH_MESSAGE);
+    }
+
+    /** */
+    protected void onMessage(UUID nodeId, InboxCancelMessage msg) {
+        Inbox<?> inbox = mailboxRegistry().inbox(msg.queryId(), msg.exchangeId());
+
+        if (inbox != null)
+            inbox.cancel();
+        else if (log.isDebugEnabled()) {
+            log.debug("Stale cancel message received: [" +
+                "nodeId=" + nodeId + ", " +
+                "queryId=" + msg.queryId() + ", " +
+                "fragmentId=" + msg.fragmentId() + ", " +
+                "exchangeId=" + msg.exchangeId() + ", " +
+                "batchId=" + msg.batchId() + "]");
+        }
+    }
+
+    /** */
+    protected void onMessage(UUID nodeId, QueryBatchAcknowledgeMessage msg) {
+        Outbox<?> outbox = mailboxRegistry().outbox(msg.queryId(), msg.exchangeId());
+
+        if (outbox != null)
+            outbox.onAcknowledge(nodeId, msg.batchId());
+        else if (log.isDebugEnabled()) {
+            log.debug("Stale acknowledge message received: [" +
+                "nodeId=" + nodeId + ", " +
+                "queryId=" + msg.queryId() + ", " +
+                "fragmentId=" + msg.fragmentId() + ", " +
+                "exchangeId=" + msg.exchangeId() + ", " +
+                "batchId=" + msg.batchId() + "]");
+        }
+    }
+
+    /** */
+    protected void onMessage(UUID nodeId, QueryBatchMessage msg) {
+        Inbox<?> inbox = mailboxRegistry().inbox(msg.queryId(), msg.exchangeId());
+
+        if (inbox == null && msg.batchId() == 0)
+            // first message sent before a fragment is built
+            // note that an inbox source fragment id is also used as an exchange id
+            inbox = mailboxRegistry().register(new Inbox<>(this, mailboxRegistry(), baseInboxContext(msg.queryId(), msg.fragmentId()), msg.exchangeId(), msg.exchangeId()));
+
+        if (inbox != null)
+            inbox.onBatchReceived(nodeId, msg.batchId(), msg.rows());
+        else if (log.isDebugEnabled()){
+            log.debug("Stale batch message received: [" +
+                "nodeId=" + nodeId + ", " +
+                "queryId=" + msg.queryId() + ", " +
+                "fragmentId=" + msg.fragmentId() + ", " +
+                "exchangeId=" + msg.exchangeId() + ", " +
+                "batchId=" + msg.batchId() + "]");
+        }
+    }
+
+    /**
+     * @return Minimal execution context to meet Inbox needs.
+     */
+    private ExecutionContext baseInboxContext(UUID queryId, long fragmentId) {
+        PlanningContext ctx = PlanningContext.builder().logger(log).build();
+        return new ExecutionContext(taskExecutor(), ctx, queryId, fragmentId, null, ImmutableMap.of());
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
new file mode 100644
index 0000000..0b4177a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
@@ -0,0 +1,154 @@
+/*
+ * 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.Map;
+import java.util.UUID;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+
+/**
+ * Runtime context allowing access to the tables in a database.
+ */
+public class ExecutionContext implements DataContext {
+    /** */
+    private final UUID queryId;
+
+    /** */
+    private final long fragmentId;
+
+    /** */
+    private final PlanningContext ctx;
+
+    /** */
+    private final int[] parts;
+
+    /** */
+    private final Map<String, Object> params;
+
+    /** */
+    private final QueryTaskExecutor executor;
+
+    /** */
+    private volatile boolean cancelled;
+
+    /**
+     * @param ctx Parent context.
+     * @param queryId Query ID.
+     * @param fragmentId Fragment ID.
+     * @param parts Partitions.
+     * @param params Parameters.
+     */
+    public ExecutionContext(QueryTaskExecutor executor, PlanningContext ctx, UUID queryId, long fragmentId, int[] parts, Map<String, Object> params) {
+        this.executor = executor;
+        this.queryId = queryId;
+        this.fragmentId = fragmentId;
+        this.parts = parts;
+        this.params = params;
+        this.ctx = ctx;
+    }
+
+    /**
+     * @return Parent context.
+     */
+    public PlanningContext parent() {
+        return ctx;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return queryId;
+    }
+
+    /**
+     * @return Fragment ID.
+     */
+    public long fragmentId() {
+        return fragmentId;
+    }
+
+    /**
+     * @return Interested partitions.
+     */
+    public int[] partitions() {
+        return parts;
+    }
+
+    /**
+     * @return Keep binary flag.
+     */
+    public boolean keepBinary() {
+        return false; // TODO
+    }
+
+    /**
+     * @return MVCC snapshot.
+     */
+    public MvccSnapshot mvccSnapshot() {
+        return null; // TODO
+    }
+
+    /**
+     * @return Cancelled flag.
+     */
+    public boolean cancelled() {
+        return cancelled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SchemaPlus getRootSchema() {
+        return ctx.schema();
+    }
+
+    /** {@inheritDoc} */
+    @Override public JavaTypeFactory getTypeFactory() {
+        return ctx.typeFactory();
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryProvider getQueryProvider() {
+        return null; // TODO
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(String name) {
+        return params.get(name);
+    }
+
+    /**
+     * Sets cancelled flag.
+     */
+    public void setCancelled() {
+        cancelled = true;
+    }
+
+    /**
+     * Executes a query task.
+     *
+     * @param task Query task.
+     */
+    public void execute(Runnable task) {
+        executor.execute(queryId, fragmentId, task);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
similarity index 59%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
index d0e9f04..16afe41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryEngine.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionService.java
@@ -15,32 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import java.util.List;
+import java.util.UUID;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.QueryContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
-public interface QueryEngine {
-
+public interface ExecutionService extends Service {
     /**
-     * @param ctx Kernal context.
+     * Executes a query.
+     *
+     * @param ctx Query external context, contains flags and connection settings like a locale or a timezone.
+     * @param schema Schema name.
+     * @param query Query.
+     * @param params Query parameters.
+     * @return Query cursor.
      */
-    void start(GridKernalContext ctx);
-
-    /** */
-    void stop();
+    FieldsQueryCursor<List<?>> executeQuery(@Nullable QueryContext ctx, String schema, String query, Object[] params);
 
     /**
-     * @param ctx Query context, may be null.
-     * @param query Query.
-     * @param params Optional query parameters.
-     * @return Query cursor.
-     * @throws IgniteSQLException If failed.
+     * Cancels a running query.
+     *
+     * @param queryId Query ID.
      */
-    List<FieldsQueryCursor<List<?>>> query(@Nullable QueryContext ctx, String query, Object... params) throws IgniteSQLException;
+    void cancelQuery(UUID queryId);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
new file mode 100644
index 0000000..e4e1fd5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -0,0 +1,825 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+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.message.MessageService;
+import org.apache.ignite.internal.processors.query.calcite.message.MessageType;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryCancelRequest;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryStartRequest;
+import org.apache.ignite.internal.processors.query.calcite.message.QueryStartResponse;
+import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
+import org.apache.ignite.internal.processors.query.calcite.prepare.CacheKey;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerPhase;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerType;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.prepare.QueryPlanCache;
+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.splitter.Fragment;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.splitter.Splitter;
+import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.ListFieldsQueryCursor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG;
+
+/**
+ *
+ */
+public class ExecutionServiceImpl extends AbstractService implements ExecutionService {
+    /** */
+    private final DiscoveryEventListener discoLsnr;
+
+    /** */
+    private UUID localNodeId;
+
+    /** */
+    private GridEventStorageManager eventManager;
+
+    /** */
+    private GridCachePartitionExchangeManager<?,?> exchangeManager;
+
+    /** */
+    private QueryPlanCache queryPlanCache;
+
+    /** */
+    private SchemaHolder schemaHolder;
+
+    /** */
+    private QueryTaskExecutor taskExecutor;
+
+    /** */
+    private FailureProcessor failureProcessor;
+
+    /** */
+    private PartitionService partitionService;
+
+    /** */
+    private MailboxRegistry mailboxRegistry;
+
+    /** */
+    private MappingService mappingService;
+
+    /** */
+    private MessageService messageService;
+
+    /** */
+    private ExchangeService exchangeService;
+
+    /** */
+    private final Map<UUID, QueryInfo> running;
+
+    /**
+     * @param ctx Kernal.
+     */
+    public ExecutionServiceImpl(GridKernalContext ctx) {
+        super(ctx);
+
+        discoLsnr = (e, c) -> onNodeLeft(e.eventNode().id());
+        running = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * @param localNodeId Local node ID.
+     */
+    public void localNodeId(UUID localNodeId) {
+        this.localNodeId = localNodeId;
+    }
+
+    /**
+     * @return Local node ID.
+     */
+    public UUID localNodeId() {
+        return localNodeId;
+    }
+
+    /**
+     * @param queryPlanCache Query cache.
+     */
+    public void queryPlanCache(QueryPlanCache queryPlanCache) {
+        this.queryPlanCache = queryPlanCache;
+    }
+
+    /**
+     * @return Query cache.
+     */
+    public QueryPlanCache queryPlanCache() {
+        return queryPlanCache;
+    }
+
+    /**
+     * @param schemaHolder Schema holder.
+     */
+    public void schemaHolder(SchemaHolder schemaHolder) {
+        this.schemaHolder = schemaHolder;
+    }
+
+    /**
+     * @return Schema holder.
+     */
+    public SchemaHolder schemaHolder() {
+        return schemaHolder;
+    }
+
+    /**
+     * @param taskExecutor Task executor.
+     */
+    public void taskExecutor(QueryTaskExecutor taskExecutor) {
+        this.taskExecutor = taskExecutor;
+    }
+
+    /**
+     * @return Task executor.
+     */
+    public QueryTaskExecutor taskExecutor() {
+        return taskExecutor;
+    }
+
+    /**
+     * @param failureProcessor Failure processor.
+     */
+    public void failureProcessor(FailureProcessor failureProcessor) {
+        this.failureProcessor = failureProcessor;
+    }
+
+    /**
+     * @return Failure processor.
+     */
+    public FailureProcessor failureProcessor() {
+        return failureProcessor;
+    }
+
+    /**
+     * @param partitionService Partition service.
+     */
+    public void partitionService(PartitionService partitionService) {
+        this.partitionService = partitionService;
+    }
+
+    /**
+     * @return Partition service.
+     */
+    public PartitionService partitionService() {
+        return partitionService;
+    }
+
+    /**
+     * @param mailboxRegistry Mailbox registry.
+     */
+    public void mailboxRegistry(MailboxRegistry mailboxRegistry) {
+        this.mailboxRegistry = mailboxRegistry;
+    }
+
+    /**
+     * @return Mailbox registry.
+     */
+    public MailboxRegistry mailboxRegistry() {
+        return mailboxRegistry;
+    }
+
+    /**
+     * @param mappingService Mapping service.
+     */
+    public void mappingService(MappingService mappingService) {
+        this.mappingService = mappingService;
+    }
+
+    /**
+     * @return Mapping service.
+     */
+    public MappingService mappingService() {
+        return mappingService;
+    }
+
+    /**
+     * @param messageService Message service.
+     */
+    public void messageService(MessageService messageService) {
+        this.messageService = messageService;
+    }
+
+    /**
+     * @return Message service.
+     */
+    public MessageService messageService() {
+        return messageService;
+    }
+
+    /**
+     * @param exchangeService Exchange service.
+     */
+    public void exchangeService(ExchangeService exchangeService) {
+        this.exchangeService = exchangeService;
+    }
+
+    /**
+     * @return Exchange service.
+     */
+    public ExchangeService exchangeService() {
+        return exchangeService;
+    }
+
+    /**
+     * @param eventManager Event manager.
+     */
+    public void eventManager(GridEventStorageManager eventManager) {
+        this.eventManager = eventManager;
+    }
+
+    /**
+     * @return Event manager.
+     */
+    public GridEventStorageManager eventManager() {
+        return eventManager;
+    }
+
+    /**
+     * @param exchangeManager Exchange manager.
+     */
+    public void exchangeManager(GridCachePartitionExchangeManager<?,?> exchangeManager) {
+        this.exchangeManager = exchangeManager;
+    }
+
+    /**
+     * @return Exchange manager.
+     */
+    public GridCachePartitionExchangeManager<?, ?> exchangeManager() {
+        return exchangeManager;
+    }
+
+    /** {@inheritDoc} */
+    @Override public FieldsQueryCursor<List<?>> executeQuery(@Nullable QueryContext ctx, String schema, String query, Object[] params) {
+        UUID queryId = UUID.randomUUID();
+
+        PlanningContext pctx = createContext(ctx, schema, query, params);
+
+        QueryPlan plan = prepare(pctx);
+
+        // Local execution
+        Fragment local = F.first(plan.fragments());
+
+        if (U.assertionsEnabled()) {
+            assert local != null;
+
+            NodesMapping mapping = local.mapping();
+
+            assert mapping != null;
+
+            List<UUID> nodes = mapping.nodes();
+
+            assert nodes != null && nodes.size() == 1 && F.first(nodes).equals(pctx.localNodeId());
+        }
+
+        ExecutionContext ectx = new ExecutionContext(
+            taskExecutor(), pctx, queryId, local.fragmentId(), local.mapping().partitions(pctx.localNodeId()), Commons.parametersMap(params)
+        );
+
+        Node<Object[]> node = new Implementor(partitionService(), mailboxRegistry(), exchangeService(), failureProcessor(), ectx, log).go(local.root());
+
+        assert !(node instanceof SenderNode);
+
+        QueryInfo info = new QueryInfo(ectx, local.root().getRowType(), new ConsumerNode(ectx, node, this::onConsumerClose));
+
+        if (plan.fragments().size() == 1)
+            running.put(queryId, info);
+        else {
+            // remote execution
+            RelOp<IgniteRel, RelGraph> converter = new RelToGraphConverter();
+            List<Pair<UUID, QueryStartRequest>> requests = new ArrayList<>();
+
+            for (Fragment remote : plan.fragments().subList(1, plan.fragments().size())) {
+                long id = remote.fragmentId();
+                NodesMapping mapping = remote.mapping();
+                RelGraph graph = converter.go(remote.root());
+
+                for (UUID nodeId : mapping.nodes()) {
+                    info.addFragment(nodeId, id);
+
+                    QueryStartRequest req = new QueryStartRequest(
+                        queryId,
+                        id,
+                        schema,
+                        graph,
+                        pctx.topologyVersion(),
+                        mapping.partitions(nodeId),
+                        params);
+
+                    requests.add(Pair.of(nodeId, req));
+                }
+            }
+
+            running.put(queryId, info);
+
+            // start remote execution
+            for (Pair<UUID, QueryStartRequest> pair : requests)
+                messageService().send(pair.left, pair.right);
+        }
+
+        // start local execution
+        info.consumer.request();
+
+        info.awaitAllReplies();
+
+        // TODO weak map to stop query on cursor collected by GC.
+        return new ListFieldsQueryCursor<>(info.type(), info.<Object[]>iterator(), Arrays::asList);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelQuery(UUID queryId) {
+        mailboxRegistry().outboxes(queryId).forEach(Outbox::cancel);
+
+        QueryInfo info = running.get(queryId);
+
+        if (info != null)
+            info.cancel();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        localNodeId(ctx.localNodeId());
+        exchangeManager(ctx.cache().context().exchange());
+        eventManager(ctx.event());
+
+        CalciteQueryProcessor proc = Objects.requireNonNull(Commons.lookupComponent(ctx, CalciteQueryProcessor.class));
+
+        queryPlanCache(proc.queryPlanCache());
+        schemaHolder(proc.schemaHolder());
+        taskExecutor(proc.taskExecutor());
+        failureProcessor(proc.failureProcessor());
+        partitionService(proc.affinityService());
+        mailboxRegistry(proc.mailboxRegistry());
+        mappingService(proc.mappingService());
+        messageService(proc.messageService());
+        exchangeService(proc.exchangeService());
+
+        init();
+     }
+
+    /** {@inheritDoc} */
+    @Override public void init() {
+        messageService().register((n,m) -> onMessage(n, (QueryStartRequest) m), MessageType.QUERY_START_REQUEST);
+        messageService().register((n,m) -> onMessage(n, (QueryStartResponse) m), MessageType.QUERY_START_RESPONSE);
+        messageService().register((n,m) -> onMessage(n, (QueryCancelRequest) m), MessageType.QUERY_CANCEL_REQUEST);
+
+        eventManager().addDiscoveryEventListener(discoLsnr, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() {
+        eventManager().removeDiscoveryEventListener(discoLsnr, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
+
+        running.clear();
+    }
+
+    /** */
+    protected AffinityTopologyVersion topologyVersion() {
+        return exchangeManager().readyAffinityVersion();
+    }
+
+    /** */
+    private QueryPlan prepare(PlanningContext ctx) {
+        CacheKey cacheKey = new CacheKey(ctx.schema().getName(), ctx.query());
+
+        QueryPlan plan = queryPlanCache().queryPlan(ctx, cacheKey, this::prepare0);
+
+        plan.init(mappingService(), ctx);
+
+        return plan;
+    }
+
+    /** */
+    private QueryPlan prepare0(PlanningContext ctx) {
+        IgnitePlanner planner = ctx.planner();
+
+        try {
+            String query = ctx.query();
+
+            assert query != null;
+
+            // Parse
+            SqlNode sqlNode = planner.parse(query);
+
+            // Validate
+            sqlNode = planner.validate(sqlNode);
+
+            // Convert to Relational operators graph
+            RelNode rel = planner.convert(sqlNode);
+
+            // Transformation chain
+            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .simplify();
+
+            IgniteRel igniteRel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
+
+            // Split query plan to multi-step one.
+            return new Splitter().go(igniteRel);
+        }
+        catch (SqlParseException e) {
+            IgniteSQLException ex = new IgniteSQLException("Failed to parse query.", IgniteQueryErrorCode.PARSING, e);
+            Commons.close(planner, ex);
+            throw ex;
+        }
+        catch (ValidationException e) {
+            IgniteSQLException ex = new IgniteSQLException("Failed to validate query.", IgniteQueryErrorCode.UNKNOWN, e);
+            Commons.close(planner, ex);
+            throw ex;
+        }
+        catch (Exception e) {
+            IgniteSQLException ex = new IgniteSQLException("Failed to plan query.", IgniteQueryErrorCode.UNKNOWN, e);
+            Commons.close(planner, ex);
+            throw ex;
+        }
+    }
+
+    /** */
+    private PlanningContext createContext(@Nullable QueryContext qryCtx, @Nullable String schemaName, String query, Object[] params) {
+        RelTraitDef<?>[] traitDefs = {
+            ConventionTraitDef.INSTANCE
+            , DistributionTraitDef.INSTANCE
+            //, RelCollationTraitDef.INSTANCE TODO
+        };
+
+        return PlanningContext.builder()
+            .localNodeId(localNodeId())
+            .parentContext(Commons.convert(qryCtx)) // TODO Connection config on the basis of query context
+            .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                .defaultSchema(schemaName != null
+                    ? schemaHolder().schema().getSubSchema(schemaName)
+                    : schemaHolder().schema())
+                .traitDefs(traitDefs)
+                .build())
+            .query(query)
+            .parameters(params)
+            .topologyVersion(topologyVersion())
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    private PlanningContext createContext(@Nullable String schemaName, UUID originatingNodeId, AffinityTopologyVersion topVer) {
+        // TODO pass to context user locale and timezone.
+
+        RelTraitDef<?>[] traitDefs = {
+            ConventionTraitDef.INSTANCE
+            //, RelCollationTraitDef.INSTANCE TODO
+        };
+
+        return PlanningContext.builder()
+            .localNodeId(localNodeId())
+            .originatingNodeId(originatingNodeId)
+            .parentContext(Contexts.empty())
+            .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
+                .defaultSchema(schemaName != null
+                    ? schemaHolder().schema().getSubSchema(schemaName)
+                    : schemaHolder().schema())
+                .traitDefs(traitDefs)
+                .build())
+            .topologyVersion(topVer)
+            .logger(log)
+            .build();
+    }
+
+    /** */
+    protected void onMessage(UUID nodeId, QueryStartRequest msg) {
+        assert nodeId != null && msg != null;
+
+        PlanningContext ctx = createContext(msg.schema(), nodeId, msg.topologyVersion());
+
+        try (IgnitePlanner planner = ctx.planner()) {
+            IgniteRel root = planner.convert(msg.plan());
+
+            assert root instanceof IgniteSender : root;
+
+            // TODO do we need a local optimisation phase here?
+            ExecutionContext execCtx = new ExecutionContext(
+                taskExecutor(),
+                ctx,
+                msg.queryId(),
+                msg.fragmentId(),
+                msg.partitions(),
+                Commons.parametersMap(msg.parameters())
+            );
+
+            Node<Object[]> node = new Implementor(partitionService(), mailboxRegistry(), exchangeService(), failureProcessor(), execCtx, log).go(root);
+
+            assert node instanceof Outbox : node;
+
+            node.request();
+
+            messageService().send(nodeId, new QueryStartResponse(msg.queryId(), msg.fragmentId()));
+        }
+        catch (Exception ex) {
+            messageService().send(nodeId, new QueryStartResponse(msg.queryId(), msg.fragmentId(), ex));
+        }
+    }
+
+    /** */
+    private void onMessage(UUID nodeId, QueryCancelRequest msg) {
+        assert nodeId != null && msg != null;
+
+        cancelQuery(msg.queryId());
+    }
+
+    /** */
+    private void onMessage(UUID nodeId, QueryStartResponse msg) {
+        assert nodeId != null && msg != null;
+
+        QueryInfo info = running.get(msg.queryId());
+
+        if (info != null)
+            info.onResponse(nodeId, msg.fragmentId(), msg.error());
+    }
+
+    /** */
+    private void onConsumerClose(ConsumerNode consumer) {
+        if (consumer.canceled())
+            cancelQuery(consumer.queryId());
+        else
+            running.remove(consumer.queryId());
+    }
+
+    /** */
+    private void onNodeLeft(UUID nodeId) {
+        running.forEach((uuid, queryInfo) -> queryInfo.onNodeLeft(nodeId));
+    }
+
+    /** */
+    private static class RemoteFragmentKey {
+        /** */
+        private final UUID nodeId;
+
+        /** */
+        private final long fragmentId;
+
+        /** */
+        private RemoteFragmentKey(UUID nodeId, long fragmentId) {
+            this.nodeId = nodeId;
+            this.fragmentId = fragmentId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            RemoteFragmentKey that = (RemoteFragmentKey) o;
+
+            if (fragmentId != that.fragmentId)
+                return false;
+            return nodeId.equals(that.nodeId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = nodeId.hashCode();
+            result = 31 * result + (int) (fragmentId ^ (fragmentId >>> 32));
+            return result;
+        }
+    }
+
+    /** */
+    private enum QueryState {
+        RUNNING, CANCELLING, CANCELLED
+    }
+
+    /** */
+    private final class QueryInfo implements QueryCancellable {
+        /** */
+        private final ExecutionContext ctx;
+
+        /** */
+        private final RelDataType type;
+
+        /** */
+        private final ConsumerNode consumer;
+
+        /** remote nodes */
+        private final Set<UUID> remotes;
+
+        /** node to fragment */
+        private final Set<RemoteFragmentKey> waiting;
+
+        /** */
+        private QueryState state;
+
+        /** */
+        private Throwable error;
+
+        /** */
+        private QueryInfo(ExecutionContext ctx, RelDataType type, ConsumerNode consumer) {
+            this.ctx = ctx;
+            this.type = type;
+            this.consumer = consumer;
+
+            remotes = new HashSet<>();
+            waiting = new HashSet<>();
+
+            state = QueryState.RUNNING;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void doCancel() {
+            cancel();
+        }
+
+        /** */
+        private ConsumerNode localNode() {
+            return consumer;
+        }
+
+        /** */
+        private RelDataType type() {
+            return type;
+        }
+
+        /** */
+        private <T> Iterator<T> iterator() {
+            return (Iterator<T>) consumer;
+        }
+
+        /** */
+        private void addFragment(UUID nodeId, long fragmentId) {
+            remotes.add(nodeId);
+            waiting.add(new RemoteFragmentKey(nodeId, fragmentId));
+        }
+
+        /** */
+        private void awaitAllReplies() {
+            Throwable error;
+
+            try {
+                synchronized (this) {
+                    while (!waiting.isEmpty())
+                        wait();
+
+                    error = this.error;
+                }
+            }
+            catch (InterruptedException e) {
+                throw new IgniteInterruptedException(e);
+            }
+
+            if (error != null)
+                throw new IgniteSQLException("Failed to execute query.", error);
+        }
+
+        /** */
+        private void cancel() {
+            boolean cancelLocal = false;
+            boolean cancelRemote = false;
+            QueryState state0 = null;
+
+            synchronized (this) {
+                if (state == QueryState.CANCELLED)
+                    return;
+
+                if (state == QueryState.RUNNING) {
+                    cancelLocal = true;
+                    state0 = state = QueryState.CANCELLING;
+                }
+
+                if (state == QueryState.CANCELLING && waiting.isEmpty()) {
+                    cancelRemote = true;
+                    state0 = state = QueryState.CANCELLED;
+                }
+            }
+
+            if (cancelLocal)
+                consumer.cancel();
+
+            if (cancelRemote)
+                messageService().send(remotes, new QueryCancelRequest(ctx.queryId()));
+
+            if (state0 == QueryState.CANCELLED)
+                running.remove(ctx.queryId());
+        }
+
+        /** */
+        private void onNodeLeft(UUID nodeId) {
+            List<RemoteFragmentKey> fragments = null;
+
+            synchronized (this) {
+                for (RemoteFragmentKey fragment : waiting) {
+                    if (!fragment.nodeId.equals(nodeId))
+                        continue;
+
+                    if (fragments == null)
+                        fragments = new ArrayList<>();
+
+                    fragments.add(fragment);
+                }
+            }
+
+            if (!F.isEmpty(fragments)) {
+                ClusterTopologyCheckedException ex = new ClusterTopologyCheckedException("Failed to start query, node left. nodeId=" + nodeId);
+
+                for (RemoteFragmentKey fragment : fragments)
+                    onResponse(fragment, ex);
+            }
+        }
+
+        /** */
+        private void onResponse(UUID nodeId, long fragmentId, Throwable error) {
+            onResponse(new RemoteFragmentKey(nodeId, fragmentId), error);
+        }
+
+        /** */
+        private void onResponse(RemoteFragmentKey fragment, Throwable error) {
+            boolean cancel;
+
+            synchronized (this) {
+                if (!waiting.remove(fragment))
+                    return;
+
+                if (error != null) {
+                    if (this.error != null)
+                        this.error.addSuppressed(error);
+                    else
+                        this.error = error;
+                }
+
+                boolean empty = waiting.isEmpty();
+
+                cancel = empty && this.error != null;
+
+                if (empty)
+                    notifyAll();
+            }
+
+            if (cancel)
+                cancel();
+        }
+    }
+}
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
index 722f4a9..5da1db2 100644
--- 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
@@ -20,20 +20,22 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
 import java.util.function.Predicate;
 
 /**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ *
  */
 public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, Sink<Object[]> {
     /** */
     private final Predicate<Object[]> predicate;
 
     /**
-     * @param target Target.
+     * @param ctx Execution context.
      * @param predicate Predicate.
      */
-    public FilterNode(Sink<Object[]> target, Predicate<Object[]> predicate) {
-        super(target);
+    public FilterNode(ExecutionContext ctx, Node<Object[]> input, Predicate<Object[]> predicate) {
+        super(ctx, input);
 
         this.predicate = predicate;
+
+        link();
     }
 
     /** {@inheritDoc} */
@@ -46,11 +48,11 @@ public class FilterNode extends AbstractNode<Object[]> implements SingleNode<Obj
 
     /** {@inheritDoc} */
     @Override public boolean push(Object[] row) {
-        return !predicate.test(row) || target.push(row);
+        return !predicate.test(row) || target().push(row);
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        target.end();
+        target().end();
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Implementor.java
index df6d10b..4199f7f 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/Implementor.java
@@ -17,17 +17,14 @@
 
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Deque;
-import java.util.List;
-import org.apache.calcite.DataContext;
+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.rex.RexBuilder;
 import org.apache.calcite.schema.ScannableTable;
-import org.apache.ignite.internal.processors.query.calcite.exchange.Outbox;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.calcite.metadata.PartitionService;
 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;
@@ -38,151 +35,116 @@ import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRelVisitor;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
 import org.apache.ignite.internal.processors.query.calcite.rel.RelOp;
+import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
 import org.apache.ignite.internal.processors.query.calcite.splitter.RelTarget;
-import org.apache.ignite.internal.processors.query.calcite.trait.DestinationFunction;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 
-import static org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue.PLANNER_CONTEXT;
-import static org.apache.ignite.internal.processors.query.calcite.prepare.ContextValue.QUERY_ID;
-
 /**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Implements a query plan.
  */
 public class Implementor implements IgniteRelVisitor<Node<Object[]>>, RelOp<IgniteRel, Node<Object[]>> {
     /** */
-    private final PlannerContext ctx;
+    private final ExecutionContext ctx;
+
+    /** */
+    private final PartitionService partitionService;
 
     /** */
-    private final DataContext root;
+    private final ExchangeService exchangeService;
 
     /** */
-    private final ScalarFactory factory;
+    private final MailboxRegistry mailboxRegistry;
 
     /** */
-    private Deque<Sink<Object[]>> stack;
+    private final ScalarFactory scalarFactory;
 
     /**
-     * @param root 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(DataContext root) {
-        this.root = root;
+    public Implementor(PartitionService partitionService, MailboxRegistry mailboxRegistry, ExchangeService exchangeService, FailureProcessor failure, ExecutionContext ctx, IgniteLogger log) {
+        this.partitionService = partitionService;
+        this.mailboxRegistry = mailboxRegistry;
+        this.exchangeService = exchangeService;
+        this.ctx = ctx;
 
-        ctx = PLANNER_CONTEXT.get(root);
-        factory = new ScalarFactory(new RexBuilder(ctx.typeFactory()));
-        stack = new ArrayDeque<>();
+        scalarFactory = new ScalarFactory(ctx.getTypeFactory(), failure, log);
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteSender rel) {
-        assert stack.isEmpty();
-
         RelTarget target = rel.target();
+        long targetFragmentId = target.fragmentId();
         IgniteDistribution distribution = target.distribution();
-        DestinationFunction function = distribution.function().toDestination(ctx, target.mapping(), distribution.getKeys());
-
-        Outbox<Object[]> res = new Outbox<>(QUERY_ID.get(root), target.exchangeId(), function);
+        Destination destination = distribution.function().destination(partitionService, target.mapping(), distribution.getKeys());
 
-        stack.push(res.sink());
+        // 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);
 
-        res.source(source(rel.getInput()));
+        mailboxRegistry.register(outbox);
 
-        return res;
+        return outbox;
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteFilter rel) {
-        assert !stack.isEmpty();
-
-        FilterNode res = new FilterNode(stack.pop(), factory.filterPredicate(root, rel.getCondition(), rel.getRowType()));
-
-        stack.push(res.sink());
-
-        res.source(source(rel.getInput()));
-
-        return res;
+        Predicate<Object[]> predicate = scalarFactory.filterPredicate(ctx, rel.getCondition(), rel.getRowType());
+        return new FilterNode(ctx, visit(rel.getInput()), predicate);
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteProject rel) {
-        assert !stack.isEmpty();
-
-        ProjectNode res = new ProjectNode(stack.pop(), factory.projectExpression(root, rel.getProjects(), rel.getInput().getRowType()));
-
-        stack.push(res.sink());
-
-        res.source(source(rel.getInput()));
-
-        return res;
+        Function<Object[], Object[]> projection = scalarFactory.projectExpression(ctx, rel.getProjects(), rel.getInput().getRowType());
+        return new ProjectNode(ctx, visit(rel.getInput()), projection);
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteJoin rel) {
-        assert !stack.isEmpty();
-
-        JoinNode res = new JoinNode(stack.pop(), factory.joinExpression(root, rel.getCondition(), rel.getLeft().getRowType(), rel.getRight().getRowType()));
-
-        stack.push(res.sink(1));
-        stack.push(res.sink(0));
-
-        res.sources(sources(rel.getInputs()));
-
-        return res;
+        BiFunction<Object[], Object[], Object[]> expression = scalarFactory.joinExpression(ctx, rel.getCondition(), rel.getLeft().getRowType(), rel.getRight().getRowType());
+        return new JoinNode(ctx, visit(rel.getLeft()), visit(rel.getRight()), expression);
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteTableScan rel) {
-        assert !stack.isEmpty();
-
-        Iterable<Object[]> source = rel.getTable().unwrap(ScannableTable.class).scan(root);
-
-        return new ScanNode(stack.pop(), source);
+        return new ScanNode(ctx, rel.getTable().unwrap(ScannableTable.class).scan(ctx));
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteReceiver rel) {
-        throw new AssertionError(); // TODO
-    }
+        RelSource source = rel.source();
 
-    /** {@inheritDoc} */
-    @Override public Node<Object[]> visit(IgniteExchange rel) {
-        throw new AssertionError();
+        // 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()));
+
+        // here may be an already created (to consume rows from remote nodes) inbox
+        // without proper context, we need to init it with a right one.
+        inbox.init(ctx, source.mapping().nodes(), scalarFactory.comparator(ctx, rel.collations(), rel.getRowType()));
+
+        return inbox;
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> visit(IgniteRel rel) {
-        throw new AssertionError();
+        return rel.accept(this);
     }
 
-    /** */
-    private Source source(RelNode rel) {
-        if (rel.getConvention() != IgniteConvention.INSTANCE)
-            throw new IllegalStateException("INTERPRETABLE is required.");
-
-        return ((IgniteRel) rel).accept(this);
+    /** {@inheritDoc} */
+    @Override public Node<Object[]> visit(IgniteExchange rel) {
+        throw new AssertionError();
     }
 
     /** */
-    private List<Source> sources(List<RelNode> rels) {
-        ArrayList<Source> res = new ArrayList<>(rels.size());
-
-        for (RelNode rel : rels) {
-            res.add(source(rel));
-        }
-
-        return res;
+    private Node<Object[]> visit(RelNode rel) {
+        return visit((IgniteRel) rel);
     }
 
     /** {@inheritDoc} */
     @Override public Node<Object[]> go(IgniteRel rel) {
-        if (rel instanceof IgniteSender)
-            return visit((IgniteSender) rel);
-
-        ConsumerNode res = new ConsumerNode();
-
-        stack.push(res.sink());
-
-        res.source(source(rel));
-
-        return res;
+        return visit(rel);
     }
 }
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/Inbox.java
new file mode 100644
index 0000000..de2118f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Inbox.java
@@ -0,0 +1,442 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+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.jetbrains.annotations.NotNull;
+
+/**
+ * A part of exchange.
+ */
+public class Inbox<T> extends AbstractNode<T> implements SingleNode<T>, AutoCloseable {
+    /** */
+    private final ExchangeService exchange;
+
+    /** */
+    private final MailboxRegistry registry;
+
+    /** */
+    private final long exchangeId;
+
+    /** */
+    private final long sourceFragmentId;
+
+    /** */
+    private final Map<UUID, Buffer> perNodeBuffers;
+
+    /** */
+    private Collection<UUID> sources;
+
+    /** */
+    private Comparator<T> comparator;
+
+    /** */
+    private List<Buffer> buffers;
+
+    /** */
+    private boolean end;
+
+    /**
+     * @param exchange Exchange service.
+     * @param registry Mailbox registry.
+     * @param ctx Execution context.
+     * @param sourceFragmentId Source fragment ID.
+     * @param exchangeId Exchange ID.
+     */
+    public Inbox(ExchangeService exchange, MailboxRegistry registry, ExecutionContext ctx, long sourceFragmentId, long exchangeId) {
+        super(ctx);
+        this.exchange = exchange;
+        this.registry = registry;
+
+        this.sourceFragmentId = sourceFragmentId;
+        this.exchangeId = exchangeId;
+
+        perNodeBuffers = new HashMap<>();
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return context().queryId();
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * Inits this Inbox.
+     *
+     * @param ctx Execution context.
+     * @param sources Source nodes.
+     * @param comparator Optional comparator for merge exchange.
+     */
+    public void init(ExecutionContext ctx, Collection<UUID> sources, Comparator<T> comparator) {
+        this.comparator = comparator;
+        this.sources = sources;
+
+        context(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request() {
+        prepare();
+        pushInternal();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        checkThread();
+        context().setCancelled();
+        close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        registry.unregister(this);
+    }
+
+    /**
+     * Pushes a batch into a buffer.
+     *
+     * @param source Source node.
+     * @param batchId Batch ID.
+     * @param rows Rows.
+     */
+    public void onBatchReceived(UUID source, int batchId, List<?> rows) {
+        checkThread();
+        Buffer buffer = getOrCreateBuffer(source);
+
+        if (buffer.add(batchId, rows))
+            pushInternal();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Sink<T> sink(int idx) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** */
+    private void pushInternal() {
+        checkThread();
+
+        if (context().cancelled())
+            close();
+        else if (!end && ready()) {
+            if (comparator != null)
+                pushOrdered();
+            else
+                pushUnordered();
+        }
+    }
+
+    /** */
+    private void pushOrdered() {
+        PriorityQueue<Pair<T, Buffer>> heap = new PriorityQueue<>(buffers.size(), Map.Entry.comparingByKey(comparator));
+
+        ListIterator<Buffer> it = buffers.listIterator();
+
+        while (it.hasNext()) {
+            Buffer buffer = it.next();
+
+            switch (buffer.check()) {
+                case END:
+                    it.remove();
+
+                    break;
+                case READY:
+                    heap.offer(Pair.of((T)buffer.peek(), buffer));
+
+                    break;
+                case WAITING:
+
+                    return;
+            }
+        }
+
+        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))
+                return;
+
+            buffer.remove();
+
+            switch (buffer.check()) {
+                case END:
+                    buffers.remove(buffer);
+
+                    break;
+                case READY:
+                    heap.offer(Pair.of((T)buffer.peek(), buffer));
+
+                    break;
+                case WAITING:
+
+                    return;
+            }
+        }
+
+        end = true;
+        target.end();
+        close();
+    }
+
+    /** */
+    private void pushUnordered() {
+        int size = buffers.size();
+
+        if (size <= 0 && !end)
+            throw new AssertionError("size=" + size + ", end=" + end);
+
+        int idx = ThreadLocalRandom.current().nextInt(size);
+        int noProgress = 0;
+
+        Sink<T> target = target();
+
+        while (size > 0) {
+            Buffer buffer = buffers.get(idx);
+
+            switch (buffer.check()) {
+                case END:
+                    buffers.remove(idx);
+
+                    if (idx == --size)
+                        idx = 0;
+
+                    continue;
+                case READY:
+                    if (!target.push((T)buffer.peek()))
+                        return;
+
+                    buffer.remove();
+                    noProgress = 0;
+
+                    break;
+                case WAITING:
+                    if (++noProgress >= size)
+                        return;
+
+                    break;
+            }
+
+            if (++idx == 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);
+
+        assert perNodeBuffers.size() == sources.size();
+
+        buffers = new ArrayList<>(perNodeBuffers.values());
+    }
+
+    /** */
+    private boolean ready() {
+        return buffers != null;
+    }
+
+    /** */
+    private Buffer getOrCreateBuffer(UUID nodeId) {
+        return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+    }
+
+    /** */
+    private Buffer createBuffer(UUID nodeId) {
+        return new Buffer(nodeId, this);
+    }
+
+    /** */
+    private static final class Batch implements Comparable<Batch> {
+        /** */
+        private final int batchId;
+
+        /** */
+        private final List<?> rows;
+
+        /** */
+        private int idx;
+
+        /** */
+        private Batch(int batchId, List<?> rows) {
+            this.batchId = batchId;
+            this.rows = rows;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Batch batch = (Batch) o;
+
+            return batchId == batch.batchId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return batchId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(@NotNull Inbox.Batch o) {
+            return Integer.compare(batchId, o.batchId);
+        }
+    }
+
+    /** */
+    private enum State {
+        END, READY, WAITING
+    }
+
+    /** */
+    private static final class Buffer {
+        /** */
+        private static final Batch WAITING = new Batch(0, null);
+
+        /** */
+        private static final Batch END = new Batch(0, null);
+
+        /** */
+        private final Inbox<?> owner;
+
+        /** */
+        private final UUID nodeId;
+
+        /** */
+        private int lastEnqueued = -1;
+
+        /** */
+        private final PriorityQueue<Batch> batches = new PriorityQueue<>(ExchangeService.PER_NODE_BATCH_COUNT);
+
+        /** */
+        private Batch curr = WAITING;
+
+        /** */
+        private Buffer(UUID nodeId, Inbox<?> owner) {
+            this.nodeId = nodeId;
+            this.owner = owner;
+        }
+
+        /** */
+        private boolean add(int id, List<?> rows) {
+            batches.offer(new Batch(id, rows));
+
+            return curr == WAITING && batches.peek().batchId == lastEnqueued + 1;
+        }
+
+        /** */
+        private Batch pollBatch() {
+            if (batches.isEmpty() || batches.peek().batchId != lastEnqueued + 1)
+                return WAITING;
+
+            Batch batch = batches.poll();
+
+            assert batch != null && batch.batchId == lastEnqueued + 1;
+
+            lastEnqueued = batch.batchId;
+
+            return batch;
+        }
+
+        /** */
+        private State check() {
+            if (curr == END)
+                return State.END;
+
+            if (curr == WAITING)
+                curr = pollBatch();
+
+            if (curr == WAITING)
+                return State.WAITING;
+
+            if (curr.rows.get(curr.idx) == EndMarker.INSTANCE) {
+                curr = END;
+
+                return State.END;
+            }
+
+            return State.READY;
+        }
+
+        /** */
+        private Object peek() {
+            assert curr != null;
+            assert curr != WAITING;
+            assert curr != END;
+            assert curr.rows.get(curr.idx) != EndMarker.INSTANCE;
+
+            return curr.rows.get(curr.idx);
+        }
+
+        /** */
+        private Object remove() {
+            assert curr != null;
+            assert curr != WAITING;
+            assert curr != END;
+            assert curr.rows.get(curr.idx) != EndMarker.INSTANCE;
+
+            Object row = curr.rows.set(curr.idx++, null);
+
+            if (curr.idx == curr.rows.size()) {
+                owner.acknowledge(nodeId, curr.batchId);
+
+                curr = pollBatch();
+            }
+
+            return row;
+        }
+    }
+}
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
index 8521f4a..46b5bdb 100644
--- 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
@@ -17,11 +17,12 @@
 
 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 https://issues.apache.org/jira/browse/IGNITE-12449
+ * TODO remove buffers.
  */
 public class JoinNode extends AbstractNode<Object[]> {
     /** */
@@ -43,16 +44,17 @@ public class JoinNode extends AbstractNode<Object[]> {
     private boolean end;
 
     /**
-     *
-     * @param target Target.
+     * @param ctx Execution context.
      * @param expression Join expression.
      */
-    public JoinNode(Sink<Object[]> target, BiFunction<Object[], Object[], Object[]> expression) {
-        super(target);
+    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<>();
 
-        left = new ArraySink<>();
-        right = new ArraySink<>();
+        link();
     }
 
     /** {@inheritDoc} */
@@ -68,29 +70,33 @@ public class JoinNode extends AbstractNode<Object[]> {
     }
 
     /** {@inheritDoc} */
-    @Override public void signal() {
-        if (end)
-            return;
+    @Override public void request() {
+        checkThread();
 
-        if (left.end && right.end)
-            tryFlush();
-
-        assert sources != null && sources.size() == 2;
+        if (context().cancelled() || end)
+            return;
 
         if (!left.end)
-            signal(0);
+            input(0).request();
         if (!right.end)
-            signal(1);
+            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)) {
+                    if (row != null && !target().push(row)) {
                         leftIdx = i;
                         rightIdx = j;
 
@@ -100,7 +106,7 @@ public class JoinNode extends AbstractNode<Object[]> {
             }
 
             end = true;
-            target.end();
+            target().end();
         }
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java
new file mode 100644
index 0000000..7dd462f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistry.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
+/**
+ *
+ */
+public interface MailboxRegistry extends Service {
+    /**
+     * Tries to register and inbox node and returns it if success or returns previously registered inbox otherwise.
+     *
+     * @param inbox Inbox.
+     * @return Registered inbox.
+     */
+    Inbox<?> register(Inbox<?> inbox);
+
+    /**
+     * Unregisters an inbox.
+     *
+     * @param inbox Inbox to unregister.
+     */
+    void unregister(Inbox<?> inbox);
+
+    /**
+     * Registers an outbox.
+     *
+     * @param outbox Outbox to register.
+     */
+    void register(Outbox<?> outbox);
+
+    /**
+     * Unregisters an outbox.
+     *
+     * @param outbox Outbox to unregister.
+     */
+    void unregister(Outbox<?> outbox);
+
+    /**
+     * Returns a registered outbox by provided query ID, exchange ID pair.
+     *
+     * @param queryId Query ID.
+     * @param exchangeId Exchange ID.
+     *
+     * @return Registered outbox. May be {@code null} if execution was cancelled.
+     */
+    Outbox<?> outbox(UUID queryId, long exchangeId);
+
+    /**
+     * Returns a registered inbox by provided query ID, exchange ID pair.
+     *
+     * @param queryId Query ID.
+     * @param exchangeId Exchange ID.
+     *
+     * @return Registered inbox. May be {@code null} if execution was cancelled.
+     */
+    Inbox<?> inbox(UUID queryId, long exchangeId);
+
+    /**
+     * Returns all registered inboxes for provided query ID.
+     *
+     * @param queryId Query ID.
+     * @return Registered inboxes.
+     */
+    List<Inbox<?>> inboxes(UUID queryId);
+
+    /**
+     * Returns all registered outboxes for provided query ID.
+     *
+     * @param queryId Query ID.
+     * @return Registered outboxes.
+     */
+    List<Outbox<?>> outboxes(UUID queryId);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.java
new file mode 100644
index 0000000..cc6e0be
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/MailboxRegistryImpl.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.exec;
+
+import java.util.List;
+import java.util.Map;
+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.util.AbstractService;
+
+/**
+ *
+ */
+public class MailboxRegistryImpl extends AbstractService implements MailboxRegistry {
+    /** */
+    private final Map<MailboxKey, Outbox<?>> locals;
+
+    /** */
+    private final Map<MailboxKey, Inbox<?>> remotes;
+
+    /**
+     * @param ctx Kernal.
+     */
+    public MailboxRegistryImpl(GridKernalContext ctx) {
+        super(ctx);
+
+        locals = new ConcurrentHashMap<>();
+        remotes = new ConcurrentHashMap<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Inbox<?> register(Inbox<?> inbox) {
+        Inbox<?> old = remotes.putIfAbsent(new MailboxKey(inbox.queryId(), inbox.exchangeId()), inbox);
+
+        return old != null ? old : inbox;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(Inbox<?> inbox) {
+        remotes.remove(new MailboxKey(inbox.queryId(), inbox.exchangeId()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(Outbox<?> outbox) {
+        Outbox<?> res = locals.put(new MailboxKey(outbox.queryId(), outbox.exchangeId()), outbox);
+
+        assert res == null : res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unregister(Outbox<?> outbox) {
+        locals.remove(new MailboxKey(outbox.queryId(), outbox.exchangeId()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Outbox<?> outbox(UUID queryId, long exchangeId) {
+        return locals.get(new MailboxKey(queryId, exchangeId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Inbox<?> inbox(UUID queryId, long exchangeId) {
+        return remotes.get(new MailboxKey(queryId, exchangeId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Inbox<?>> inboxes(UUID queryId) {
+        return remotes.entrySet().stream()
+            .filter(e -> e.getKey().queryId.equals(queryId))
+            .map(Map.Entry::getValue)
+            .collect(Collectors.toList());
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Outbox<?>> outboxes(UUID queryId) {
+        return locals.entrySet().stream()
+            .filter(e -> e.getKey().queryId.equals(queryId))
+            .map(Map.Entry::getValue)
+            .collect(Collectors.toList());    }
+
+    /** */
+    private static class MailboxKey {
+        /** */
+        private final UUID queryId;
+
+        /** */
+        private final long exchangeId;
+
+        /** */
+        private MailboxKey(UUID queryId, long exchangeId) {
+            this.queryId = queryId;
+            this.exchangeId = exchangeId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            MailboxKey that = (MailboxKey) o;
+
+            if (exchangeId != that.exchangeId)
+                return false;
+            return queryId.equals(that.queryId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = queryId.hashCode();
+            result = 31 * result + (int) (exchangeId ^ (exchangeId >>> 32));
+            return result;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Node.java
index 83dc92b..f0e9506 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/Node.java
@@ -21,10 +21,20 @@ import java.util.List;
 
 /**
  * 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.
  */
-public interface Node<T> extends Source {
+public interface Node<T> {
+    /**
+     * Returns runtime context allowing access to the tables in a database.
+     *
+     * @return Execution context.
+     */
+    ExecutionContext context();
+
     /**
-     * Requests a target sink of the node. The sink is used to push data into the node by its children.
+     * Requests a sink of the node. The sink is used to push data into the node by its children.
      *
      * @param idx Sink index.
      * @return Sink object.
@@ -33,9 +43,42 @@ public interface Node<T> extends Source {
     Sink<T> sink(int idx);
 
     /**
-     * Registers sources of this node. Sources are used to notify children when the node is ready to consume data.
+     * Registers target sink.
      *
-     * @param sources Sources.
+     * @param sink Target sink.
+     */
+    void target(Sink<T> sink);
+
+    /**
+     * @return Registered target.
+     */
+    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();
+
+    /**
+     * Cancels execution.
+     */
+    void cancel();
+
+    /**
+     * Resets execution sub-tree to initial state.
      */
-    void sources(List<Source> sources);
+    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/Outbox.java
new file mode 100644
index 0000000..3840cb4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Outbox.java
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.trait.Destination;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * A part of exchange.
+ */
+public class Outbox<T> extends AbstractNode<T> implements SingleNode<T>, Sink<T>, AutoCloseable {
+    /** */
+    private final ExchangeService exchange;
+
+    /** */
+    private final long exchangeId;
+
+    /** */
+    private final MailboxRegistry registry;
+
+    /** */
+    private final long targetFragmentId;
+
+    /** */
+    private final Map<UUID, Buffer> perNodeBuffers = new HashMap<>();
+
+    /** */
+    private final Destination destination;
+
+    /** */
+    private boolean cancelled;
+
+    /**
+     * @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 destination Destination.
+     */
+    public Outbox(ExchangeService exchange, MailboxRegistry registry, ExecutionContext ctx, long targetFragmentId, long exchangeId, Node<T> input, Destination destination) {
+        super(ctx, input);
+        this.exchange = exchange;
+        this.registry = registry;
+        this.targetFragmentId = targetFragmentId;
+        this.exchangeId = exchangeId;
+        this.destination = destination;
+
+        link();
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return context().queryId();
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * callback method.
+     *
+     * @param nodeId Target ID.
+     * @param batchId Batch ID.
+     */
+    public void onAcknowledge(UUID nodeId, int batchId) {
+        perNodeBuffers.get(nodeId).onAcknowledge(batchId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request() {
+        checkThread();
+
+        if (context().cancelled())
+            cancelInternal();
+        else
+            input().request();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        checkThread();
+
+        context().setCancelled();
+        cancelInternal();
+    }
+
+    /** */
+    private void cancelInternal() {
+        if (cancelled)
+            return;
+
+        try {
+            perNodeBuffers.values().forEach(Buffer::cancel);
+            input().cancel();
+        }
+        finally {
+            cancelled = true;
+            close();
+        }
+    }
+
+    /** {@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;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean push(T row) {
+        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())
+                return false;
+
+            buffers.add(dest);
+        }
+
+        for (Buffer dest : buffers)
+            dest.add(row);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void end() {
+        for (UUID node : destination.targets())
+            getOrCreateBuffer(node).end();
+
+        close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        registry.unregister(this);
+    }
+
+    /** */
+    private void sendBatch(UUID nodeId, int batchId, List<?> rows) {
+        exchange.sendBatch(this, nodeId, queryId(), targetFragmentId, exchangeId, batchId, rows);
+    }
+
+    /** */
+    private void sendCancel(UUID nodeId, int batchId) {
+        exchange.cancel(this, nodeId, queryId(), targetFragmentId, exchangeId, batchId);
+    }
+
+    /** */
+    private Buffer getOrCreateBuffer(UUID nodeId) {
+        return perNodeBuffers.computeIfAbsent(nodeId, this::createBuffer);
+    }
+
+    /** */
+    private Buffer createBuffer(UUID nodeId) {
+        return new Buffer(nodeId, this);
+    }
+
+    /** */
+    private static final class Buffer {
+        /** */
+        private final Outbox<?> owner;
+
+        /** */
+        private final UUID nodeId;
+
+        /** */
+        private int hwm = -1;
+
+        /** */
+        private int lwm = -1;
+
+        /** */
+        private List<Object> curr;
+
+        /** */
+        private Buffer(UUID nodeId, Outbox<?> owner) {
+            this.nodeId = nodeId;
+            this.owner = owner;
+
+            curr = new ArrayList<>(ExchangeService.BATCH_SIZE + 1); // extra space for end marker;
+        }
+
+        /**
+         * Adds a row to current batch.
+         *
+         * @param row Row.
+         */
+        public void add(Object row) {
+            assert ready();
+
+            if (curr.size() == ExchangeService.BATCH_SIZE) {
+                owner.sendBatch(nodeId, ++hwm, curr);
+                curr = new ArrayList<>(ExchangeService.BATCH_SIZE + 1); // extra space for end marker;
+            }
+
+            curr.add(row);
+        }
+
+        /**
+         * Signals data is over.
+         */
+        public void end() {
+            int batchId = hwm + 1;
+            hwm = Integer.MAX_VALUE;
+
+            List<Object> tmp = curr;
+            curr = null;
+
+            tmp.add(EndMarker.INSTANCE);
+            owner.sendBatch(nodeId, batchId, tmp);
+        }
+
+        public void cancel() {
+            int batchId = hwm + 1;
+            hwm = Integer.MAX_VALUE;
+
+            curr = null;
+            owner.sendCancel(nodeId, batchId);
+        }
+
+        /**
+         * Checks whether there is a place for a new row.
+         *
+         * @return {@code True} is it possible to add a row to a batch.
+         */
+        private boolean ready() {
+            if (curr == null)
+                throw new AssertionError();
+
+            return hwm != Integer.MAX_VALUE
+                && hwm - lwm < ExchangeService.PER_NODE_BATCH_COUNT
+                || curr.size() < ExchangeService.BATCH_SIZE;
+        }
+
+        /**
+         * Callback method.
+         *
+         * @param id batch ID.
+         */
+        private void onAcknowledge(int id) {
+            if (lwm > id)
+                return;
+
+            boolean request = hwm - lwm == ExchangeService.PER_NODE_BATCH_COUNT;
+
+            lwm = id;
+
+            if (request)
+                owner.request();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ProjectNode.java
index 210bea69..796bacd 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/ProjectNode.java
@@ -27,13 +27,15 @@ public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Ob
     private final Function<Object[], Object[]> projection;
 
     /**
-     * @param target Target.
+     * @param ctx Execution context.
      * @param projection Projection.
      */
-    public ProjectNode(Sink<Object[]> target, Function<Object[], Object[]> projection) {
-        super(target);
+    public ProjectNode(ExecutionContext ctx, Node<Object[]> input, Function<Object[], Object[]> projection) {
+        super(ctx, input);
 
         this.projection = projection;
+
+        link();
     }
 
     /** {@inheritDoc} */
@@ -46,11 +48,11 @@ public class ProjectNode extends AbstractNode<Object[]> implements SingleNode<Ob
 
     /** {@inheritDoc} */
     @Override public boolean push(Object[] row) {
-        return target.push(projection.apply(row));
+        return target().push(projection.apply(row));
     }
 
     /** {@inheritDoc} */
     @Override public void end() {
-        target.end();
+        target().end();
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
similarity index 68%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
index dd5226a..1ff87f5 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Source.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
@@ -17,12 +17,19 @@
 
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
+import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+
 /**
- * Represents an execution node source. Sources are used to notify children when an execution node is ready to consume data.
+ *
  */
-public interface Source {
+public interface QueryTaskExecutor extends Service {
     /**
-     * Signals that consumer is ready to consume data. Called by consumer node.
+     * Executes a query task in a thread, responsible for particular query fragment.
+     *
+     * @param queryId Query ID.
+     * @param fragmentId Fragment ID.
+     * @param queryTask Query task.
      */
-    void signal();
+    void execute(UUID queryId, long fragmentId, Runnable queryTask);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
new file mode 100644
index 0000000..c23cee1
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.UUID;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME;
+
+/**
+ * TODO use {@link org.apache.ignite.internal.util.StripedExecutor}, registered in core pols.
+ */
+public class QueryTaskExecutorImpl extends AbstractService implements QueryTaskExecutor {
+    /** */
+    private IgniteStripedThreadPoolExecutor stripedThreadPoolExecutor;
+
+    /** */
+    public QueryTaskExecutorImpl(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /**
+     * @param stripedThreadPoolExecutor Executor.
+     */
+    public void stripedThreadPoolExecutor(IgniteStripedThreadPoolExecutor stripedThreadPoolExecutor) {
+        this.stripedThreadPoolExecutor = stripedThreadPoolExecutor;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(UUID queryId, long fragmentId, Runnable queryTask) {
+        stripedThreadPoolExecutor.execute(queryTask, hash(queryId, fragmentId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        stripedThreadPoolExecutor(new IgniteStripedThreadPoolExecutor(
+            ctx.config().getQueryThreadPoolSize(),
+            ctx.igniteInstanceName(),
+            "calciteQry",
+            ctx.uncaughtExceptionHandler(),
+            true,
+            DFLT_THREAD_KEEP_ALIVE_TIME
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() {
+        U.shutdownNow(getClass(), stripedThreadPoolExecutor, log);
+    }
+
+    /** */
+    private static int hash(UUID queryId, long fragmentId) {
+        // inlined Objects.hash(...)
+        return U.safeAbs(31 * (31 + (queryId != null ? queryId.hashCode() : 0)) + Long.hashCode(fragmentId));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
index e0adce7..d11f29a5 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScalarFactory.java
@@ -19,19 +19,32 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import com.google.common.collect.ImmutableList;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.List;
+import java.util.Map;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
 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.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.
@@ -39,18 +52,43 @@ import org.apache.calcite.rex.RexNode;
  */
 public class ScalarFactory {
     /** */
+    private static final int CACHE_SIZE = 1024;
+
+    /** */
+    private static final Map<String, Scalar> CACHE = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+
+    /** */
+    private final JavaTypeFactory typeFactory;
+
+    /** */
     private final JaninoRexCompiler rexCompiler;
 
     /** */
-    private final RexBuilder builder;
+    private final ExceptionHandler handler;
 
     /**
-     * @param builder RexBuilder.
+     * @param typeFactory Type factory.
+     * @param failure Failure processor.
+     * @param log Logger.
      */
-    public ScalarFactory(RexBuilder builder) {
-        this.builder = builder;
+    public ScalarFactory(JavaTypeFactory typeFactory, FailureProcessor failure, IgniteLogger log) {
+        this.typeFactory = typeFactory;
 
-        rexCompiler = new JaninoRexCompiler(builder);
+        rexCompiler = new JaninoRexCompiler(new RexBuilder(typeFactory));
+        handler = new ExceptionHandler(failure, log.getLogger(ScalarFactory.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(DataContext root, List<RelCollation> collations, RelDataType rowType) {
+        return null; // TODO
     }
 
     /**
@@ -62,10 +100,10 @@ public class ScalarFactory {
      * @return Filter predicate.
      */
     public <T> Predicate<T> filterPredicate(DataContext root, RexNode filter, RelDataType rowType) {
-        Scalar scalar = rexCompiler.compile(ImmutableList.of(filter), rowType);
+        Scalar scalar = scalar(ImmutableList.of(filter), rowType);
         Context ctx = InterpreterUtils.createContext(root);
 
-        return new FilterPredicate<>(ctx, scalar);
+        return new FilterPredicate<>(ctx, scalar, handler);
     }
 
     /**
@@ -78,11 +116,11 @@ public class ScalarFactory {
      * @return Project function.
      */
     public <T> Function<T, T> projectExpression(DataContext root, List<RexNode> projects, RelDataType rowType) {
-        Scalar scalar = rexCompiler.compile(projects, rowType);
+        Scalar scalar = scalar(projects, rowType);
         Context ctx = InterpreterUtils.createContext(root);
         int count = projects.size();
 
-        return new ProjectExpression<>(ctx, scalar, count);
+        return new ProjectExpression<>(ctx, scalar, count, handler);
     }
 
     /**
@@ -96,17 +134,16 @@ public class ScalarFactory {
      */
     public <T> BiFunction<T, T, T> joinExpression(DataContext root, RexNode expression, RelDataType leftType, RelDataType rightType) {
         RelDataType rowType = combinedType(leftType, rightType);
-
-        Scalar scalar = rexCompiler.compile(ImmutableList.of(expression), rowType);
+        Scalar scalar = scalar(ImmutableList.of(expression), rowType);
         Context ctx = InterpreterUtils.createContext(root);
         ctx.values = new Object[rowType.getFieldCount()];
 
-        return new JoinExpression<>(ctx, scalar);
+        return new JoinExpression<>(ctx, scalar, handler);
     }
 
     /** */
     private RelDataType combinedType(RelDataType... types) {
-        RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(typeFactory());
+        RelDataTypeFactory.Builder typeBuilder = new RelDataTypeFactory.Builder(typeFactory);
 
         for (RelDataType type : types)
             typeBuilder.addAll(type.getFieldList());
@@ -115,8 +152,20 @@ public class ScalarFactory {
     }
 
     /** */
-    private RelDataTypeFactory typeFactory() {
-        return builder.getTypeFactory();
+    private Scalar scalar(List<RexNode> nodes, RelDataType type) {
+        assert !F.isEmpty(nodes);
+
+        return CACHE.computeIfAbsent(cacheKey(nodes, type), k -> rexCompiler.compile(nodes, type));
+    }
+
+    /** */
+    private String cacheKey(List<RexNode> nodes, RelDataType type) {
+        StrBuilder b = new StrBuilder("[").append(F.first(nodes));
+        for (int i = 1; i < nodes.size(); i++)
+            b.append(";").append(nodes.get(i));
+        b.append("]:").append(type.getFullTypeString());
+
+        return b.toString();
     }
 
     /** */
@@ -128,24 +177,35 @@ public class ScalarFactory {
         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) {
+        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) {
-            ctx.values = (Object[]) r;
-            scalar.execute(ctx, vals);
-            return (Boolean) vals[0];
+            try {
+                ctx.values = (Object[]) r;
+                scalar.execute(ctx, vals);
+                return (Boolean) vals[0];
+            }
+            catch (Exception e) {
+                handler.onException(e);
+
+                throw e;
+            }
         }
     }
 
@@ -161,35 +221,46 @@ public class ScalarFactory {
         private final Scalar scalar;
 
         /** */
+        private final ExceptionHandler handler;
+
+        /** */
         private Object[] left0;
 
         /**
          * @param ctx Interpreter context.
          * @param scalar Scalar.
          */
-        private JoinExpression(Context ctx, 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) {
-            if (left0 != left) {
-                left0 = (Object[]) left;
-                System.arraycopy(left0, 0, ctx.values, 0, left0.length);
-            }
+            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);
+                Object[] right0 = (Object[]) right;
+                System.arraycopy(right0, 0, ctx.values, left0.length, right0.length);
 
-            scalar.execute(ctx, vals);
+                scalar.execute(ctx, vals);
 
-            if ((Boolean) vals[0])
-                return (T) Arrays.copyOf(ctx.values, ctx.values.length);
+                if ((Boolean) vals[0])
+                    return (T) Arrays.copyOf(ctx.values, ctx.values.length);
 
-            return null;
+                return null;
+            }
+            catch (Exception e) {
+                handler.onException(e);
+
+                throw e;
+            }
         }
     }
 
@@ -204,24 +275,58 @@ public class ScalarFactory {
         /** */
         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) {
+        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) {
-            ctx.values = (Object[]) r;
-            Object[] res = new Object[count];
-            scalar.execute(ctx, res);
+            try {
+                ctx.values = (Object[]) r;
+                Object[] res = new Object[count];
+                scalar.execute(ctx, res);
+
+                return (T) res;
+            }
+            catch (Exception e) {
+                handler.onException(e);
+
+                throw e;
+            }
+        }
+    }
+
+    /** */
+    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);
 
-            return (T) res;
+            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/ScanNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ScanNode.java
index 3fabc97..393d26d 100644
--- 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
@@ -18,18 +18,12 @@
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import java.util.Iterator;
-import java.util.List;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
- * TODO https://issues.apache.org/jira/browse/IGNITE-12449
+ * Scan node.
  */
-public class ScanNode implements SingleNode<Object[]> {
-    /** */
-    private static final Object[] END = new Object[0];
-
-    /** */
-    private final Sink<Object[]> target;
-
+public class ScanNode extends AbstractNode<Object[]> implements SingleNode<Object[]>, AutoCloseable {
     /** */
     private final Iterable<Object[]> source;
 
@@ -37,48 +31,83 @@ public class ScanNode implements SingleNode<Object[]> {
     private Iterator<Object[]> it;
 
     /** */
-    private Object[] row;
+    private Object row;
 
     /**
-     * @param target Target.
+     * @param ctx Execution context.
      * @param source Source.
      */
-    public ScanNode(Sink<Object[]> target, Iterable<Object[]> source) {
-        this.target = target;
+    public ScanNode(ExecutionContext ctx, Iterable<Object[]> source) {
+        super(ctx);
+
         this.source = source;
     }
 
     /** {@inheritDoc} */
-    @Override public void signal() {
-        if (row == END)
-            return;
+    @Override public void request() {
+        try {
+            requestInternal();
+        }
+        catch (Exception e) {
+            Commons.close(it, e);
 
-        if (row != null && !target.push(row))
-            return;
+            throw e;
+        }
+    }
+
+    /** */
+    private void requestInternal() {
+        checkThread();
 
-        row = null;
+        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();
 
-            if (!target.push(row))
+            // TODO load balancing - resubmit this::request() in case of long execution
+
+            if (!target().push((Object[]) row))
                 return;
         }
 
-        row = END;
-        target.end();
+        try {
+            row = EndMarker.INSTANCE;
+
+            target().end();
+        }
+        finally {
+            close();
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void sources(List<Source> sources) {
-        throw new UnsupportedOperationException();
+    @Override public void close() {
+        Commons.close(it);
+
+        it = null;
+
+        if (row != EndMarker.INSTANCE)
+            row = null;
     }
 
     /** {@inheritDoc} */
     @Override public Sink<Object[]> sink(int idx) {
-        throw new UnsupportedOperationException();
+        throw new AssertionError();
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SingleNode.java
index 8dd7e42..cebb0d2 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/SingleNode.java
@@ -17,14 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
-import java.util.Collections;
 import java.util.Objects;
 
 /**
  * A node with a single input
  */
 public interface SingleNode<T> extends Node<T> {
-
     /**
      * @return Single sink object.
      */
@@ -33,10 +31,9 @@ public interface SingleNode<T> extends Node<T> {
     }
 
     /**
-     * Registers a single source.
-     * @param source Source.
+     * @return Single input.
      */
-    default void source(Source source) {
-        sources(Collections.singletonList(source));
+    default Node<T> input() {
+        return Objects.requireNonNull(input(0));
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/Sink.java
index 5023a63..8d09c5f 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/Sink.java
@@ -19,20 +19,11 @@ package org.apache.ignite.internal.processors.query.calcite.exec;
 
 /**
  * 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.
  */
 public interface Sink<T> {
-    /** */
-    @SuppressWarnings("rawtypes")
-    Sink NO_OP = new Sink() {
-        /** {@inheritDoc} */
-        @Override public boolean push(Object row) {
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void end() {}
-    };
-
     /**
      * Pushes a row to consumer.
      * @param row Data row.
@@ -45,11 +36,4 @@ public interface Sink<T> {
      * Signals that data is over.
      */
     void end();
-
-    /**
-     * @return No-op sink object.
-     */
-    static <T> Sink<T> noOp() {
-        return (Sink<T>) NO_OP;
-    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessage.java
similarity index 65%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessage.java
index d4368c7..811f6f9 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exchange/EndMarker.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessage.java
@@ -15,25 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.exchange;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.io.ObjectStreamException;
-import java.io.Serializable;
+import org.apache.ignite.plugin.extensions.communication.Message;
 
 /**
- * Poison pill object.
+ *
  */
-public final class EndMarker implements Serializable {
-    /** */
-    public static final EndMarker INSTANCE = new EndMarker();
-
+public interface CalciteMessage extends Message {
     /**
-     * Default constructor.
+     * @return Message type.
      */
-    private EndMarker(){}
+    MessageType type();
+
+    /** {@inheritDoc} */
+    @Override default short directType() {
+        return type().directType();
+    }
 
-    /** */
-    private Object readResolve() throws ObjectStreamException {
-        return INSTANCE;
+    /** {@inheritDoc} */
+    @Override default void onAckReceived() {
+        // No-op.
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessageFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessageFactory.java
new file mode 100644
index 0000000..258e03c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/CalciteMessageFactory.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageFactory;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Message factory.
+ */
+public class CalciteMessageFactory implements MessageFactory {
+    /** {@inheritDoc} */
+    @Override public @Nullable Message create(short type) {
+        return MessageType.newMessage(type);
+    }
+
+    /**
+     * Produces a row message.
+     *
+     * TODO In future a row is expected to implement Message interface.
+     */
+    public static Message asMessage(Object row) {
+        return new GenericRowMessage(row);
+    }
+
+    /**
+     * Produces a row from a message.
+     *
+     * TODO In future a row is expected to implement Message interface.
+     */
+    public static Object asRow(Message mRow) {
+        if (mRow instanceof GenericRowMessage)
+            return ((GenericRowMessage) mRow).row();
+
+        throw new AssertionError("Unexpected message type. [message=" + mRow + "]");
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAware.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAware.java
index c995ef2..779dc91 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/ExecutionContextAware.java
@@ -15,23 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.util.List;
 import java.util.UUID;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 
 /**
- * Determines where to send a row to.
+ * Execution context is used to determine a stripe where to process a message.
  */
-public interface DestinationFunction {
+public interface ExecutionContextAware extends CalciteMessage {
     /**
-     * @param row Row.
-     * @return Target nodes list.
+     * @return Query ID.
      */
-    List<UUID> destination(Object row);
+    UUID queryId();
 
     /**
-     * @return All target nodes.
+     * @return Fragment ID.
      */
-    List<UUID> targets();
+    long fragmentId();
+
+    /**
+     * @return IO policy.
+     */
+    default byte ioPolicy() {
+        return GridIoPolicy.CALLER_THREAD;
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/GenericRowMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/GenericRowMessage.java
new file mode 100644
index 0000000..d9e7590
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/GenericRowMessage.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class GenericRowMessage implements MarshalableMessage {
+    /** */
+    @GridDirectTransient
+    private Object row;
+
+    /** */
+    private byte[] serRow;
+
+    /** */
+    public GenericRowMessage() {
+
+    }
+
+    /** */
+    public GenericRowMessage(Object row) {
+        this.row = row;
+    }
+
+    /** */
+    public Object row() {
+        return row;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+        if (row != null && serRow == null)
+            serRow = marshaller.marshal(row);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+        if (serRow != null && row == null)
+            row = marshaller.unmarshal(serRow, loader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("serRow", serRow))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                serRow = reader.readByteArray("serRow");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GenericRowMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.GENERIC_ROW_MESSAGE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCancelMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCancelMessage.java
new file mode 100644
index 0000000..9368052
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/InboxCancelMessage.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class InboxCancelMessage implements ExecutionContextAware {
+    /** */
+    private UUID queryId;
+
+    /** */
+    private long fragmentId;
+
+    /** */
+    private long exchangeId;
+
+    /** */
+    private int batchId;
+
+    /** */
+    public InboxCancelMessage(){}
+
+    /** */
+    public InboxCancelMessage(UUID queryId, long fragmentId, long exchangeId, int batchId) {
+        this.queryId = queryId;
+        this.fragmentId = fragmentId;
+        this.exchangeId = exchangeId;
+        this.batchId = batchId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID queryId() {
+        return queryId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long fragmentId() {
+        return fragmentId;
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * @return Batch ID.
+     */
+    public int batchId() {
+        return batchId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeInt("batchId", batchId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("exchangeId", exchangeId))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeLong("fragmentId", fragmentId))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeUuid("queryId", queryId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                batchId = reader.readInt("batchId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                exchangeId = reader.readLong("exchangeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                fragmentId = reader.readLong("fragmentId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                queryId = reader.readUuid("queryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(InboxCancelMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.QUERY_INBOX_CANCEL_MESSAGE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MarshalableMessage.java
similarity index 57%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MarshalableMessage.java
index c995ef2..b555157 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MarshalableMessage.java
@@ -15,23 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.util.List;
-import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.marshaller.Marshaller;
 
 /**
- * Determines where to send a row to.
+ *
  */
-public interface DestinationFunction {
+public interface MarshalableMessage extends CalciteMessage {
     /**
-     * @param row Row.
-     * @return Target nodes list.
+     * Prepares the message before sending.
+     *
+     * @param marshaller Marchaller.
      */
-    List<UUID> destination(Object row);
+    void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException;
 
     /**
-     * @return All target nodes.
+     * Prepares the message before processing.
+     *
+     * @param marshaller Marchaller.
+     * @param loader Class loader.
      */
-    List<UUID> targets();
+    void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException;
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
similarity index 75%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
index 112aad9..61fc897 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageListener.java
@@ -15,12 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.io.Serializable;
+import java.util.UUID;
 
 /**
- * Graph node.
+ *
  */
-public interface GraphNode extends Serializable {
+public interface MessageListener {
+    /**
+     * @param nodeId Sender node ID.
+     * @param msg Message.
+     */
+    void onMessage(UUID nodeId, CalciteMessage msg);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
similarity index 53%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
index c995ef2..02bbe46 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/trait/DestinationFunction.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageService.java
@@ -15,23 +15,37 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.trait;
+package org.apache.ignite.internal.processors.query.calcite.message;
 
-import java.util.List;
+import java.util.Collection;
 import java.util.UUID;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
- * Determines where to send a row to.
+ *
  */
-public interface DestinationFunction {
+public interface MessageService extends Service {
+    /**
+     * Sends a message to given nodes.
+     *
+     * @param nodeIds Nodes IDs.
+     * @param msg Message.
+     */
+    void send(Collection<UUID> nodeIds, CalciteMessage msg);
+
     /**
-     * @param row Row.
-     * @return Target nodes list.
+     * Sends a message to given node.
+     *
+     * @param nodeId Node ID.
+     * @param msg Message.
      */
-    List<UUID> destination(Object row);
+    void send(UUID nodeId, CalciteMessage msg);
 
     /**
-     * @return All target nodes.
+     * Registers a listener for messages of a given type.
+     *
+     * @param lsnr Listener.
+     * @param type Message type.
      */
-    List<UUID> targets();
+    void register(MessageListener lsnr, MessageType type);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java
new file mode 100644
index 0000000..c6e3c2b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageServiceImpl.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.GridTopic;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.managers.communication.GridMessageListener;
+import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.exec.QueryTaskExecutor;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.marshaller.MarshallerUtils;
+import org.apache.ignite.plugin.extensions.communication.Message;
+
+/**
+ *
+ */
+public class MessageServiceImpl extends AbstractService implements MessageService {
+    /** */
+    private final GridMessageListener msgLsnr;
+
+    private UUID localNodeId;
+
+    private GridIoManager ioManager;
+
+    private ClassLoader classLoader;
+
+    private QueryTaskExecutor taskExecutor;
+
+    private FailureProcessor failureProcessor;
+
+    private Marshaller marsh;
+
+    /** */
+    private EnumMap<MessageType, MessageListener> lsnrs;
+
+    /** */
+    public MessageServiceImpl(GridKernalContext ctx) {
+        super(ctx);
+
+        msgLsnr = this::onMessage;
+    }
+
+    /**
+     * @param localNodeId Local node ID.
+     */
+    public void localNodeId(UUID localNodeId) {
+        this.localNodeId = localNodeId;
+    }
+
+    /**
+     * @return Local node ID.
+     */
+    public UUID localNodeId() {
+        return localNodeId;
+    }
+
+    /**
+     * @param ioManager IO manager.
+     */
+    public void ioManager(GridIoManager ioManager) {
+        this.ioManager = ioManager;
+    }
+
+    /**
+     * @return IO manager.
+     */
+    public GridIoManager ioManager() {
+        return ioManager;
+    }
+
+    /**
+     * @param classLoader Class loader.
+     */
+    public void classLoader(ClassLoader classLoader) {
+        this.classLoader = classLoader;
+    }
+
+    /**
+     * @return Class loader.
+     */
+    public ClassLoader classLoader() {
+        return classLoader;
+    }
+
+    /**
+     * @param taskExecutor Task executor.
+     */
+    public void taskExecutor(QueryTaskExecutor taskExecutor) {
+        this.taskExecutor = taskExecutor;
+    }
+
+    /**
+     * @return Task executor.
+     */
+    public QueryTaskExecutor taskExecutor() {
+        return taskExecutor;
+    }
+
+    /**
+     * @param marsh Marshaller.
+     */
+    public void marshaller(Marshaller marsh) {
+        this.marsh = marsh;
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    public Marshaller marshaller() {
+        return marsh;
+    }
+
+    /**
+     * @param failureProcessor Failure processor.
+     */
+    public void failureProcessor(FailureProcessor failureProcessor) {
+        this.failureProcessor = failureProcessor;
+    }
+
+    /**
+     * @return Failure processor.
+     */
+    public FailureProcessor failureProcessor() {
+        return failureProcessor;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        localNodeId(ctx.localNodeId());
+        classLoader(U.resolveClassLoader(ctx.config()));
+        ioManager(ctx.io());
+
+        @SuppressWarnings("deprecation")
+        Marshaller marsh0 = ctx.config().getMarshaller();
+
+        if (marsh0 == null) // Stubbed context doesn't have a marshaller
+            marsh0 = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName());
+
+        marshaller(marsh0);
+
+        CalciteQueryProcessor proc = Objects.requireNonNull(Commons.lookupComponent(ctx, CalciteQueryProcessor.class));
+
+        taskExecutor(proc.taskExecutor());
+        failureProcessor(proc.failureProcessor());
+
+        init();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init() {
+        ioManager().addMessageListener(GridTopic.TOPIC_QUERY, msgLsnr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() {
+        ioManager().removeMessageListener(GridTopic.TOPIC_QUERY, msgLsnr);
+        lsnrs = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void send(Collection<UUID> nodeIds, CalciteMessage msg) {
+        for (UUID nodeId : nodeIds)
+            send(nodeId, msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void send(UUID nodeId, CalciteMessage msg) {
+        if (localNodeId().equals(nodeId))
+            onMessage(nodeId, msg, true);
+        else {
+            byte plc = msg instanceof ExecutionContextAware ?
+                ((ExecutionContextAware) msg).ioPolicy() : GridIoPolicy.QUERY_POOL;
+
+            if (!prepareMarshal(msg))
+                return;
+
+            try {
+                ioManager().sendToGridTopic(nodeId, GridTopic.TOPIC_QUERY, msg, plc);
+            }
+            catch (ClusterTopologyCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send message, node failed: " + nodeId);
+            }
+            catch (IgniteCheckedException e) {
+                failureProcessor().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(MessageListener lsnr, MessageType type) {
+        if (lsnrs == null)
+            lsnrs = new EnumMap<>(MessageType.class);
+
+        MessageListener old = lsnrs.put(type, lsnr);
+
+        assert old == null : old;
+    }
+
+    /** */
+    protected void onMessage(UUID nodeId, CalciteMessage msg, boolean async) {
+        if (msg instanceof ExecutionContextAware) {
+            ExecutionContextAware msg0 = (ExecutionContextAware) msg;
+            taskExecutor().execute(msg0.queryId(), msg0.fragmentId(), () -> onMessageInternal(nodeId, msg));
+        }
+        else if (async)
+            taskExecutor().execute(IgniteUuid.VM_ID, ThreadLocalRandom.current().nextLong(1024), () -> onMessageInternal(nodeId, msg));
+        else
+            onMessageInternal(nodeId, msg);
+    }
+
+    /** */
+    protected boolean prepareMarshal(Message msg) {
+        try {
+            if (msg instanceof MarshalableMessage)
+                ((MarshalableMessage) msg).prepareMarshal(marshaller());
+
+            return true;
+        }
+        catch (IgniteCheckedException e) {
+            failureProcessor().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+        }
+
+        return false;
+    }
+
+    /** */
+    protected boolean prepareUnmarshal(Message msg) {
+        try {
+            if (msg instanceof MarshalableMessage)
+                ((MarshalableMessage) msg).prepareUnmarshal(marshaller(), classLoader());
+
+            return true;
+        }
+        catch (IgniteCheckedException e) {
+            failureProcessor().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+        }
+
+        return false;
+    }
+
+    /** */
+    private void onMessage(UUID nodeId, Object msg, byte plc) {
+        if (msg instanceof CalciteMessage)
+            onMessage(nodeId, (CalciteMessage) msg, false);
+    }
+
+    /** */
+    private void onMessageInternal(UUID nodeId, CalciteMessage msg) {
+        if (!prepareUnmarshal(msg))
+            return;
+
+        MessageListener lsnr = Objects.requireNonNull(lsnrs.get(msg.type()));
+        lsnr.onMessage(nodeId, msg);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java
new file mode 100644
index 0000000..24ee972
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.util.function.Supplier;
+
+/**
+ *
+ */
+public enum MessageType {
+    QUERY_START_REQUEST(300, QueryStartRequest::new),
+    QUERY_START_RESPONSE(301, QueryStartResponse::new),
+    QUERY_CANCEL_REQUEST(302, QueryCancelRequest::new),
+    QUERY_BATCH_MESSAGE(303, QueryBatchMessage::new),
+    QUERY_ACKNOWLEDGE_MESSAGE(304, QueryBatchAcknowledgeMessage::new),
+    QUERY_INBOX_CANCEL_MESSAGE(305, InboxCancelMessage::new),
+    GENERIC_ROW_MESSAGE(306, GenericRowMessage::new);
+
+    /** */
+    private final int directType;
+
+    /** */
+    private final Supplier<CalciteMessage> factory;
+
+    /**
+     * @param directType Message direct type.
+     */
+    MessageType(int directType, Supplier<CalciteMessage> factory) {
+        this.directType = directType;
+        this.factory = factory;
+    }
+
+    /**
+     * @return Message direct type;
+     */
+    public short directType() {
+        return (short) directType;
+    }
+
+    /** */
+    private CalciteMessage newMessage() {
+        CalciteMessage msg = factory.get();
+
+        assert msg.type() == this;
+
+        return msg;
+    }
+
+    /**
+     * Message factory method.
+     *
+     * @param directType Message direct type.
+     * @return new message or {@code null} in case of unknown message direct type.
+     */
+    public static CalciteMessage newMessage(short directType) {
+        switch (directType) {
+            case 300:
+                return QUERY_START_REQUEST.newMessage();
+            case 301:
+                return QUERY_START_RESPONSE.newMessage();
+            case 302:
+                return QUERY_CANCEL_REQUEST.newMessage();
+            case 303:
+                return QUERY_BATCH_MESSAGE.newMessage();
+            case 304:
+                return QUERY_ACKNOWLEDGE_MESSAGE.newMessage();
+            case 305:
+                return QUERY_INBOX_CANCEL_MESSAGE.newMessage();
+            case 306:
+                return GENERIC_ROW_MESSAGE.newMessage();
+            default:
+                return null;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
new file mode 100644
index 0000000..7258272
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchAcknowledgeMessage.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryBatchAcknowledgeMessage implements ExecutionContextAware {
+    /** */
+    private UUID queryId;
+
+    /** */
+    private long fragmentId;
+
+    /** */
+    private long exchangeId;
+
+    /** */
+    private int batchId;
+
+    /** */
+    public QueryBatchAcknowledgeMessage() {
+
+    }
+
+    /** */
+    public QueryBatchAcknowledgeMessage(UUID queryId, long fragmentId, long exchangeId, int batchId) {
+        this.queryId = queryId;
+        this.fragmentId = fragmentId;
+        this.exchangeId = exchangeId;
+        this.batchId = batchId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID queryId() {
+        return queryId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long fragmentId() {
+        return fragmentId;
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * @return Batch ID.
+     */
+    public int batchId() {
+        return batchId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeInt("batchId", batchId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("exchangeId", exchangeId))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeLong("fragmentId", fragmentId))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeUuid("queryId", queryId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                batchId = reader.readInt("batchId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                exchangeId = reader.readLong("exchangeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                fragmentId = reader.readLong("fragmentId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                queryId = reader.readUuid("queryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(QueryBatchAcknowledgeMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.QUERY_ACKNOWLEDGE_MESSAGE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java
new file mode 100644
index 0000000..bc15b80
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryBatchMessage.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryBatchMessage implements MarshalableMessage, ExecutionContextAware {
+    /** */
+    private UUID queryId;
+
+    /** */
+    private long fragmentId;
+
+    /** */
+    private long exchangeId;
+
+    /** */
+    private int batchId;
+
+    /** */
+    @GridDirectTransient
+    private List<Object> rows;
+
+    /** */
+    @GridDirectCollection(Message.class)
+    private List<Message> mRows;
+
+    /** */
+    public QueryBatchMessage() {
+
+    }
+
+    /** */
+    public QueryBatchMessage(UUID queryId, long fragmentId, long exchangeId, int batchId, List<?> rows) {
+        this.queryId = queryId;
+        this.fragmentId = fragmentId;
+        this.exchangeId = exchangeId;
+        this.batchId = batchId;
+        this.rows = Commons.cast(rows);
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID queryId() {
+        return queryId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long fragmentId() {
+        return fragmentId;
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public long exchangeId() {
+        return exchangeId;
+    }
+
+    /**
+     * @return Batch ID.
+     */
+    public int batchId() {
+        return batchId;
+    }
+
+    /**
+     * @return Rows.
+     */
+    public List<Object> rows() {
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+        if (mRows != null || rows == null)
+            return;
+
+        mRows = new ArrayList<>(rows.size());
+
+        for (Object row : rows) {
+            Message mRow = CalciteMessageFactory.asMessage(row);
+
+            if (mRow instanceof MarshalableMessage)
+                ((MarshalableMessage) mRow).prepareMarshal(marshaller);
+
+            mRows.add(mRow);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+        if (rows != null || mRows == null)
+            return;
+
+        rows = new ArrayList<>(mRows.size());
+
+        for (Message mRow : mRows) {
+            if (mRow instanceof MarshalableMessage)
+                ((MarshalableMessage) mRow).prepareUnmarshal(marshaller, loader);
+
+            Object row = CalciteMessageFactory.asRow(mRow);
+
+            rows.add(row);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeInt("batchId", batchId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("exchangeId", exchangeId))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeLong("fragmentId", fragmentId))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeCollection("mRows", mRows, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeUuid("queryId", queryId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                batchId = reader.readInt("batchId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                exchangeId = reader.readLong("exchangeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                fragmentId = reader.readLong("fragmentId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                mRows = reader.readCollection("mRows", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                queryId = reader.readUuid("queryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(QueryBatchMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.QUERY_BATCH_MESSAGE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 5;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryCancelRequest.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryCancelRequest.java
new file mode 100644
index 0000000..fe0e2ac
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryCancelRequest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryCancelRequest implements CalciteMessage {
+    /** */
+    private UUID queryId;
+
+    /** */
+    QueryCancelRequest(){}
+
+    /** */
+    public QueryCancelRequest(UUID queryId) {
+        this.queryId = queryId;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return queryId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeUuid("queryId", queryId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                queryId = reader.readUuid("queryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(QueryCancelRequest.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.QUERY_CANCEL_REQUEST;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 1;
+    }
+}
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
new file mode 100644
index 0000000..c5c386d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+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.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryStartRequest implements MarshalableMessage {
+    /** */
+    private String schema;
+
+    /** */
+    private UUID queryId;
+
+    /** */
+    private long fragmentId;
+
+    /** */
+    private int[] partitions;
+
+    /** */
+    private AffinityTopologyVersion version;
+
+    /** */
+    @GridDirectTransient
+    private RelGraph plan;
+
+    /** */
+    private byte[] planBytes;
+
+    /** */
+    @GridDirectTransient
+    private Object[] params;
+
+    /** */
+    private byte[] paramsBytes;
+
+    /** */
+    public QueryStartRequest(UUID queryId, long fragmentId, String schema, RelGraph plan, 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.params = params;
+    }
+
+    /** */
+    QueryStartRequest() {
+
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return queryId;
+    }
+
+    /**
+     * @return Fragment ID.
+     */
+    public long fragmentId() {
+        return fragmentId;
+    }
+
+    /**
+     * @return Partitions.
+     */
+    public int[] partitions() {
+        return partitions;
+    }
+
+    /**
+     * @return Topology version.
+     */
+    public AffinityTopologyVersion topologyVersion() {
+        return version;
+    }
+
+    /**
+     * @return Fragment plan.
+     */
+    public RelGraph plan() {
+        return plan;
+    }
+
+    /**
+     * @return Query parameters.
+     */
+    public Object[] parameters() {
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+        if (planBytes == null && plan != null)
+            planBytes = marshaller.marshal(plan);
+
+        if (paramsBytes == null && params != null)
+            paramsBytes = marshaller.marshal(params);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+        if (plan == null && planBytes != null)
+            plan = marshaller.unmarshal(planBytes, loader);
+
+        if (params == null && paramsBytes != null)
+            params = marshaller.unmarshal(paramsBytes, loader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeLong("fragmentId", fragmentId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeByteArray("paramsBytes", paramsBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeIntArray("partitions", partitions))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeByteArray("planBytes", planBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeUuid("queryId", queryId))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeString("schema", schema))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeAffinityTopologyVersion("version", version))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                fragmentId = reader.readLong("fragmentId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                paramsBytes = reader.readByteArray("paramsBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                partitions = reader.readIntArray("partitions");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                planBytes = reader.readByteArray("planBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                queryId = reader.readUuid("queryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                schema = reader.readString("schema");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
+                version = reader.readAffinityTopologyVersion("version");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(QueryStartRequest.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.QUERY_START_REQUEST;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 7;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java
new file mode 100644
index 0000000..05140b5
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartResponse.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.message;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ *
+ */
+public class QueryStartResponse implements MarshalableMessage {
+    /** */
+    private UUID queryId;
+
+    /** */
+    private long fragmentId;
+
+    /** */
+    @GridDirectTransient
+    private Throwable error;
+
+    /** */
+    private byte[] errBytes;
+
+    /** */
+    public QueryStartResponse() {}
+
+    /** */
+    public QueryStartResponse(UUID queryId, long fragmentId) {
+        this(queryId, fragmentId, null);
+    }
+
+    /** */
+    public QueryStartResponse(UUID queryId, long fragmentId, Throwable error) {
+        this.queryId = queryId;
+        this.fragmentId = fragmentId;
+        this.error = error;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return queryId;
+    }
+
+    /**
+     * @return Fragment ID.
+     */
+    public long fragmentId() {
+        return fragmentId;
+    }
+
+    /**
+     * @return Error.
+     */
+    public Throwable error() {
+        return error;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(Marshaller marshaller) throws IgniteCheckedException {
+        if (error != null)
+            errBytes = marshaller.marshal(error);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareUnmarshal(Marshaller marshaller, ClassLoader loader) throws IgniteCheckedException {
+        if (errBytes != null)
+            error = marshaller.unmarshal(errBytes, loader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("errBytes", errBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("fragmentId", fragmentId))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeUuid("queryId", queryId))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                errBytes = reader.readByteArray("errBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                fragmentId = reader.readLong("fragmentId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                queryId = reader.readUuid("queryId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(QueryStartResponse.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MessageType type() {
+        return MessageType.QUERY_START_RESPONSE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 3;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
index be4c0e8..b3c5720 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/FragmentInfo.java
@@ -71,6 +71,13 @@ public class FragmentInfo {
     }
 
     /**
+     * @return {@code True} if the fragment has nodes mapping.
+     */
+    public boolean mapped() {
+        return mapping != null;
+    }
+
+    /**
      * @return Pairs of underlying Receiver relational nodes and theirs data source information.
      */
     public ImmutableList<Pair<IgniteReceiver, RelSource>> sources() {
@@ -99,9 +106,8 @@ public class FragmentInfo {
         if (mapping != null) {
             NodesMapping newMapping = mapping.prune(filter);
 
-            if (newMapping != mapping) {
+            if (newMapping != mapping)
                 return new FragmentInfo(sources, newMapping);
-            }
         }
 
         return this;
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
index 8271fc0..1c4c940 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdDistribution.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.query.calcite.metadata;
 import java.util.List;
 import org.apache.calcite.plan.hep.HepRelVertex;
 import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -105,7 +106,11 @@ public class IgniteMdDistribution implements MetadataHandler<BuiltInMetadata.Dis
      * See {@link IgniteMdDistribution#distribution(RelNode, RelMetadataQuery)}
      */
     public IgniteDistribution distribution(LogicalTableScan rel, RelMetadataQuery mq) {
-        return rel.getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
+        RelDistribution distr = rel.getTable().getDistribution();
+
+        assert distr instanceof IgniteDistribution : distr;
+
+        return (IgniteDistribution) distr;
     }
 
     /**
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
index e2ae675..7b82c33 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdFragmentInfo.java
@@ -34,13 +34,13 @@ import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetada
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteFilter;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
-import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.schema.DistributedTable;
 import org.apache.ignite.internal.processors.query.calcite.splitter.Edge;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
 
 /**
- * Implementation class for {@link RelMetadataQueryEx#getFragmentLocation(RelNode)} method call.
+ * Implementation class for {@link RelMetadataQueryEx#getFragmentInfo(RelNode)} method call.
  */
 public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
     /**
@@ -136,7 +136,7 @@ public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
      * See {@link IgniteMdFragmentInfo#getFragmentInfo(RelNode, RelMetadataQuery)}
      */
     public FragmentInfo getFragmentInfo(IgniteTableScan rel, RelMetadataQuery mq) {
-        return rel.getTable().unwrap(IgniteTable.class).fragmentInfo(Commons.plannerContext(rel));
+        return new FragmentInfo(rel.getTable().unwrap(DistributedTable.class).mapping(Commons.context(rel)));
     }
 
     /**
@@ -146,7 +146,7 @@ public class IgniteMdFragmentInfo implements MetadataHandler<FragmentMetadata> {
      * @return Fragment meta information.
      */
     public static FragmentInfo fragmentInfo(RelNode rel, RelMetadataQuery mq) {
-        return RelMetadataQueryEx.wrap(mq).getFragmentLocation(rel);
+        return RelMetadataQueryEx.wrap(mq).getFragmentInfo(rel);
     }
 
     /** */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
index c46f733..ba44d60 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingService.java
@@ -17,30 +17,25 @@
 
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
+import java.util.function.Predicate;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Service is responsible for nodes mapping calculation.
  */
-public interface MappingService {
-    /**
-     * @return Local node mapping that consists of local node only, uses for root query fragment.
-     */
-    NodesMapping local();
-
+public interface MappingService extends Service {
     /**
      * Returns Nodes mapping for intermediate fragments, without Scan nodes leafs. Such fragments may be executed
-     * on any cluster node, actual list of nodes is chosen on the basis of adopted selection strategy.
+     * on any cluster node, actual list of nodes is chosen on the basis of adopted selection strategy (using node filter).
      *
      * @param topVer Topology version.
+     * @param desiredCnt desired nodes count, {@code 0} means all possible nodes.
+     * @param nodeFilter Node filter.
      * @return Nodes mapping for intermediate fragments.
      */
-    NodesMapping random(AffinityTopologyVersion topVer);
-
-    /**
-     * @param cacheId Cache ID.
-     * @param topVer Topology version.
-     * @return Nodes mapping for particular table, depends on underlying cache distribution.
-     */
-    NodesMapping distributed(int cacheId, AffinityTopologyVersion topVer);
+    NodesMapping mapBalanced(@NotNull AffinityTopologyVersion topVer, int desiredCnt, @Nullable Predicate<ClusterNode> nodeFilter);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java
new file mode 100644
index 0000000..f790c63
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/MappingServiceImpl.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.metadata;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping.DEDUPLICATED;
+
+/**
+ *
+ */
+public class MappingServiceImpl extends AbstractService implements MappingService {
+    /** */
+    private GridDiscoveryManager discoveryManager;
+
+    /**
+     * @param ctx Kernal.
+     */
+    public MappingServiceImpl(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /**
+     * @param discoveryManager Discovery manager.
+     */
+    public void discoveryManager(GridDiscoveryManager discoveryManager) {
+        this.discoveryManager = discoveryManager;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        discoveryManager(ctx.discovery());
+    }
+
+    /** {@inheritDoc} */
+    @Override public NodesMapping mapBalanced(@NotNull AffinityTopologyVersion topVer, int desiredCnt, @Nullable Predicate<ClusterNode> nodeFilter) {
+        assert desiredCnt >= 0;
+
+        List<ClusterNode> nodes = discoveryManager.discoCache(topVer).serverNodes();
+
+        if (nodeFilter != null)
+            nodes = nodes.stream().filter(nodeFilter).collect(Collectors.toList());
+
+        if (desiredCnt != 0 && desiredCnt < nodes.size()) {
+            Collections.shuffle(nodes);
+
+            nodes = nodes.subList(0, desiredCnt);
+        }
+
+        return new NodesMapping(Commons.transform(nodes, ClusterNode::id), null, DEDUPLICATED);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
index 1feecfb..cd28a28 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/NodesMapping.java
@@ -53,6 +53,9 @@ public class NodesMapping implements Serializable {
     public static final byte DEDUPLICATED           = 1 << 4;
 
     /** */
+    public static final byte CLIENT                 = 1 << 5;
+
+    /** */
     private final List<UUID> nodes;
 
     /** */
@@ -102,7 +105,7 @@ public class NodesMapping implements Serializable {
     public NodesMapping mergeWith(NodesMapping other) throws LocationMappingException {
         byte flags = (byte) (this.flags | other.flags);
 
-        if ((flags & PARTIALLY_REPLICATED) == 0)
+        if ((flags & (PARTIALLY_REPLICATED | CLIENT)) == 0)
             return new NodesMapping(U.firstNotNull(nodes, other.nodes), mergeAssignments(other, null), flags);
 
         List<UUID> nodes;
@@ -216,7 +219,9 @@ public class NodesMapping implements Serializable {
 
     /** */
     private List<List<UUID>> mergeAssignments(NodesMapping other, List<UUID> nodes) throws LocationMappingException {
-        byte flags = (byte) (this.flags | other.flags); List<List<UUID>> left = assignments, right = other.assignments;
+        byte flags = (byte) (this.flags | other.flags);
+        List<List<UUID>> left = assignments;
+        List<List<UUID>> right = other.assignments;
 
         if (left == null && right == null)
             return null; // nothing to intersect;
@@ -224,7 +229,7 @@ public class NodesMapping implements Serializable {
         if (left == null || right == null || (flags & HAS_MOVING_PARTITIONS) == 0) {
             List<List<UUID>> assignments = U.firstNotNull(left, right);
 
-            if (nodes == null || (flags & PARTIALLY_REPLICATED) == 0)
+            if (nodes == null)
                 return assignments;
 
             List<List<UUID>> assignments0 = new ArrayList<>(assignments.size());
@@ -238,7 +243,7 @@ public class NodesMapping implements Serializable {
                         partNodes0.add(partNode);
                 }
 
-                if (partNodes0.isEmpty())
+                if (partNodes0.isEmpty()) // TODO check with partition filters
                     throw new LocationMappingException("Failed to map fragment to location.");
 
                 assignments0.add(partNodes0);
@@ -251,7 +256,8 @@ public class NodesMapping implements Serializable {
         HashSet<UUID> nodesSet = nodes != null ? new HashSet<>(nodes) : null;
 
         for (int i = 0; i < left.size(); i++) {
-            List<UUID> leftNodes = left.get(i), partNodes = new ArrayList<>(leftNodes.size());
+            List<UUID> leftNodes = left.get(i);
+            List<UUID> partNodes = new ArrayList<>(leftNodes.size());
             HashSet<UUID> rightNodesSet = new HashSet<>(right.get(i));
 
             for (UUID partNode : leftNodes) {
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
index 9744264..fd4c33b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/OptimisticPlanningException.java
@@ -38,6 +38,16 @@ public class OptimisticPlanningException extends RuntimeException{
     }
 
     /**
+     *
+     * @param message Message.
+     * @param edge Edge of query plan, where the exception was thrown.
+     */
+    public OptimisticPlanningException(String message, Edge edge) {
+        super(message);
+        this.edge = edge;
+    }
+
+    /**
      * @return Edge of query plan, where the exception was thrown.
      */
     public Edge edge() {
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionService.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionService.java
index 4079743..54638d1 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionService.java
@@ -15,19 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.prepare;
+package org.apache.ignite.internal.processors.query.calcite.metadata;
 
-import java.util.List;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
- * Operation object, encapsulates query execution logic.
+ *
  */
-public interface QueryExecution {
+public interface PartitionService extends Service {
     /**
-     * Executes a query.
+     * Creates a partition mapping function on the basis of affinity function of cache with given ID.
      *
-     * @return Query cursor.
+     * @param cacheId Cache ID.
+     * @return Partition mapping function.
      */
-    FieldsQueryCursor<List<?>> execute();
+    ToIntFunction<Object> partitionFunction(int cacheId);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionServiceImpl.java
new file mode 100644
index 0000000..2b8e35d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/PartitionServiceImpl.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.metadata;
+
+import java.util.function.ToIntFunction;
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+
+/**
+ *
+ */
+public class PartitionServiceImpl extends AbstractService implements PartitionService {
+    /** */
+    private GridCacheSharedContext<?,?> cacheSharedContext;
+
+    /**
+     * @param ctx Kernal.
+     */
+    public PartitionServiceImpl(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /**
+     * @param cacheSharedContext Cache shared context.
+     */
+    public void cacheSharedContext(GridCacheSharedContext<?,?> cacheSharedContext) {
+        this.cacheSharedContext = cacheSharedContext;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        cacheSharedContext(ctx.cache().context());
+    }
+
+    /** {@inheritDoc} */
+    @Override public ToIntFunction<Object> partitionFunction(int cacheId) {
+        if (cacheId == CU.UNDEFINED_CACHE_ID)
+            return k -> k == null ? 0 : k.hashCode();
+
+        AffinityFunction affinity = cacheSharedContext.cacheContext(cacheId).group().affinityFunction();
+
+        return affinity::partition;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
index f63753a..28be9ef 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/RelMetadataQueryEx.java
@@ -118,7 +118,7 @@ public class RelMetadataQueryEx extends RelMetadataQuery {
      * @param rel Relational node.
      * @return Fragment meta information.
      */
-    public FragmentInfo getFragmentLocation(RelNode rel) {
+    public FragmentInfo getFragmentInfo(RelNode rel) {
         for (;;) {
             try {
                 return sourceDistributionHandler.getFragmentInfo(rel, this);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
new file mode 100644
index 0000000..a1c5bf9
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/CacheKey.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.prepare;
+
+import java.util.Objects;
+
+/**
+ *
+ */
+public class CacheKey {
+    /** */
+    private final String schemaName;
+
+    /** */
+    private final String query;
+
+    /** */
+    private final Object contextKey;
+
+    /**
+     * @param schemaName Schema name.
+     * @param query Query string.
+     * @param contextKey Optional context key to differ queries with and without/different flags, having an impact
+     *                   on result plan (like LOCAL flag)
+     */
+    public CacheKey(String schemaName, String query, Object contextKey) {
+        this.schemaName = schemaName;
+        this.query = query;
+        this.contextKey = contextKey;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param query Query string.
+     */
+    public CacheKey(String schemaName, String query) {
+        this(schemaName, query, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        CacheKey cacheKey = (CacheKey) o;
+
+        if (!schemaName.equals(cacheKey.schemaName))
+            return false;
+        if (!query.equals(cacheKey.query))
+            return false;
+        return Objects.equals(contextKey, cacheKey.contextKey);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName.hashCode();
+        result = 31 * result + query.hashCode();
+        result = 31 * result + (contextKey != null ? contextKey.hashCode() : 0);
+        return result;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
deleted file mode 100644
index eb1c36a..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ContextValue.java
+++ /dev/null
@@ -1,59 +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.prepare;
-
-import org.apache.calcite.DataContext;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-
-/**
- * Helpers to get named parameters from {@link DataContext}
- */
-public enum ContextValue {
-    QUERY_ID("_query_id", GridCacheVersion.class),
-    PLANNER_CONTEXT("_planner_context", PlannerContext.class);
-
-    /** */
-    private final String valueName;
-
-    /** */
-    private final Class<?> type;
-
-    /**
-     * @param valueName Value name.
-     * @param type value type.
-     */
-    ContextValue(String valueName, Class<?> type) {
-        this.valueName = valueName;
-        this.type = type;
-    }
-
-    /**
-     * @return valueName.
-     */
-    public String valueName() {
-        return valueName;
-    }
-
-    /**
-     * @param ctx Data context.
-     * @return Parameter value.
-     */
-    public <T> T get(DataContext ctx) {
-        return (T) type.cast(ctx.get(valueName));
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
deleted file mode 100644
index 62b4be5..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DataContextImpl.java
+++ /dev/null
@@ -1,67 +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.prepare;
-
-import java.util.Map;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.schema.SchemaPlus;
-
-/**
- * Runtime context allowing access to the tables in a database.
- */
-public class DataContextImpl implements DataContext {
-    /** */
-    private final PlannerContext ctx;
-
-    /** */
-    private final Map<String, Object> params;
-
-    /**
-     * @param params Parameters.
-     * @param ctx Query context.
-     */
-    public DataContextImpl(Map<String, Object> params, PlannerContext ctx) {
-        this.params = params;
-        this.ctx = ctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public SchemaPlus getRootSchema() {
-        return ctx.schema();
-    }
-
-    /** {@inheritDoc} */
-    @Override public JavaTypeFactory getTypeFactory() {
-        return ctx.typeFactory();
-    }
-
-    /** {@inheritDoc} */
-    @Override public QueryProvider getQueryProvider() {
-        return ctx.queryProvider();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object get(String name) {
-        if (ContextValue.PLANNER_CONTEXT.valueName().equals(name))
-            return ctx;
-
-        return params.get(name);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
deleted file mode 100644
index 253f10d..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/DistributedExecution.java
+++ /dev/null
@@ -1,110 +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.prepare;
-
-import java.util.Arrays;
-import java.util.List;
-import org.apache.calcite.linq4j.Linq4j;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelDistributionTraitDef;
-import org.apache.calcite.rel.RelDistributions;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelRoot;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
-import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
-import org.apache.ignite.internal.processors.query.calcite.util.ListFieldsQueryCursor;
-
-/**
- * Operation object, encapsulates query execution logic.
- */
-public class DistributedExecution implements QueryExecution {
-    /** */
-    private final PlannerContext ctx;
-
-    /**
-     * @param ctx Query context.
-     */
-    public DistributedExecution(PlannerContext ctx) {
-        this.ctx = ctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public FieldsQueryCursor<List<?>> execute() {
-        CalciteQueryProcessor proc = ctx.queryProcessor();
-        Query query = ctx.query();
-
-        RelTraitDef[] traitDefs = {
-            RelDistributionTraitDef.INSTANCE,
-            ConventionTraitDef.INSTANCE,
-            RelCollationTraitDef.INSTANCE
-        };
-
-        RelRoot relRoot;
-
-        try (IgnitePlanner planner = proc.planner(traitDefs, ctx)) {
-            // Parse
-            SqlNode sqlNode = planner.parse(query.sql());
-
-            // Validate
-            sqlNode = planner.validate(sqlNode);
-
-            // Convert to Relational operators graph
-            relRoot = planner.rel(sqlNode);
-
-            RelNode rel = relRoot.rel;
-
-            // Transformation chain
-            rel = planner.transform(PlannerType.HEP, PlannerPhase.SUBQUERY_REWRITE, rel, rel.getTraitSet());
-
-            RelTraitSet desired = rel.getTraitSet()
-                .replace(relRoot.collation)
-                .replace(IgniteConvention.INSTANCE)
-                .replace(RelDistributions.ANY)
-                .simplify();
-
-            rel = planner.transform(PlannerType.VOLCANO, PlannerPhase.OPTIMIZATION, rel, desired);
-
-            relRoot = relRoot.withRel(rel).withKind(sqlNode.getKind());
-        } catch (SqlParseException | ValidationException e) {
-            String msg = "Failed to parse query.";
-
-            ctx.logger().error(msg, e);
-
-            throw new IgniteSQLException(msg, IgniteQueryErrorCode.PARSING, e);
-        } catch (Exception e) {
-            String msg = "Failed to create query execution graph.";
-
-            ctx.logger().error(msg, e);
-
-            throw new IgniteSQLException(msg, IgniteQueryErrorCode.UNKNOWN, e);
-        }
-
-        // TODO physical plan.
-
-        return new ListFieldsQueryCursor<>(relRoot.rel.getRowType(), Linq4j.emptyEnumerable(), Arrays::asList);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
index d237c5d..40dcf0e 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
@@ -20,13 +20,9 @@ package org.apache.ignite.internal.processors.query.calcite.prepare;
 import com.google.common.collect.ImmutableList;
 import java.io.Reader;
 import java.util.List;
-import java.util.Properties;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.config.CalciteConnectionConfig;
-import org.apache.calcite.config.CalciteConnectionConfigImpl;
-import org.apache.calcite.config.CalciteConnectionProperty;
 import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCostImpl;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -47,7 +43,6 @@ import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexExecutor;
 import org.apache.calcite.schema.SchemaPlus;
@@ -73,10 +68,6 @@ 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.splitter.QueryPlan;
-import org.apache.ignite.internal.processors.query.calcite.splitter.Splitter;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
@@ -93,7 +84,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
     private final FrameworkConfig frameworkConfig;
 
     /** */
-    private final Context context;
+    private final PlanningContext ctx;
 
     /** */
     private final CalciteConnectionConfig connectionConfig;
@@ -112,7 +103,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
     private final SqlRexConvertletTable convertletTable;
 
     /** */
-    private final RexExecutor executor;
+    private final RexExecutor rexExecutor;
 
     /** */
     private final SchemaPlus defaultSchema;
@@ -133,42 +124,26 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
     private SqlValidator validator;
 
     /**
-     * @param config Framework config.
+     * @param ctx Planner context.
      */
-    public IgnitePlanner(FrameworkConfig config) {
-        frameworkConfig = config;
-        defaultSchema = config.getDefaultSchema();
-        operatorTable = config.getOperatorTable();
-        programs = config.getPrograms();
-        parserConfig = config.getParserConfig();
-        sqlToRelConverterConfig = config.getSqlToRelConverterConfig();
-        traitDefs = config.getTraitDefs();
-        convertletTable = config.getConvertletTable();
-        executor = config.getExecutor();
-        context = config.getContext();
-        connectionConfig = connConfig();
-
-        RelDataTypeSystem typeSystem = connectionConfig
-            .typeSystem(RelDataTypeSystem.class, IgniteTypeSystem.DEFAULT);
-
-        typeFactory = new IgniteTypeFactory(typeSystem);
-
-        Commons.plannerContext(context).planner(this);
+    IgnitePlanner(PlanningContext ctx) {
+        this.ctx = ctx;
+
+        frameworkConfig = ctx.frameworkConfig();
+        connectionConfig = ctx.connectionConfig();
+        typeFactory = ctx.typeFactory();
+
+        defaultSchema = frameworkConfig.getDefaultSchema();
+        operatorTable = frameworkConfig.getOperatorTable();
+        programs = frameworkConfig.getPrograms();
+        parserConfig = frameworkConfig.getParserConfig();
+        sqlToRelConverterConfig = frameworkConfig.getSqlToRelConverterConfig();
+        convertletTable = frameworkConfig.getConvertletTable();
+        rexExecutor = frameworkConfig.getExecutor();
+        traitDefs = frameworkConfig.getTraitDefs();
     }
 
-    /** */
-    private CalciteConnectionConfig connConfig() {
-        CalciteConnectionConfig unwrapped = context.unwrap(CalciteConnectionConfig.class);
-        if (unwrapped != null)
-            return unwrapped;
-
-        Properties properties = new Properties();
-        properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
-            String.valueOf(parserConfig.caseSensitive()));
-        properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
-            String.valueOf(frameworkConfig.getParserConfig().conformance()));
-        return new CalciteConnectionConfigImpl(properties);
-    }
+
 
     /** {@inheritDoc} */
     @Override public RelTraitSet getEmptyTraitSet() {
@@ -191,11 +166,18 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
         open = false;
     }
 
+    /**
+     * @return Planner context.
+     */
+    public PlanningContext context() {
+        return ctx;
+    }
+
     /** */
     private void ready() {
         if (!open) {
-            planner = VolcanoUtils.impatient(new VolcanoPlanner(frameworkConfig.getCostFactory(), context));
-            planner.setExecutor(executor);
+            planner = VolcanoUtils.impatient(new VolcanoPlanner(frameworkConfig.getCostFactory(), ctx));
+            planner.setExecutor(rexExecutor);
             metadataProvider = new CachingRelMetadataProvider(IgniteMetadata.METADATA_PROVIDER, planner);
 
             validator = new IgniteSqlValidator(operatorTable(), createCatalogReader(), typeFactory, conformance());
@@ -246,21 +228,27 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
      * @param graph Relational nodes tree representation.
      * @return Root node of relational tree.
      */
-    public RelNode convert(RelGraph graph) {
+    public IgniteRel convert(RelGraph graph) {
         ready();
 
-        RelOptCluster cluster = createCluster(createRexBuilder());
+        RelOptCluster cluster = createCluster();
         RelBuilder relBuilder = createRelBuilder(cluster, createCatalogReader());
 
         return new GraphToRelConverter(this, relBuilder, operatorTable).convert(graph);
     }
 
+    /** Creates a cluster. */
+    RelOptCluster createCluster() {
+        ready();
+
+        return createCluster(createRexBuilder());
+    }
+
     /** {@inheritDoc} */
     @Override public RelRoot rel(SqlNode sql) {
         ready();
 
-        RexBuilder rexBuilder = createRexBuilder();
-        RelOptCluster cluster = createCluster(rexBuilder);
+        RelOptCluster cluster = createCluster();
         SqlToRelConverter.Config config = SqlToRelConverter.configBuilder()
             .withConfig(sqlToRelConverterConfig)
             .withTrimUnusedFields(false)
@@ -276,20 +264,6 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
     }
 
     /**
-     * Splits relational nodes tree into fragments and returns a query plan.
-     * @param rel Root node of relational tree.
-     * @return Query plan.
-     */
-    public QueryPlan plan(RelNode rel) {
-        ready();
-
-        if (rel.getConvention() != IgniteConvention.INSTANCE)
-            throw new IllegalArgumentException("Physical node is required.");
-
-        return new Splitter().go((IgniteRel) rel);
-    }
-
-    /**
      * Creates an intermediate relational nodes tree representation for a given relational nodes tree.
      *
      * @param rel Root node of relational tree.
@@ -323,8 +297,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
         SqlValidator validator = new IgniteSqlValidator(operatorTable(), catalogReader, typeFactory, conformance);
         validator.setIdentifierExpansion(true);
 
-        RexBuilder rexBuilder = createRexBuilder();
-        RelOptCluster cluster = createCluster(rexBuilder);
+        RelOptCluster cluster = createCluster();
         SqlToRelConverter.Config config = SqlToRelConverter
             .configBuilder()
             .withConfig(sqlToRelConverterConfig)
@@ -369,7 +342,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
      * @param targetTraits Target traits.
      * @return The root of the new RelNode tree.
      */
-    public RelNode transform(PlannerType plannerType, PlannerPhase plannerPhase, RelNode input, RelTraitSet targetTraits)  {
+    public <T extends RelNode> T transform(PlannerType plannerType, PlannerPhase plannerPhase, RelNode input, RelTraitSet targetTraits)  {
         ready();
 
         RelTraitSet toTraits = targetTraits.simplify();
@@ -380,12 +353,12 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
             case HEP:
                 final HepProgramBuilder programBuilder = new HepProgramBuilder();
 
-                for (RelOptRule rule : plannerPhase.getRules(Commons.plannerContext(context))) {
+                for (RelOptRule rule : plannerPhase.getRules(Commons.context(ctx))) {
                     programBuilder.addRuleInstance(rule);
                 }
 
                 final HepPlanner hepPlanner =
-                    new HepPlanner(programBuilder.build(), context, true, null, RelOptCostImpl.FACTORY);
+                    new HepPlanner(programBuilder.build(), ctx, true, null, RelOptCostImpl.FACTORY);
 
                 hepPlanner.setRoot(input);
 
@@ -396,7 +369,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
 
                 break;
             case VOLCANO:
-                Program program = Programs.of(plannerPhase.getRules(Commons.plannerContext(context)));
+                Program program = Programs.of(plannerPhase.getRules(Commons.context(ctx)));
 
                 output = program.run(planner, input, toTraits,
                     ImmutableList.of(), ImmutableList.of());
@@ -406,7 +379,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
                 throw new AssertionError("Unknown planner type: " + plannerType);
         }
 
-        return output;
+        return (T) output;
     }
 
     /** {@inheritDoc} */
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 6fba164..aaa7ad2 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
@@ -17,13 +17,20 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.validate.SelectScope;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.ignite.internal.processors.query.calcite.type.SystemType;
 
 /** Validator. */
 public class IgniteSqlValidator extends SqlValidatorImpl {
@@ -44,16 +51,34 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
     /** {@inheritDoc} */
     @Override protected RelDataType getLogicalSourceRowType(
         RelDataType sourceRowType, SqlInsert insert) {
-        final RelDataType superType =
-            super.getLogicalSourceRowType(sourceRowType, insert);
-        return ((JavaTypeFactory) typeFactory).toSql(superType);
+        return typeFactory().toSql(super.getLogicalSourceRowType(sourceRowType, insert));
     }
 
     /** {@inheritDoc} */
     @Override protected RelDataType getLogicalTargetRowType(
         RelDataType targetRowType, SqlInsert insert) {
-        final RelDataType superType =
-            super.getLogicalTargetRowType(targetRowType, insert);
-        return ((JavaTypeFactory) typeFactory).toSql(superType);
+        return typeFactory().toSql(super.getLogicalTargetRowType(targetRowType, insert));
+    }
+
+    /** {@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)))
+            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;
+    }
+
+    /** */
+    private JavaTypeFactory typeFactory() {
+        return (JavaTypeFactory) typeFactory;
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
deleted file mode 100644
index 14343a7..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerContext.java
+++ /dev/null
@@ -1,345 +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.prepare;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.plan.Context;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
-import org.apache.ignite.internal.processors.query.calcite.exchange.ExchangeProcessor;
-import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService;
-import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
-
-/**
- * Planner context, encapsulates services, kernal context, query string and its flags and parameters and helper methods
- * to work with them.
- */
-public final class PlannerContext implements Context {
-    /** */
-    private final Context parentContext;
-
-    /** */
-    private final Query query;
-
-    /** */
-    private final AffinityTopologyVersion topologyVersion;
-
-    /** */
-    private final SchemaPlus schema;
-
-    /** */
-    private final IgniteLogger logger;
-
-    /** */
-    private final GridKernalContext kernalContext;
-
-    /** */
-    private final CalciteQueryProcessor queryProcessor;
-
-    /** */
-    private final MappingService mappingService;
-
-    /** */
-    private final ExchangeProcessor exchangeProcessor;
-
-    /** */
-    private IgnitePlanner planner;
-
-    /**
-     * Private constructor, used by a builder.
-     */
-    private PlannerContext(Context parentContext, Query query, AffinityTopologyVersion topologyVersion,
-        SchemaPlus schema, IgniteLogger logger, GridKernalContext kernalContext, CalciteQueryProcessor queryProcessor, MappingService mappingService,
-        ExchangeProcessor exchangeProcessor) {
-        this.parentContext = parentContext;
-        this.query = query;
-        this.topologyVersion = topologyVersion;
-        this.schema = schema;
-        this.logger = logger;
-        this.kernalContext = kernalContext;
-        this.queryProcessor = queryProcessor;
-        this.mappingService = mappingService;
-        this.exchangeProcessor = exchangeProcessor;
-    }
-
-    /**
-     * @return Query and its parameters.
-     */
-    public Query query() {
-        return query;
-    }
-
-    /**
-     * @return Topology version.
-     */
-    public AffinityTopologyVersion topologyVersion() {
-        return topologyVersion;
-    }
-
-    /**
-     * @return Schema.
-     */
-    public SchemaPlus schema() {
-        return schema;
-    }
-
-    /**
-     * @return Logger.
-     */
-    public IgniteLogger logger() {
-        return logger;
-    }
-
-    /**
-     * @return Kernal context.
-     */
-    public GridKernalContext kernalContext() {
-        return kernalContext;
-    }
-
-    /**
-     * Returns an affinityFunction for a given cache ID.
-     *
-     * @param cacheId Cache ID.
-     * @return Affinity function.
-     */
-    public AffinityFunction affinityFunction(int cacheId) {
-        return kernalContext.cache().context().cacheContext(cacheId).group().affinityFunction();
-    }
-
-    /**
-     * @return Query processor.
-     */
-    public CalciteQueryProcessor queryProcessor() {
-        return queryProcessor;
-    }
-
-    /**
-     * Package private method to set a planner after it creates using provided PlannerContext.
-     *
-     * @param planner Planner.
-     */
-    void planner(IgnitePlanner planner) {
-        this.planner = planner;
-    }
-
-    /**
-     * @return Planner.
-     */
-    public IgnitePlanner planner() {
-        return planner;
-    }
-
-    /**
-     * @return Mapping service.
-     */
-    public MappingService mappingService() {
-        return mappingService;
-    }
-
-    /**
-     * @return Exchange processor.
-     */
-    public ExchangeProcessor exchangeProcessor() {
-        return exchangeProcessor;
-    }
-
-    // Helper methods
-
-    /**
-     * @return Type factory.
-     */
-    public JavaTypeFactory typeFactory() {
-        return planner.getTypeFactory();
-    }
-
-    /**
-     * @return Local node mapping that consists of local node only, uses for root query fragment.
-     */
-    public NodesMapping mapForLocal() {
-        return mappingService.local();
-    }
-
-    /**
-     * Returns Nodes mapping for intermediate fragments, without Scan nodes leafs. Such fragments may be executed
-     * on any cluster node, actual list of nodes is chosen on the basis of adopted selection strategy.
-     *
-     * @return Nodes mapping for intermediate fragments.
-     */
-    public NodesMapping mapForRandom() {
-        return mappingService.random(topologyVersion);
-    }
-
-    /**
-     * @param cacheId Cache ID.
-     * @return Nodes mapping for particular table, depends on underlying cache distribution.
-     */
-    public NodesMapping mapForCache(int cacheId) {
-        return mappingService.distributed(cacheId, topologyVersion);
-    }
-
-    /**
-     * @return Query provider. Used to execute a query to remote database (federated database case)
-     * or to execute a correlated query.
-     */
-    public QueryProvider queryProvider() {
-        return null; // TODO
-    }
-
-    /** {@inheritDoc} */
-    @Override public <C> C unwrap(Class<C> aClass) {
-        if (aClass == getClass())
-            return aClass.cast(this);
-
-        return parentContext.unwrap(aClass);
-    }
-
-    /**
-     * @return Context builder.
-     */
-    public static Builder builder() {
-        return new Builder();
-    }
-
-    /**
-     * Planner context builder.
-     */
-    public static class Builder {
-        /** */
-        private Context parentContext;
-
-        /** */
-        private Query query;
-
-        /** */
-        private AffinityTopologyVersion topologyVersion;
-
-        /** */
-        private SchemaPlus schema;
-
-        /** */
-        private IgniteLogger logger;
-
-        /** */
-        private GridKernalContext kernalContext;
-
-        /** */
-        private CalciteQueryProcessor queryProcessor;
-
-        /** */
-        private MappingService mappingService;
-
-        /** */
-        private ExchangeProcessor exchangeProcessor;
-
-        /**
-         * @param parentContext Parent context.
-         * @return Builder for chaining.
-         */
-        public Builder parentContext(Context parentContext) {
-            this.parentContext = parentContext;
-            return this;
-        }
-
-        /**
-         * @param query Query.
-         * @return Builder for chaining.
-         */
-        public Builder query(Query query) {
-            this.query = query;
-            return this;
-        }
-
-        /**
-         * @param topologyVersion Topology version.
-         * @return Builder for chaining.
-         */
-        public Builder topologyVersion(AffinityTopologyVersion topologyVersion) {
-            this.topologyVersion = topologyVersion;
-            return this;
-        }
-
-        /**
-         * @param schema Schema.
-         * @return Builder for chaining.
-         */
-        public Builder schema(SchemaPlus schema) {
-            this.schema = schema;
-            return this;
-        }
-
-        /**
-         * @param logger Logger.
-         * @return Builder for chaining.
-         */
-        public Builder logger(IgniteLogger logger) {
-            this.logger = logger;
-            return this;
-        }
-
-        /**
-         * @param kernalContext Kernal context.
-         * @return Builder for chaining.
-         */
-        public Builder kernalContext(GridKernalContext kernalContext) {
-            this.kernalContext = kernalContext;
-            return this;
-        }
-
-        /**
-         * @param queryProcessor Query processor.
-         * @return Builder for chaining.
-         */
-        public Builder queryProcessor(CalciteQueryProcessor queryProcessor) {
-            this.queryProcessor = queryProcessor;
-            return this;
-        }
-
-        /**
-         * @param mappingService Mapping service.
-         * @return Builder for chaining.
-         */
-        public Builder mappingService(MappingService mappingService) {
-            this.mappingService = mappingService;
-            return this;
-        }
-
-        /**
-         * @param exchangeProcessor Exchange processor.
-         * @return Builder for chaining.
-         */
-        public Builder exchangeProcessor(ExchangeProcessor exchangeProcessor) {
-            this.exchangeProcessor = exchangeProcessor;
-            return this;
-        }
-
-        /**
-         * Builds planner context.
-         *
-         * @return Planner context.
-         */
-        public PlannerContext build() {
-            return new PlannerContext(parentContext, query, topologyVersion, schema, logger, kernalContext, queryProcessor, mappingService, exchangeProcessor);
-        }
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlannerPhase.java
index 526ca08..9417af2 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
@@ -31,7 +31,7 @@ public enum PlannerPhase {
     /** */
     SUBQUERY_REWRITE("Sub-queries rewrites") {
         /** {@inheritDoc} */
-        @Override public RuleSet getRules(PlannerContext ctx) {
+        @Override public RuleSet getRules(PlanningContext ctx) {
             return RuleSets.ofList(
                 SubQueryRemoveRule.FILTER,
                 SubQueryRemoveRule.PROJECT,
@@ -42,7 +42,7 @@ public enum PlannerPhase {
     /** */
     OPTIMIZATION("Main optimization phase") {
         /** {@inheritDoc} */
-        @Override public RuleSet getRules(PlannerContext ctx) {
+        @Override public RuleSet getRules(PlanningContext ctx) {
             return RuleSets.ofList(
                 JoinConverter.INSTANCE,
                 ProjectConverter.INSTANCE,
@@ -65,5 +65,5 @@ public enum PlannerPhase {
      * @param ctx Planner context.
      * @return Rule set.
      */
-    public abstract RuleSet getRules(PlannerContext ctx);
+    public abstract RuleSet getRules(PlanningContext ctx);
 }
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
new file mode 100644
index 0000000..fabe1d6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/PlanningContext.java
@@ -0,0 +1,342 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.prepare;
+
+import java.util.Properties;
+import java.util.UUID;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/**
+ * Planning context.
+ */
+public final class PlanningContext implements Context {
+    /** */
+    private final FrameworkConfig frameworkConfig;
+
+    /** */
+    private final Context parentContext;
+
+    /** */
+    private final UUID localNodeId;
+
+    /** */
+    private final UUID originatingNodeId;
+
+    /** */
+    private final String query;
+
+    /** */
+    private final Object[] parameters;
+
+    /** */
+    private final AffinityTopologyVersion topologyVersion;
+
+    /** */
+    private final IgniteLogger logger;
+
+    /** */
+    private IgnitePlanner planner;
+
+    /** */
+    private CalciteConnectionConfig connectionConfig;
+
+    /** */
+    private JavaTypeFactory typeFactory;
+
+    /**
+     * Private constructor, used by a builder.
+     */
+    private PlanningContext(FrameworkConfig config, Context parentContext, UUID localNodeId, UUID originatingNodeId,
+        String query, Object[] parameters, AffinityTopologyVersion topologyVersion, IgniteLogger logger) {
+        this.parentContext = parentContext;
+        this.localNodeId = localNodeId;
+        this.parameters = parameters;
+        this.originatingNodeId = originatingNodeId == null ? localNodeId : originatingNodeId;
+        this.query = query;
+        this.topologyVersion = topologyVersion;
+        this.logger = logger;
+
+        // link frameworkConfig#context() to this.
+        Frameworks.ConfigBuilder b = config == null ? Frameworks.newConfigBuilder() :
+            Frameworks.newConfigBuilder(config);
+
+        frameworkConfig = b.context(this).build();
+    }
+
+    /**
+     * @return Local node ID.
+     */
+    public UUID localNodeId() {
+        return localNodeId;
+    }
+
+    /**
+     * @return Originating node ID (the node, who started the execution).
+     */
+    public UUID originatingNodeId() {
+        return originatingNodeId;
+    }
+
+    /**
+     * @return Framework config.
+     */
+    public FrameworkConfig frameworkConfig() {
+        return frameworkConfig;
+    }
+
+    /**
+     * @return Query.
+     */
+    public String query() {
+        return query;
+    }
+
+    /**
+     * @return Query parameters.
+     */
+    public Object[] parameters() {
+        return parameters;
+    }
+
+    /**
+     * @return Topology version.
+     */
+    public AffinityTopologyVersion topologyVersion() {
+        return topologyVersion;
+    }
+
+    /**
+     * @return Logger.
+     */
+    public IgniteLogger logger() {
+        return logger;
+    }
+
+    // Helper methods
+
+    /**
+     * @return Planner.
+     */
+    public IgnitePlanner planner() {
+        if (planner == null)
+            planner = new IgnitePlanner(this);
+
+        return planner;
+    }
+
+    /**
+     * @return Schema.
+     */
+    public SchemaPlus schema() {
+        return frameworkConfig.getDefaultSchema();
+    }
+
+    /**
+     * @return Type factory.
+     */
+    public JavaTypeFactory typeFactory() {
+        if (typeFactory != null)
+            return typeFactory;
+
+        RelDataTypeSystem typeSystem = connectionConfig().typeSystem(RelDataTypeSystem.class, frameworkConfig.getTypeSystem());
+
+        return typeFactory = new IgniteTypeFactory(typeSystem);
+    }
+
+    /**
+     * @return Connection config. Defines connected user parameters like TimeZone or Locale.
+     */
+    public CalciteConnectionConfig connectionConfig() {
+        if (connectionConfig != null)
+            return connectionConfig;
+
+        CalciteConnectionConfig connConfig = unwrap(CalciteConnectionConfig.class);
+
+        if (connConfig != null)
+            return connectionConfig = connConfig;
+
+        Properties properties = new Properties();
+
+        properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
+            String.valueOf(frameworkConfig.getParserConfig().caseSensitive()));
+        properties.setProperty(CalciteConnectionProperty.CONFORMANCE.camelName(),
+            String.valueOf(frameworkConfig.getParserConfig().conformance()));
+
+        return connectionConfig = new CalciteConnectionConfigImpl(properties);
+    }
+
+    /**
+     * @return New cluster based on a planner and its configuration.
+     */
+    public RelOptCluster createCluster() {
+        return planner().createCluster();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <C> C unwrap(Class<C> aClass) {
+        if (aClass == getClass())
+            return aClass.cast(this);
+
+        return parentContext.unwrap(aClass);
+    }
+
+    /**
+     * @return Context builder.
+     */
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    /**
+     * @return Context builder.
+     */
+    public static Builder builder(PlanningContext template) {
+        return new Builder()
+            .logger(template.logger)
+            .topologyVersion(template.topologyVersion)
+            .query(template.query)
+            .parameters(template.parameters)
+            .parentContext(template.parentContext)
+            .frameworkConfig(template.frameworkConfig)
+            .originatingNodeId(template.originatingNodeId)
+            .localNodeId(template.localNodeId);
+    }
+
+    /**
+     * Planner context builder.
+     */
+    public static class Builder {
+        /** */
+        private UUID localNodeId;
+
+        /** */
+        private UUID originatingNodeId;
+
+        /** */
+        private FrameworkConfig frameworkConfig;
+
+        /** */
+        private Context parentContext;
+
+        /** */
+        private String query;
+
+        /** */
+        private Object[] parameters;
+
+        /** */
+        private AffinityTopologyVersion topologyVersion;
+
+        /** */
+        private IgniteLogger logger;
+
+        /**
+         * @param localNodeId Local node ID.
+         * @return Builder for chaining.
+         */
+        public Builder localNodeId(UUID localNodeId) {
+            this.localNodeId = localNodeId;
+            return this;
+        }
+
+        /**
+         * @param originatingNodeId Originating node ID (the node, who started the execution).
+         * @return Builder for chaining.
+         */
+        public Builder originatingNodeId(UUID originatingNodeId) {
+            this.originatingNodeId = originatingNodeId;
+            return this;
+        }
+
+        /**
+         * @param frameworkConfig Framework config.
+         * @return Builder for chaining.
+         */
+        public Builder frameworkConfig(FrameworkConfig frameworkConfig) {
+            this.frameworkConfig = frameworkConfig;
+            return this;
+        }
+
+        /**
+         * @param parentContext Parent context.
+         * @return Builder for chaining.
+         */
+        public Builder parentContext(Context parentContext) {
+            this.parentContext = parentContext;
+            return this;
+        }
+
+        /**
+         * @param query Query.
+         * @return Builder for chaining.
+         */
+        public Builder query(String query) {
+            this.query = query;
+            return this;
+        }
+
+        /**
+         * @param parameters Query parameters.
+         * @return Builder for chaining.
+         */
+        public Builder parameters(Object[] parameters) {
+            this.parameters = parameters;
+            return this;
+        }
+
+        /**
+         * @param topologyVersion Topology version.
+         * @return Builder for chaining.
+         */
+        public Builder topologyVersion(AffinityTopologyVersion topologyVersion) {
+            this.topologyVersion = topologyVersion;
+            return this;
+        }
+
+        /**
+         * @param logger Logger.
+         * @return Builder for chaining.
+         */
+        public Builder logger(IgniteLogger logger) {
+            this.logger = logger;
+            return this;
+        }
+
+        /**
+         * Builds planner context.
+         *
+         * @return Planner context.
+         */
+        public PlanningContext build() {
+            return new PlanningContext(frameworkConfig, parentContext, localNodeId, originatingNodeId, query,
+                parameters, topologyVersion, logger);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.java
deleted file mode 100644
index 651da36..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/Query.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.prepare;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.internal.util.typedef.F;
-
-/**
- * Represents a query and its parameters.
- */
-public class Query {
-    /** */
-    private final String sql;
-
-    /** */
-    private final Object[] params;
-
-    /**
-     * @param sql Query text.
-     * @param params Query parameters.
-     */
-    public Query(String sql, Object[] params) {
-        this.sql = sql;
-        this.params = params;
-    }
-
-    /**
-     * @return Query text.
-     */
-    public String sql() {
-        return sql;
-    }
-
-    /**
-     * @return Query parameters.
-     */
-    public Object[] params() {
-        return params;
-    }
-
-    /**
-     * Populates a given parameters map with named parameters (using theirs ordinal)
-     * @param stashed Stashed parameters map.
-     * @return Populated parameters map.
-     */
-    public Map<String, Object> params(Map<String, Object> stashed) {
-        Map<String, Object> res = new HashMap<>(stashed);
-        if (!F.isEmpty(params)) {
-            for (int i = 0; i < params.length; i++) {
-                res.put("?" + i, params[i]);
-            }
-        }
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (!(o instanceof Query))
-            return false;
-
-        Query query = (Query) o;
-
-        if (!sql.equals(query.sql))
-            return false;
-        return Arrays.equals(params, query.params);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int result = sql.hashCode();
-        result = 31 * result + Arrays.hashCode(params);
-        return result;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
similarity index 66%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
index 4079743..0f2586e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
@@ -17,17 +17,18 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-import java.util.List;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
- * Operation object, encapsulates query execution logic.
+ *
  */
-public interface QueryExecution {
+public interface QueryPlanCache extends Service {
     /**
-     * Executes a query.
-     *
-     * @return Query cursor.
+     * @param ctx Context.
+     * @param key Cache key.
+     * @param factory Factory method to generate a plan on cache miss.
+     * @return Query plan.
      */
-    FieldsQueryCursor<List<?>> execute();
+    QueryPlan queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
new file mode 100644
index 0000000..9605be4
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.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.prepare;
+
+import java.util.Map;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+
+/**
+ *
+ */
+public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCache, SchemaChangeListener {
+    /** */
+    private static final int CACHE_SIZE = 1024;
+
+    /** */
+    private GridInternalSubscriptionProcessor subscriptionProcessor;
+
+    /** */
+    private volatile Map<CacheKey, QueryPlan> cache;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public QueryPlanCacheImpl(GridKernalContext ctx) {
+        super(ctx);
+
+        cache = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+        subscriptionProcessor(ctx.internalSubscriptionProcessor());
+
+        init();
+    }
+
+    /**
+     * @param subscriptionProcessor Subscription processor.
+     */
+    public void subscriptionProcessor(GridInternalSubscriptionProcessor subscriptionProcessor) {
+        this.subscriptionProcessor = subscriptionProcessor;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init() {
+        subscriptionProcessor.registerSchemaChangeListener(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryPlan queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory) {
+        Map<CacheKey, QueryPlan> cache = this.cache;
+
+        QueryPlan template = cache.get(key);
+
+        if (template != null)
+            return template.clone(ctx.createCluster());
+
+        QueryPlan plan = factory.create(ctx);
+
+        cache.putIfAbsent(key, plan.clone(Commons.EMPTY_CLUSTER));
+
+        return plan;
+    }
+
+    /**
+     * Clear cached plans.
+     */
+    public void clear() {
+        cache = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSchemaDrop(String schemaName) {
+        clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+        clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSchemaCreate(String schemaName) {
+        // No-op
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+        // No-op
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
similarity index 76%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
index 4079743..da9ad4d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryExecution.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
@@ -17,17 +17,15 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-import java.util.List;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.internal.processors.query.calcite.splitter.QueryPlan;
 
 /**
- * Operation object, encapsulates query execution logic.
+ *
  */
-public interface QueryExecution {
+public interface QueryPlanFactory {
     /**
-     * Executes a query.
-     *
-     * @return Query cursor.
+     * @param ctx Planning context.
+     * @return Query plan.
      */
-    FieldsQueryCursor<List<?>> execute();
+    QueryPlan create(PlanningContext ctx);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
index c3c869b..5289c90 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rel/IgniteReceiver.java
@@ -21,6 +21,8 @@ import java.util.List;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.ignite.internal.processors.query.calcite.splitter.RelSource;
@@ -72,4 +74,11 @@ public class IgniteReceiver extends AbstractRelNode implements IgniteRel {
     public IgniteDistribution distribution() {
         return getTraitSet().getTrait(DistributionTraitDef.INSTANCE);
     }
+
+    /**
+     * @return Node collations.
+     */
+    public List<RelCollation> collations() {
+        return getTraitSet().getTraits(RelCollationTraitDef.INSTANCE);
+    }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/DistributedTable.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/DistributedTable.java
index e10fdb4..e03b73d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/splitter/RelTarget.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/DistributedTable.java
@@ -15,33 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.splitter;
+package org.apache.ignite.internal.processors.query.calcite.schema;
 
-import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.schema.Table;
 import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
-import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 
 /**
  *
  */
-public interface RelTarget {
+public interface DistributedTable extends Table {
     /**
-     * @return Exchange id, has to be unique in scope of query.
+     * Returns locations mapping of the table.
+     * @param ctx Context.
+     * @return Nodes mapping.
      */
-    long exchangeId();
+    NodesMapping mapping(PlanningContext ctx);
 
     /**
-     * Returns target mapping. It's used in calculation where to send a particular row to.
-     * See {@link DistributionFunction#toDestination(PlannerContext, NodesMapping, ImmutableIntList)}
-     *
-     * @return Target mapping.
-     */
-    NodesMapping mapping();
-
-    /**
-     * @return Target distribution.
+     * @return Distribution of the table.
      */
     IgniteDistribution distribution();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
index eee1d62..7af76e2 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteSchema.java
@@ -22,10 +22,6 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * Ignite schema.
@@ -59,33 +55,10 @@ public class IgniteSchema extends AbstractSchema {
     }
 
     /**
-     * Callback method.
-     *
-     * @param typeDesc Query type descriptor.
-     * @param cacheInfo Cache info.
-     */
-    public void onSqlTypeCreate(GridQueryTypeDescriptor typeDesc, GridCacheContextInfo<?,?> cacheInfo) {
-        Object identityKey = cacheInfo.config().getCacheMode() == CacheMode.PARTITIONED ?
-            cacheInfo.cacheContext().group().affinity().similarAffinityKey() : null;
-
-        addTable(new IgniteTable(typeDesc.tableName(), cacheInfo.name(), Commons.rowType(typeDesc), identityKey));
-    }
-
-    /**
-     * Callback method.
-     *
-     * @param typeDesc Query type descriptor.
-     * @param cacheInfo Cache info.
-     */
-    public void onSqlTypeDrop(GridQueryTypeDescriptor typeDesc, GridCacheContextInfo<?,?> cacheInfo) {
-        removeTable(typeDesc.tableName());
-    }
-
-    /**
      * @param table Table.
      */
-    public void addTable(IgniteTable table) {
-        tableMap.put(table.tableName(), table);
+    public void addTable(String tableName, Table table) {
+        tableMap.put(tableName, table);
     }
 
     /**
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
index 6932a17..a5f9184 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTable.java
@@ -18,14 +18,18 @@
 package org.apache.ignite.internal.processors.query.calcite.schema;
 
 import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
+import java.util.UUID;
 import org.apache.calcite.DataContext;
 import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.type.RelDataType;
@@ -35,111 +39,186 @@ import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.impl.AbstractTable;
 import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentInfo;
-import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerContext;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheStoppedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.metadata.NodesMapping;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
 import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
-import org.apache.ignite.internal.processors.query.calcite.trait.DistributionFunction;
 import org.apache.ignite.internal.processors.query.calcite.trait.DistributionTraitDef;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
-import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
-import org.apache.ignite.internal.processors.query.calcite.type.RowType;
-import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.TableScan;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
 
 /** */
-public class IgniteTable extends AbstractTable implements TranslatableTable, ScannableTable {
+public class IgniteTable extends AbstractTable implements TranslatableTable, ScannableTable, DistributedTable, SortedTable {
     /** */
-    private final String tableName;
+    private final String name;
 
     /** */
-    private final String cacheName;
+    private final TableDescriptor desc;
 
     /** */
-    private final RowType rowType;
-
-    /** */
-    private final Object identityKey;
+    private final Statistic statistic;
 
     /**
-     * @param tableName Table name.
-     * @param cacheName Cache name.
-     * @param rowType Row type.
-     * @param identityKey Affinity identity key.
+     * @param name Table full name.
      */
-    public IgniteTable(String tableName, String cacheName, RowType rowType, Object identityKey) {
-        this.tableName = tableName;
-        this.cacheName = cacheName;
-        this.rowType = rowType;
-        this.identityKey = identityKey;
-    }
+    public IgniteTable(String name, TableDescriptor desc) {
+        this.name = name;
+        this.desc = desc;
 
-    /**
-     * @return Table name;
-     */
-    public String tableName() {
-        return tableName;
+        statistic = new StatisticsImpl();
     }
 
     /**
-     * @return Cache name.
+     * @return Table name.
      */
-    public String cacheName() {
-        return cacheName;
+    public String name() {
+        return name;
     }
 
     /** {@inheritDoc} */
     @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-        return rowType.asRelDataType(typeFactory);
+        return desc.apply(typeFactory);
     }
 
     /** {@inheritDoc} */
     @Override public Statistic getStatistic() {
-        return new TableStatistics();
+        return statistic;
     }
 
     /** {@inheritDoc} */
     @Override public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
         RelOptCluster cluster = context.getCluster();
         RelTraitSet traitSet = cluster.traitSetOf(IgniteConvention.INSTANCE)
-                .replaceIf(DistributionTraitDef.INSTANCE, this::distribution);
+            .replaceIfs(RelCollationTraitDef.INSTANCE, this::collations)
+            .replaceIf(DistributionTraitDef.INSTANCE, this::distribution);
 
         return new IgniteTableScan(cluster, traitSet, relOptTable);
     }
 
-    /**
-     * @return Table distribution trait.
-     */
-    public IgniteDistribution distribution() {
-        Object key = identityKey();
+    /** {@inheritDoc} */
+    @Override public NodesMapping mapping(PlanningContext ctx) {
+        GridCacheContext<?, ?> cctx = desc.cacheContext();
+
+        assert cctx != null;
 
-        if (key == null)
-            return IgniteDistributions.broadcast();
+        if (!cctx.gate().enterIfNotStopped())
+            throw U.convertException(new CacheStoppedException(cctx.name()));
 
-        return IgniteDistributions.hash(rowType.distributionKeys(), new DistributionFunction.AffinityDistribution(CU.cacheId(cacheName), key));
+        try {
+            if (cctx.isReplicated())
+                return replicatedMapping(cctx, ctx.topologyVersion());
+
+            return partitionedMapping(cctx, ctx.topologyVersion());
+        }
+        finally {
+            cctx.gate().leave();
+        }
     }
 
-    /**
-     * @return Affinity identity key.
-     */
-    protected Object identityKey() {
-        return identityKey;
+    /** {@inheritDoc} */
+    @Override public IgniteDistribution distribution() {
+        return desc.distribution();
     }
 
-    /**
-     * @param ctx Planner context.
-     * @return Fragment meta information.
-     */
-    public FragmentInfo fragmentInfo(PlannerContext ctx) {
-        return new FragmentInfo(ctx.mapForCache(CU.cacheId(cacheName)));
+    /** {@inheritDoc} */
+    @Override public List<RelCollation> collations() {
+        return desc.collations();
     }
 
     /** {@inheritDoc} */
     @Override public Enumerable<Object[]> scan(DataContext root) {
-        throw new AssertionError(); // TODO
+        return Linq4j.asEnumerable(new TableScan((ExecutionContext) root, desc));
+    }
+
+    /** */
+    private NodesMapping partitionedMapping(@NotNull GridCacheContext<?,?> cctx, @NotNull AffinityTopologyVersion topVer) {
+        byte flags = NodesMapping.HAS_PARTITIONED_CACHES;
+
+        List<List<ClusterNode>> assignments = cctx.affinity().assignments(topVer);
+        List<List<UUID>> res;
+
+        if (cctx.config().getWriteSynchronizationMode() == CacheWriteSynchronizationMode.PRIMARY_SYNC) {
+            res = new ArrayList<>(assignments.size());
+
+            for (List<ClusterNode> partNodes : assignments)
+                res.add(F.isEmpty(partNodes) ? Collections.emptyList() : Collections.singletonList(F.first(partNodes).id()));
+        }
+        else if (!cctx.topology().rebalanceFinished(topVer)) {
+            res = new ArrayList<>(assignments.size());
+
+            flags |= NodesMapping.HAS_MOVING_PARTITIONS;
+
+            for (int part = 0; part < assignments.size(); part++) {
+                List<ClusterNode> partNodes = assignments.get(part);
+                List<UUID> partIds = new ArrayList<>(partNodes.size());
+
+                for (ClusterNode node : partNodes) {
+                    if (cctx.topology().partitionState(node.id(), part) == GridDhtPartitionState.OWNING)
+                        partIds.add(node.id());
+                }
+
+                res.add(partIds);
+            }
+        }
+        else
+            res = Commons.transform(assignments, nodes -> Commons.transform(nodes, ClusterNode::id));
+
+        return new NodesMapping(null, res, flags);
+    }
+
+    /** */
+    private NodesMapping replicatedMapping(@NotNull GridCacheContext<?,?> cctx, @NotNull AffinityTopologyVersion topVer) {
+        byte flags = NodesMapping.HAS_REPLICATED_CACHES;
+
+        if (cctx.config().getNodeFilter() != null)
+            flags |= NodesMapping.PARTIALLY_REPLICATED;
+
+        GridDhtPartitionTopology topology = cctx.topology();
+
+        List<ClusterNode> nodes = cctx.discovery().discoCache(topVer).cacheGroupAffinityNodes(cctx.cacheId());
+        List<UUID> res;
+
+        if (!topology.rebalanceFinished(topVer)) {
+            flags |= NodesMapping.PARTIALLY_REPLICATED;
+
+            res = new ArrayList<>(nodes.size());
+
+            int parts = topology.partitions();
+
+            for (ClusterNode node : nodes) {
+                if (isOwner(node.id(), topology, parts))
+                    res.add(node.id());
+            }
+        }
+        else
+            res = Commons.transform(nodes, ClusterNode::id);
+
+        return new NodesMapping(res, null, flags);
+    }
+
+    /** */
+    private boolean isOwner(UUID nodeId, GridDhtPartitionTopology topology, int parts) {
+        for (int p = 0; p < parts; p++) {
+            if (topology.partitionState(nodeId, p) != GridDhtPartitionState.OWNING)
+                return false;
+        }
+        return true;
     }
 
     /** */
-    private class TableStatistics implements Statistic {
+    private class StatisticsImpl implements Statistic {
         /** {@inheritDoc} */
         @Override public Double getRowCount() {
             return null;
@@ -157,11 +236,11 @@ public class IgniteTable extends AbstractTable implements TranslatableTable, Sca
 
         /** {@inheritDoc} */
         @Override public List<RelCollation> getCollations() {
-            return ImmutableList.of();
+            return collations();
         }
 
         /** {@inheritDoc} */
-        @Override public RelDistribution getDistribution() {
+        @Override public IgniteDistribution getDistribution() {
             return distribution();
         }
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
similarity index 73%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
index 112aad9..311ef65 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolder.java
@@ -15,12 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize;
+package org.apache.ignite.internal.processors.query.calcite.schema;
 
-import java.io.Serializable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
- * Graph node.
+ *
  */
-public interface GraphNode extends Serializable {
+public interface SchemaHolder extends Service {
+    /**
+     * @return Schema.
+     */
+    SchemaPlus schema();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
similarity index 55%
rename from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
rename to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
index 3c3433e..83f2453 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CalciteSchemaHolder.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
@@ -21,22 +21,46 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.tools.Frameworks;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
 import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 
 /**
  * Holds actual schema and mutates it on schema change, requested by Ignite.
  */
-public class CalciteSchemaHolder implements SchemaChangeListener {
+public class SchemaHolderImpl extends AbstractService implements SchemaHolder, SchemaChangeListener {
     /** */
     private final Map<String, IgniteSchema> schemas = new HashMap<>();
 
     /** */
+    private GridInternalSubscriptionProcessor subscriptionProcessor;
+
+    /** */
     private volatile SchemaPlus schema;
 
     /**
+     * @param ctx Kernal context.
+     */
+    public SchemaHolderImpl(GridKernalContext ctx) {
+        super(ctx);
+
+        subscriptionProcessor(ctx.internalSubscriptionProcessor());
+
+        init();
+    }
+
+    /**
+     * @param subscriptionProcessor Subscription processor.
+     */
+    public void subscriptionProcessor(GridInternalSubscriptionProcessor subscriptionProcessor) {
+        this.subscriptionProcessor = subscriptionProcessor;
+    }
+
+    /**
      * Sets updated schema.
      * @param schema New schema.
      */
@@ -44,10 +68,18 @@ public class CalciteSchemaHolder implements SchemaChangeListener {
         this.schema = schema;
     }
 
-    /**
-     * @return Actual schema.
-     */
-    public SchemaPlus schema() {
+    /** {@inheritDoc} */
+    @Override public void init() {
+        subscriptionProcessor.registerSchemaChangeListener(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStart(GridKernalContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public SchemaPlus schema() {
         return schema;
     }
 
@@ -64,21 +96,36 @@ public class CalciteSchemaHolder implements SchemaChangeListener {
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
-        schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeCreate(typeDescriptor, cacheInfo);
+    @Override public synchronized void onSqlTypeCreate(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+        IgniteSchema schema = schemas.computeIfAbsent(schemaName, IgniteSchema::new);
+
+        String tableName = typeDescriptor.tableName();
+        TableDescriptorImpl desc = new TableDescriptorImpl(cacheInfo.cacheContext(), typeDescriptor, affinityIdentity(cacheInfo));
+
+        schema.addTable(tableName, new IgniteTable(tableName, desc));
+
         rebuild();
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo cacheInfo) {
-        schemas.computeIfAbsent(schemaName, IgniteSchema::new).onSqlTypeDrop(typeDescriptor, cacheInfo);
+    @Override public synchronized void onSqlTypeDrop(String schemaName, GridQueryTypeDescriptor typeDescriptor, GridCacheContextInfo<?,?> cacheInfo) {
+        IgniteSchema schema = schemas.computeIfAbsent(schemaName, IgniteSchema::new);
+
+        schema.removeTable(typeDescriptor.tableName());
+
         rebuild();
     }
 
     /** */
+    private Object affinityIdentity(GridCacheContextInfo<?, ?> cacheInfo) {
+        return cacheInfo.config().getCacheMode() == CacheMode.PARTITIONED ?
+            cacheInfo.cacheContext().group().affinity().similarAffinityKey() : null;
+    }
+
+    /** */
     private void rebuild() {
         SchemaPlus schema = Frameworks.createRootSchema(false);
         schemas.forEach(schema::add);
-        schema(schema.getSubSchema(QueryUtils.DFLT_SCHEMA));
+        schema(schema);
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SortedTable.java
similarity index 75%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SortedTable.java
index 112aad9..09bc18a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/GraphNode.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SortedTable.java
@@ -15,12 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.calcite.serialize;
+package org.apache.ignite.internal.processors.query.calcite.schema;
 
-import java.io.Serializable;
+import java.util.List;
+import org.apache.calcite.rel.RelCollation;
 
 /**
- * Graph node.
+ *
  */
-public interface GraphNode extends Serializable {
+public interface SortedTable {
+    /**
+     * @return The table collations.
+     */
+    List<RelCollation> collations();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
new file mode 100644
index 0000000..a70809f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.schema;
+
+import java.util.List;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+
+/**
+ *
+ */
+public interface TableDescriptor extends RelProtoDataType {
+    /**
+     * @return Underlying cache context.
+     */
+    GridCacheContext<?,?> cacheContext();
+
+    /**
+     * @return Distribution.
+     */
+    IgniteDistribution distribution();
+
+    /**
+     * @return Collations.
+     */
+    List<RelCollation> collations();
+
+    /**
+     * Checks whether a provided cache row belongs to described table.
+     *
+     * @param row Cache row.
+     * @return {@code True} If a provided cache row matches a defined query type.
+     */
+    boolean match(CacheDataRow row);
+
+    /**
+     * Converts a cache row to relational node row.
+     *
+     * @param ectx Execution context.
+     * @param row Cache row.
+     * @return Relational node row.
+     * @throws IgniteCheckedException If failed.
+     */
+    <T> T toRow(ExecutionContext ectx, CacheDataRow row) throws IgniteCheckedException;
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
new file mode 100644
index 0000000..31c607f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.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.schema;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import java.util.Map;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/**
+ *
+ */
+public class TableDescriptorImpl implements TableDescriptor {
+    /** */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    private final GridQueryTypeDescriptor typeDesc;
+
+    /** */
+    private final Object affinityIdentity;
+
+    /** */
+    private final int affinityFieldIdx;
+
+    /** */
+    public TableDescriptorImpl(GridCacheContext<?,?> cctx, GridQueryTypeDescriptor typeDesc, Object affinityIdentity) {
+        this.cctx = cctx;
+        this.typeDesc = typeDesc;
+        this.affinityIdentity = affinityIdentity;
+
+        affinityFieldIdx = lookupAffinityIndex(typeDesc);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType apply(RelDataTypeFactory factory) {
+        IgniteTypeFactory f = (IgniteTypeFactory) factory;
+
+        RelDataTypeFactory.Builder b = new RelDataTypeFactory.Builder(f);
+
+        b.add(QueryUtils.KEY_FIELD_NAME, f.createSystemType(typeDesc.keyClass()));
+        b.add(QueryUtils.VAL_FIELD_NAME, f.createSystemType(typeDesc.valueClass()));
+
+        for (Map.Entry<String, Class<?>> field : typeDesc.fields().entrySet())
+            b.add(field.getKey(), f.createJavaType(field.getValue()));
+
+        return b.build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheContext<?, ?> cacheContext() {
+        return cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteDistribution distribution() {
+        if (affinityIdentity == null)
+            return IgniteDistributions.broadcast();
+
+        return IgniteDistributions.affinity(affinityFieldIdx, cctx.cacheId(), affinityIdentity);
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<RelCollation> collations() {
+        return ImmutableList.of();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean match(CacheDataRow row) {
+        return typeDesc.matchType(row.value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T toRow(ExecutionContext ectx, CacheDataRow row) throws IgniteCheckedException {
+        Object[] res = new Object[typeDesc.fields().size() + 2];
+
+        int i = 0;
+
+        res[i++] = cctx.unwrapBinaryIfNeeded(row.key(), ectx.keepBinary());
+        res[i++] = cctx.unwrapBinaryIfNeeded(row.value(), ectx.keepBinary());
+
+        for (String field : typeDesc.fields().keySet())
+            res[i++] = cctx.unwrapBinaryIfNeeded(typeDesc.value(field, row.key(), row.value()), ectx.keepBinary());
+
+        return (T) res;
+    }
+
+    /** */
+    private static int lookupAffinityIndex(GridQueryTypeDescriptor queryTypeDesc) {
+        if (queryTypeDesc.affinityKey() != null) {
+            int idx = 2;
+
+            String affField = queryTypeDesc.affinityKey();
+
+            for (String s : queryTypeDesc.fields().keySet()) {
+                if (affField.equals(s))
+                    return idx;
+
+                idx++;
+            }
+        }
+
+        return 0;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.java
deleted file mode 100644
index 3230b3a..0000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/Graph.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;
-
-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;
-
-/**
- * A graph, represented by adjacency list.
- */
-public class Graph<T extends GraphNode> implements Serializable {
-    /** */
-    private final List<T> nodes = new ArrayList<>();
-
-    /** */
-    private final List<GridIntList> edges = new ArrayList<>();
-
-    /**
-     * @return Pairs of node and its ID.
-     */
-    public List<Ord<T>> 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, T node) {
-        int id = addNode(node);
-
-        addEdge(parentId, id);
-
-        return id;
-    }
-
-    /**
-     * Adds a new node.
-     *
-     * @param node Node.
-     * @return New node ID.
-     */
-    public int addNode(T 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<T>> children(int parentId) {
-        GridIntList children = edges.get(parentId);
-
-        ArrayList<Ord<T>> ords = new ArrayList<>(children.size());
-
-        for (int i = 0; i < children.size(); i++)
-            ords.add(Ord.of(children.get(i), nodes.get(children.get(i))));
-
-        return ords;
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/CallExpression.java
index 8087e53..124df13 100644
--- 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
@@ -18,8 +18,11 @@
 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}.
@@ -32,15 +35,20 @@ public class CallExpression implements Expression {
     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, List<Expression> operands) {
-        this.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();
     }
 
@@ -59,6 +67,13 @@ public class CallExpression implements Expression {
     }
 
     /**
+     * @return Data type.
+     */
+    public DataType dataType() {
+        return type;
+    }
+
+    /**
      * @return Operands.
      */
     public List<Expression> operands() {
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/DynamicParamExpression.java
index 74fe3c7..09e5290 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/serialize/expression/DynamicParamExpression.java
@@ -19,6 +19,7 @@ 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.RexDynamicParam}.
@@ -35,7 +36,7 @@ public class DynamicParamExpression implements Expression {
      * @param index Index.
      */
     public DynamicParamExpression(RelDataType type, int index) {
-        this.type = DataType.fromType(type);
+        this.type = Types.fromType(type);
         this.index = index;
     }
 
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
index 176a1a0..3ee1784 100644
--- 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
@@ -81,7 +81,7 @@ public class ExpToRexTranslator implements ExpImplementor<RexNode> {
 
     /** {@inheritDoc} */
     @Override public RexNode implement(CallExpression exp) {
-        return builder.makeCall(op(exp.name(), exp.syntax()), translate(exp.operands()));
+        return builder.makeCall(exp.dataType().toRelDataType(builder.getTypeFactory()), op(exp.name(), exp.syntax()), translate(exp.operands()));
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/serialize/expression/InputRefExpression.java
index 1facc80..840b198 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/serialize/expression/InputRefExpression.java
@@ -19,6 +19,7 @@ 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.RexInputRef}.
@@ -35,7 +36,7 @@ public class InputRefExpression implements Expression {
      * @param index Index.
      */
     public InputRefExpression(RelDataType type, int index) {
-        this.type = DataType.fromType(type);
+        this.type = Types.fromType(type);
         this.index = 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/serialize/expression/LiteralExpression.java
index 7ba50b2..ccac4ea 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/serialize/expression/LiteralExpression.java
@@ -19,24 +19,24 @@ 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.RexLiteral}.
  */
-@SuppressWarnings("rawtypes")
 public class LiteralExpression implements Expression {
     /** */
     private final DataType type;
 
     /** */
-    private final Comparable value;
+    private final Comparable<?> value;
 
     /**
      * @param type Data type.
      * @param value Value.
      */
-    public LiteralExpression(RelDataType type, Comparable value) {
-        this.type = DataType.fromType(type);
+    public LiteralExpression(RelDataType type, Comparable<?> value) {
+        this.type = Types.fromType(type);
         this.value = value;
     }
 
@@ -50,7 +50,7 @@ public class LiteralExpression implements Expression {
     /**
      * @return Value.
      */
-    public Comparable value() {
+    public Comparable<?> value() {
         return value;
     }
 
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
index a152180..521a3229 100644
--- 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
@@ -19,6 +19,7 @@ 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}.
@@ -35,7 +36,7 @@ public class LocalRefExpression implements Expression {
      * @param index Index.
      */
... 7670 lines suppressed ...