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

[ignite-3] branch main updated: IGNITE-14835 Calcite SQL engine. Move execution related code to Ignite 3.0 (#182)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 972672f  IGNITE-14835 Calcite SQL engine. Move execution related code to Ignite 3.0 (#182)
972672f is described below

commit 972672ffcd7b0b4bdc31b3511fdcb90f51cc99ec
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Wed Jun 23 11:29:49 2021 +0300

    IGNITE-14835 Calcite SQL engine. Move execution related code to Ignite 3.0 (#182)
---
 .../internal/processors/query/calcite/Stubs.java   |    6 -
 .../query/calcite/exec/AbstractIndexScan.java      |  107 +
 .../query/calcite/exec/ArrayRowHandler.java        |   77 +
 .../ExecutionCancelledException.java}              |   24 +-
 .../query/calcite/exec/ExecutionContext.java       |  292 +++
 .../query/calcite/exec/PlannerHelper.java          |  229 ++
 .../query/calcite/exec/QueryTaskExecutor.java      |   47 +
 .../query/calcite/exec/QueryTaskExecutorImpl.java  |   95 +
 .../processors/query/calcite/exec/RowHandler.java  |   73 +
 .../query/calcite/exec/RuntimeHashIndex.java       |  113 +
 .../calcite/{Stubs.java => exec/RuntimeIndex.java} |   32 +-
 .../query/calcite/exec/RuntimeTreeIndex.java       |  196 ++
 .../query/calcite/exec/TableFunctionScan.java      |   48 +
 .../calcite/{Stubs.java => exec/TreeIndex.java}    |   37 +-
 .../query/calcite/exec/exp/CallImplementor.java    |   42 +
 .../query/calcite/exec/exp/ConverterUtils.java     |  460 ++++
 .../query/calcite/exec/exp/ExpressionFactory.java  |  125 +
 .../calcite/exec/exp/ExpressionFactoryImpl.java    |  509 ++++
 .../calcite/exec/exp/IgniteBuiltInMethod.java      |   45 +
 .../query/calcite/exec/exp/IgniteSqlFunctions.java |  154 ++
 .../calcite/exec/exp/ImplementableFunction.java    |   36 +
 .../query/calcite/exec/exp/NotNullImplementor.java |   43 +
 .../exec/exp/ReflectiveCallNotNullImplementor.java |   82 +
 .../query/calcite/exec/exp/RexImpTable.java        | 2574 ++++++++++++++++++++
 .../query/calcite/exec/exp/RexToLixTranslator.java | 1312 ++++++++++
 .../calcite/{Stubs.java => exec/exp/Scalar.java}   |   24 +-
 .../{Stubs.java => exec/exp/agg/Accumulator.java}  |   32 +-
 .../exp/agg/AccumulatorWrapper.java}               |   23 +-
 .../query/calcite/exec/exp/agg/Accumulators.java   | 1068 ++++++++
 .../calcite/exec/exp/agg/AccumulatorsFactory.java  |  330 +++
 .../exp/agg/AggregateType.java}                    |   30 +-
 .../query/calcite/exec/exp/agg/GroupKey.java       |  101 +
 .../query/calcite/exec/rel/AbstractNode.java       |  202 ++
 .../query/calcite/exec/rel/AbstractSetOpNode.java  |  367 +++
 .../exec/rel/CorrelatedNestedLoopJoinNode.java     |  479 ++++
 .../{Stubs.java => exec/rel/Downstream.java}       |   43 +-
 .../query/calcite/exec/rel/FilterNode.java         |  145 ++
 .../query/calcite/exec/rel/HashAggregateNode.java  |  372 +++
 .../query/calcite/exec/rel/IndexSpoolNode.java     |  205 ++
 .../query/calcite/exec/rel/IntersectNode.java      |  118 +
 .../query/calcite/exec/rel/LimitNode.java          |  132 +
 .../calcite/{Stubs.java => exec/rel/Mailbox.java}  |   36 +-
 .../query/calcite/exec/rel/MergeJoinNode.java      | 1134 +++++++++
 .../query/calcite/exec/rel/MinusNode.java          |  102 +
 .../query/calcite/exec/rel/NestedLoopJoinNode.java |  812 ++++++
 .../processors/query/calcite/exec/rel/Node.java    |   81 +
 .../query/calcite/exec/rel/ProjectNode.java        |   84 +
 .../query/calcite/exec/rel/RootNode.java           |  283 +++
 .../query/calcite/exec/rel/ScanNode.java           |  130 +
 .../{Stubs.java => exec/rel/SingleNode.java}       |   29 +-
 .../query/calcite/exec/rel/SortAggregateNode.java  |  316 +++
 .../query/calcite/exec/rel/SortNode.java           |  148 ++
 .../query/calcite/exec/rel/TableSpoolNode.java     |  160 ++
 .../query/calcite/exec/rel/UnionAllNode.java       |   97 +
 .../calcite/metadata/FragmentDescription.java      |   75 +
 .../calcite/metadata/cost/IgniteCostFactory.java   |    4 +-
 .../calcite/prepare/ddl/ColumnDefinition.java      |    2 +-
 .../processors/query/calcite/util/Commons.java     |   27 +-
 .../query/calcite/util/FilteringIterator.java      |   79 +
 .../query/calcite/util/IgniteMethod.java           |   25 +
 .../processors/query/calcite/util/RexUtils.java    |    8 +-
 .../calcite/util/StripedThreadPoolExecutor.java    |  226 ++
 .../{Stubs.java => util/TransformingIterator.java} |   37 +-
 .../processors/query/calcite/util/TypeUtils.java   |  111 +
 .../query/calcite/exec/RuntimeTreeIndexTest.java   |  199 ++
 .../calcite/exec/rel/AbstractExecutionTest.java    |  226 ++
 .../exec/rel/AbstractSetOpExecutionTest.java       |  166 ++
 .../query/calcite/exec/rel/BaseAggregateTest.java  |  552 +++++
 .../query/calcite/exec/rel/ExecutionTest.java      |  667 +++++
 .../exec/rel/HashAggregateExecutionTest.java       |  135 +
 .../rel/HashAggregateSingleGroupExecutionTest.java |  497 ++++
 .../exec/rel/HashIndexSpoolExecutionTest.java      |  167 ++
 .../calcite/exec/rel/IntersectExecutionTest.java   |   83 +
 .../calcite/exec/rel/MergeJoinExecutionTest.java   |  378 +++
 .../query/calcite/exec/rel/MinusExecutionTest.java |   83 +
 .../exec/rel/NestedLoopJoinExecutionTest.java      |  363 +++
 .../exec/rel/SortAggregateExecutionTest.java       |  141 ++
 .../calcite/exec/rel/TableSpoolExecutionTest.java  |  152 ++
 .../exec/rel/TreeIndexSpoolExecutionTest.java      |  194 ++
 .../apache/ignite/internal/util/ArrayUtils.java    |    2 +
 .../apache/ignite/internal/util/IgniteUtils.java   |   30 +-
 .../internal/testframework/IgniteTestUtils.java    |   71 +
 .../internal/util/IgniteIntListSelfTest.java       |  159 ++
 83 files changed, 18485 insertions(+), 215 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
index ce7d926..093146c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
@@ -23,12 +23,6 @@ public class Stubs {
         return args == null ? 0 : args.length;
     }
 
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
-
-    /** */
     public static boolean boolFoo(Object... args) {
         return args == null;
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java
new file mode 100644
index 0000000..fd61d52
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Iterator;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.util.FilteringIterator;
+import org.apache.ignite.internal.processors.query.calcite.util.TransformingIterator;
+import org.apache.ignite.lang.IgniteInternalException;
+
+/**
+ * Abstract index scan.
+ */
+public abstract class AbstractIndexScan<Row, IdxRow> implements Iterable<Row>, AutoCloseable {
+    /** */
+    private final TreeIndex<IdxRow> idx;
+
+    /** Additional filters. */
+    private final Predicate<Row> filters;
+
+    /** Lower index scan bound. */
+    private final Supplier<Row> lowerBound;
+
+    /** Upper index scan bound. */
+    private final Supplier<Row> upperBound;
+
+    /** */
+    private final Function<Row, Row> rowTransformer;
+
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    protected final RelDataType rowType;
+
+    /**
+     * @param ectx Execution context.
+     * @param idx Physical index.
+     * @param filters Additional filters.
+     * @param lowerBound Lower index scan bound.
+     * @param upperBound Upper index scan bound.
+     */
+    protected AbstractIndexScan(
+        ExecutionContext<Row> ectx,
+        RelDataType rowType,
+        TreeIndex<IdxRow> idx,
+        Predicate<Row> filters,
+        Supplier<Row> lowerBound,
+        Supplier<Row> upperBound,
+        Function<Row, Row> rowTransformer
+    ) {
+        this.ectx = ectx;
+        this.rowType = rowType;
+        this.idx = idx;
+        this.filters = filters;
+        this.lowerBound = lowerBound;
+        this.upperBound = upperBound;
+        this.rowTransformer = rowTransformer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized Iterator<Row> iterator() {
+        IdxRow lower = lowerBound == null ? null : row2indexRow(lowerBound.get());
+        IdxRow upper = upperBound == null ? null : row2indexRow(upperBound.get());
+
+        Iterator<Row> it = new TransformingIterator<>(
+            idx.find(lower, upper),
+            this::indexRow2Row
+        );
+
+        it = new FilteringIterator<>(it, filters);
+
+        if (rowTransformer != null)
+            it = new TransformingIterator<>(it, rowTransformer);
+
+        return it;
+    }
+
+    /** */
+    protected abstract IdxRow row2indexRow(Row bound);
+
+    /** */
+    protected abstract Row indexRow2Row(IdxRow idxRow) throws IgniteInternalException;
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        // No-op.
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
new file mode 100644
index 0000000..f6b9274
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.lang.reflect.Type;
+
+import org.apache.ignite.internal.util.ArrayUtils;
+
+/**
+ * Handler for rows that implemented as a simple objects array.
+ */
+public class ArrayRowHandler implements RowHandler<Object[]> {
+    /** */
+    public static final RowHandler<Object[]> INSTANCE = new ArrayRowHandler();
+
+    /** */
+    private ArrayRowHandler() {}
+
+    /** {@inheritDoc} */
+    @Override public Object get(int field, Object[] row) {
+        return row[field];
+    }
+
+    /** {@inheritDoc} */
+    @Override public void set(int field, Object[] row, Object val) {
+        row[field] = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object[] concat(Object[] left, Object[] right) {
+        return ArrayUtils.concat(left, right);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int columnCount(Object[] row) {
+        return row.length;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowFactory<Object[]> factory(Type... types) {
+        int rowLen = types.length;
+
+        return new RowFactory<>() {
+            /** {@inheritDoc} */
+            @Override public RowHandler<Object[]> handler() {
+                return ArrayRowHandler.this;
+            }
+
+            /** {@inheritDoc} */
+            @Override public Object[] create() {
+                return new Object[rowLen];
+            }
+
+            /** {@inheritDoc} */
+            @Override public Object[] create(Object... fields) {
+                assert fields.length == rowLen;
+
+                return fields;
+            }
+        };
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
index ce7d926..ce2e4c7 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
@@ -14,27 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+import org.apache.ignite.lang.IgniteInternalCheckedException;
 
-    /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
-
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/** */
+public class ExecutionCancelledException extends IgniteInternalCheckedException {
 }
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..96ad599
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
@@ -0,0 +1,292 @@
+/*
+ * 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.TimeZone;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactoryImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.checkRange;
+
+/**
+ * Runtime context allowing access to the tables in a database.
+ */
+public class ExecutionContext<Row> implements DataContext {
+    /** */
+    private static final TimeZone TIME_ZONE = TimeZone.getDefault(); // TODO DistributedSqlConfiguration#timeZone
+
+    /** */
+    private final UUID qryId;
+
+    /** */
+    private final PlanningContext ctx;
+
+    /** */
+    private final FragmentDescription fragmentDesc;
+
+    /** */
+    private final Map<String, Object> params;
+
+    /** */
+    private final QueryTaskExecutor executor;
+
+    /** */
+    private final RowHandler<Row> handler;
+
+    /** */
+    private final ExpressionFactory<Row> expressionFactory;
+
+    /** */
+    private final AtomicBoolean cancelFlag = new AtomicBoolean();
+
+    /** */
+    private Object[] correlations = new Object[16];
+
+    /**
+     * @param executor Task executor.
+     * @param ctx Parent context.
+     * @param qryId Query ID.
+     * @param fragmentDesc Partitions information.
+     * @param handler Row handler.
+     * @param params Parameters.
+     */
+    @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+    public ExecutionContext(
+        QueryTaskExecutor executor,
+        PlanningContext ctx,
+        UUID qryId,
+        FragmentDescription fragmentDesc,
+        RowHandler<Row> handler,
+        Map<String, Object> params
+    ) {
+        this.executor = executor;
+        this.ctx = ctx;
+        this.qryId = qryId;
+        this.fragmentDesc = fragmentDesc;
+        this.handler = handler;
+        this.params = params;
+
+        expressionFactory = new ExpressionFactoryImpl<>(this, ctx.typeFactory(), ctx.conformance());
+    }
+
+    /**
+     * @return Parent context.
+     */
+    public PlanningContext planningContext() {
+        return ctx;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return qryId;
+    }
+
+    /**
+     * @return Fragment ID.
+     */
+    public long fragmentId() {
+        return fragmentDesc.fragmentId();
+    }
+
+    /**
+     * @return Target mapping.
+     */
+    public ColocationGroup target() {
+        return fragmentDesc.target();
+    }
+
+    /**
+     * @param exchangeId ExchangeId to find remote nodes for.
+     * @return Remote nodes for given exchangeId.
+     */
+    public List<UUID> remotes(long exchangeId) {
+        return fragmentDesc.remotes().get(exchangeId);
+    }
+
+    /**
+     * @param sourceId SourceId to find colocation group for.
+     * @return Colocation group for given sourceId.
+     */
+    public ColocationGroup group(long sourceId) {
+        return fragmentDesc.mapping().findGroup(sourceId);
+    }
+
+    /**
+     * @return Keep binary flag.
+     */
+    public boolean keepBinary() {
+        return true; // TODO
+    }
+
+    /**
+     * @return Handler to access row fields.
+     */
+    public RowHandler<Row> rowHandler() {
+        return handler;
+    }
+
+    /**
+     * @return Expression factory.
+     */
+    public ExpressionFactory<Row> expressionFactory() {
+        return expressionFactory;
+    }
+
+    /**
+     * @return Originating node ID.
+     */
+    public UUID originatingNodeId() {
+        return planningContext().originatingNodeId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public SchemaPlus getRootSchema() {
+        return ctx.schema();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteTypeFactory getTypeFactory() {
+        return ctx.typeFactory();
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryProvider getQueryProvider() {
+        return null; // TODO
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object get(String name) {
+        if (Variable.CANCEL_FLAG.camelName.equals(name))
+            return cancelFlag;
+        if (Variable.TIME_ZONE.camelName.equals(name))
+            return TIME_ZONE; // TODO DistributedSqlConfiguration#timeZone
+        if (name.startsWith("?"))
+            return TypeUtils.toInternal(this, params.get(name));
+
+        return params.get(name);
+    }
+
+    /**
+     * Gets correlated value.
+     *
+     * @param id Correlation ID.
+     * @return Correlated value.
+     */
+    public @NotNull Object getCorrelated(int id) {
+        checkRange(correlations, id);
+
+        return correlations[id];
+    }
+
+    /**
+     * Sets correlated value.
+     *
+     * @param id Correlation ID.
+     * @param value Correlated value.
+     */
+    public void setCorrelated(@NotNull Object value, int id) {
+        correlations = Commons.ensureCapacity(correlations, id + 1);
+
+        correlations[id] = value;
+    }
+
+    /**
+     * Executes a query task.
+     *
+     * @param task Query task.
+     */
+    public void execute(RunnableX task, Consumer<Throwable> onError) {
+        if (isCancelled())
+            return;
+
+        executor.execute(qryId, fragmentId(), () -> {
+            try {
+                task.run();
+            }
+            catch (Throwable e) {
+                onError.accept(e);
+
+                throw new IgniteInternalException("Unexpected exception", e);
+            }
+        });
+    }
+
+    /**
+     * Submits a Runnable task for execution and returns a Future
+     * representing that task. The Future's {@code get} method will
+     * return {@code null} upon <em>successful</em> completion.
+     *
+     * @param task the task to submit.
+     * @return a {@link CompletableFuture} representing pending task
+     */
+    public CompletableFuture<?> submit(RunnableX task, Consumer<Throwable> onError) {
+        assert !isCancelled() : "Call submit after execution was cancelled.";
+
+        return executor.submit(qryId, fragmentId(), () -> {
+            try {
+                task.run();
+            }
+            catch (Throwable e) {
+                onError.accept(e);
+
+                throw new IgniteInternalException("Unexpected exception", e);
+            }
+        });
+    }
+
+    /** */
+    @FunctionalInterface
+    public interface RunnableX {
+        /** */
+        void run() throws Exception;
+    }
+
+    /**
+     * Sets cancel flag, returns {@code true} if flag was changed by this call.
+     *
+     * @return {@code True} if flag was changed by this call.
+     */
+    public boolean cancel() {
+        return !cancelFlag.get() && cancelFlag.compareAndSet(false, true);
+    }
+
+    /** */
+    public boolean isCancelled() {
+        return cancelFlag.get();
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PlannerHelper.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PlannerHelper.java
new file mode 100644
index 0000000..4d53326
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PlannerHelper.java
@@ -0,0 +1,229 @@
+/*
+ * 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.List;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.core.Spool;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgniteRelShuttle;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerPhase;
+import org.apache.ignite.internal.processors.query.calcite.rel.AbstractIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableSpool;
+import org.apache.ignite.internal.processors.query.calcite.schema.ColumnDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.HintUtils;
+import org.apache.ignite.lang.IgniteLogger;
+
+/** */
+public class PlannerHelper {
+    /**
+     * Default constructor.
+     */
+    private PlannerHelper() {
+
+    }
+
+    /**
+     * @param sqlNode Sql node.
+     * @param planner Planner.
+     * @param log Logger.
+     */
+    public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteLogger log) {
+        try {
+            // Convert to Relational operators graph
+            RelRoot root = planner.rel(sqlNode);
+
+            RelNode rel = root.rel;
+
+            if (HintUtils.containsDisabledRules(root.hints))
+                planner.setDisabledRules(HintUtils.disabledRules(root.hints));
+
+            // Transformation chain
+            rel = planner.transform(PlannerPhase.HEURISTIC_OPTIMIZATION, rel.getTraitSet(), rel);
+
+            RelTraitSet desired = rel.getCluster().traitSet()
+                .replace(IgniteConvention.INSTANCE)
+                .replace(IgniteDistributions.single())
+                .replace(root.collation == null ? RelCollations.EMPTY : root.collation)
+                .simplify();
+
+            IgniteRel igniteRel = planner.transform(PlannerPhase.OPTIMIZATION, desired, rel);
+
+            if (!root.isRefTrivial()) {
+                final List<RexNode> projects = new ArrayList<>();
+                final RexBuilder rexBuilder = igniteRel.getCluster().getRexBuilder();
+
+                for (int field : Pair.left(root.fields))
+                    projects.add(rexBuilder.makeInputRef(igniteRel, field));
+
+                igniteRel = new IgniteProject(igniteRel.getCluster(), desired, igniteRel, projects, root.validatedRowType);
+            }
+
+            if (sqlNode.isA(ImmutableSet.of(SqlKind.INSERT, SqlKind.UPDATE, SqlKind.MERGE)))
+                igniteRel = new FixDependentModifyNodeShuttle().visit(igniteRel);
+
+            return igniteRel;
+        }
+        catch (Throwable ex) {
+            log.error("Unexpected error at query optimizer.", ex);
+            log.error(planner.dump());
+
+            throw ex;
+        }
+    }
+
+    /**
+     * This shuttle analyzes a relational tree and inserts an eager spool node
+     * just under the TableModify node in case latter depends upon a table used
+     * to query the data for modify node to avoid the double processing
+     * of the retrieved rows.
+     * <p/>
+     * It considers two cases: <ol>
+     *     <li>
+     *         Modify node produces rows to insert, then a spool is required.
+     *     </li>
+     *     <li>
+     *         Modify node updates rows only, then a spool is required if 1) we
+     *         are scaning an index and 2) any of the indexed column is updated
+     *         by modify node.
+     *     </li>
+     * <ol/>
+     *
+     */
+    private static class FixDependentModifyNodeShuttle extends IgniteRelShuttle {
+        /**
+         * Flag indicates whether we should insert a spool or not.
+         */
+        private boolean spoolNeeded;
+
+        /** Current modify node. */
+        private IgniteTableModify modifyNode;
+
+        /** {@inheritDoc} */
+        @Override public IgniteRel visit(IgniteTableModify rel) {
+            assert modifyNode == null;
+
+            modifyNode = rel;
+
+            if (rel.isDelete())
+                return rel;
+
+            processNode(rel);
+
+            if (spoolNeeded) {
+                IgniteTableSpool spool = new IgniteTableSpool(
+                    rel.getCluster(),
+                    rel.getInput().getTraitSet(),
+                    Spool.Type.EAGER,
+                    rel.getInput()
+                );
+
+                rel.replaceInput(0, spool);
+            }
+
+            return rel;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteRel visit(IgniteTableScan rel) {
+            return processScan(rel);
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteRel visit(IgniteIndexScan rel) {
+            return processScan(rel);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected IgniteRel processNode(IgniteRel rel) {
+            List<IgniteRel> inputs = Commons.cast(rel.getInputs());
+
+            for (int i = 0; i < inputs.size(); i++) {
+                if (spoolNeeded)
+                    break;
+
+                visitChild(rel, i, inputs.get(i));
+            }
+
+            return rel;
+        }
+
+        /**
+         * Process a scan node and raise a {@link #spoolNeeded flag} if needed.
+         *
+         * @param scan TableScan to analize.
+         * @return The input rel.
+         */
+        private IgniteRel processScan(TableScan scan) {
+            IgniteTable tbl = modifyNode != null ? modifyNode.getTable().unwrap(IgniteTable.class) : null;
+
+            if (tbl == null || scan.getTable().unwrap(IgniteTable.class) != tbl)
+                return (IgniteRel)scan;
+
+            if (modifyNodeInsertsData()) {
+                spoolNeeded = true;
+
+                return (IgniteRel)scan;
+            }
+
+            // for update-only node the spool needed if any of the updated
+            // column is part of the index we are going to scan
+            if (scan instanceof IgniteTableScan)
+                return (IgniteRel)scan;
+
+            ImmutableSet<Integer> indexedCols = ImmutableSet.copyOf(
+                tbl.getIndex(((AbstractIndexScan)scan).indexName()).collation().getKeys());
+
+            spoolNeeded = modifyNode.getUpdateColumnList().stream()
+                .map(tbl.descriptor()::columnDescriptor)
+                .map(ColumnDescriptor::fieldIndex)
+                .anyMatch(indexedCols::contains);
+
+            return (IgniteRel)scan;
+        }
+
+        /**
+         * @return {@code true} in case {@link #modifyNode} produces any insert.
+         */
+        private boolean modifyNodeInsertsData() {
+            return modifyNode.isInsert(); // MERGE should be analyzed too
+            // but currently it is not implemented
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
new file mode 100644
index 0000000..b4a5fb2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
@@ -0,0 +1,47 @@
+/*
+ * 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 java.util.concurrent.CompletableFuture;
+
+/**
+ *
+ */
+public interface QueryTaskExecutor {
+    /**
+     * Executes a query task in a thread, responsible for particular query fragment.
+     *
+     * @param qryId Query ID.
+     * @param fragmentId Fragment ID.
+     * @param qryTask Query task.
+     */
+    void execute(UUID qryId, long fragmentId, Runnable qryTask);
+
+    /**
+     * Returns a new CompletableFuture that is asynchronously completed
+     * by a task running in the given executor after it runs the given
+     * action.
+     *
+     * @param qryId Id of the query this task created for.
+     * @param fragmentId Id of the particular fragment this task created for.
+     * @param qryTask The task to submit.
+     * @return the new CompletableFuture
+     */
+    CompletableFuture<?> submit(UUID qryId, long fragmentId, Runnable qryTask);
+}
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..abcad0a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
@@ -0,0 +1,95 @@
+/*
+ * 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 java.util.concurrent.CompletableFuture;
+
+import org.apache.ignite.internal.processors.query.calcite.util.StripedThreadPoolExecutor;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteLogger;
+
+/** */
+public class QueryTaskExecutorImpl implements QueryTaskExecutor, Thread.UncaughtExceptionHandler {
+    /** */
+    private static final IgniteLogger LOG = IgniteLogger.forClass(QueryTaskExecutorImpl.class);
+
+    /** */
+    private StripedThreadPoolExecutor stripedThreadPoolExecutor;
+
+    /** */
+    private Thread.UncaughtExceptionHandler eHnd;
+
+    /**
+     * @param stripedThreadPoolExecutor Executor.
+     */
+    public void stripedThreadPoolExecutor(StripedThreadPoolExecutor stripedThreadPoolExecutor) {
+        this.stripedThreadPoolExecutor = stripedThreadPoolExecutor;
+    }
+
+    /**
+     * @param eHnd Uncaught exception handler.
+     */
+    public void exceptionHandler(Thread.UncaughtExceptionHandler eHnd) {
+        this.eHnd = eHnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(UUID qryId, long fragmentId, Runnable qryTask) {
+        stripedThreadPoolExecutor.execute(
+            () -> {
+                try {
+                    qryTask.run();
+                }
+                catch (Throwable e) {
+                    LOG.warn("Uncaught exception", e);
+
+                    /*
+                     * No exceptions are rethrown here to preserve the current thread from being destroyed,
+                     * because other queries may be pinned to the current thread id.
+                     * However, unrecoverable errors must be processed by FailureHandler.
+                     */
+                    uncaughtException(Thread.currentThread(), e);
+                }
+            },
+            hash(qryId, fragmentId)
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public CompletableFuture<?> submit(UUID qryId, long fragmentId, Runnable qryTask) {
+        return stripedThreadPoolExecutor.submit(qryTask, hash(qryId, fragmentId));
+    }
+
+    /** Releases resources. */
+    public void tearDown() {
+        stripedThreadPoolExecutor.shutdownNow();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void uncaughtException(Thread t, Throwable e) {
+        if (eHnd != null)
+            eHnd.uncaughtException(t, e);
+    }
+
+    /** */
+    private static int hash(UUID qryId, long fragmentId) {
+        // inlined Objects.hash(...)
+        return IgniteUtils.safeAbs(31 * (31 + (qryId != null ? qryId.hashCode() : 0)) + Long.hashCode(fragmentId));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
new file mode 100644
index 0000000..6dac05c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.lang.reflect.Type;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/**
+ * Universal accessor and mutator for rows. It also has factory methods.
+ */
+public interface RowHandler<Row> {
+    /** */
+    Object get(int field, Row row);
+
+    /** */
+    void set(int field, Row row, Object val);
+
+    /** */
+    Row concat(Row left, Row right);
+
+    /** */
+    int columnCount(Row row);
+
+    /** */
+    default RowFactory<Row> factory(IgniteTypeFactory typeFactory, RelDataType rowType) {
+        if (rowType.isStruct())
+            return factory(typeFactory, RelOptUtil.getFieldTypeList(rowType));
+
+        return factory(typeFactory.getJavaClass(rowType));
+    }
+
+    /** */
+    default RowFactory<Row> factory(IgniteTypeFactory typeFactory, List<RelDataType> fieldTypes) {
+        Type[] types = new Type[fieldTypes.size()];
+        for (int i = 0; i < fieldTypes.size(); i++)
+            types[i] = typeFactory.getJavaClass(fieldTypes.get(i));
+
+        return factory(types);
+    }
+
+    RowFactory<Row> factory(Type... types);
+
+    @SuppressWarnings("PublicInnerClass")
+    interface RowFactory<Row> {
+        /** */
+        RowHandler<Row> handler();
+
+        /** */
+        Row create();
+
+        /** */
+        Row create(Object... fields);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java
new file mode 100644
index 0000000..03abac6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java
@@ -0,0 +1,113 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Supplier;
+
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Runtime hash index based on on-heap hash map.
+ */
+public class RuntimeHashIndex<Row> implements RuntimeIndex<Row> {
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    private final ImmutableBitSet keys;
+
+    /** Rows. */
+    private HashMap<GroupKey, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeHashIndex(
+        ExecutionContext<Row> ectx,
+        ImmutableBitSet keys
+    ) {
+        this.ectx = ectx;
+
+        assert !nullOrEmpty(keys);
+
+        this.keys = keys;
+        rows = new HashMap<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void push(Row r) {
+        List<Row> eqRows = rows.computeIfAbsent(key(r), k -> new ArrayList<>());
+
+        eqRows.add(r);
+    }
+
+    /** */
+    @Override public void close() {
+        rows.clear();
+    }
+
+    /** */
+    public Iterable<Row> scan(Supplier<Row> searchRow) {
+        return new IndexScan(searchRow);
+    }
+
+    /** */
+    private GroupKey key(Row r) {
+        GroupKey.Builder b = GroupKey.builder(keys.cardinality());
+
+        for (Integer field : keys)
+            b.add(ectx.rowHandler().get(field, r));
+
+        return b.build();
+    }
+
+    /**
+     *
+     */
+    private class IndexScan implements Iterable<Row>, AutoCloseable {
+        /** Search row. */
+        private final Supplier<Row> searchRow;
+
+        /**
+         * @param searchRow Search row.
+         */
+        IndexScan(Supplier<Row> searchRow) {
+            this.searchRow = searchRow;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @NotNull @Override public Iterator<Row> iterator() {
+            List<Row> eqRows = rows.get(key(searchRow.get()));
+
+            return eqRows == null ? Collections.emptyIterator() : eqRows.iterator();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeIndex.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeIndex.java
index ce7d926..01d257c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeIndex.java
@@ -14,27 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
-
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
-
-    /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
-
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ * Runtime index interface.
+ * The temporary index is built and available only on query execution. Not stored at the schema.
+ */
+public interface RuntimeIndex<Row> extends AutoCloseable {
+    /**
+     * Add row to index.
+     */
+    void push(Row r);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
new file mode 100644
index 0000000..474f5dc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
@@ -0,0 +1,196 @@
+/*
+ * 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.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> implements RuntimeIndex<Row>, TreeIndex<Row> {
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    protected final Comparator<Row> comp;
+
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        this.ectx = ectx;
+        this.comp = comp;
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void push(Row r) {
+        List<Row> newEqRows = new ArrayList<>();
+
+        List<Row> eqRows = rows.putIfAbsent(r, newEqRows);
+
+        if (eqRows != null)
+            eqRows.add(r);
+        else
+            newEqRows.add(r);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        rows.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor<Row> find(Row lower, Row upper) {
+        int firstCol = first(collation.getKeys());
+
+        if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) != null)
+            return new CursorImpl(rows.subMap(lower, true, upper, true));
+        else if (ectx.rowHandler().get(firstCol, lower) == null && ectx.rowHandler().get(firstCol, upper) != null)
+            return new CursorImpl(rows.headMap(upper, true));
+        else if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) == null)
+            return new CursorImpl(rows.tailMap(lower, true));
+        else
+            return new CursorImpl(rows);
+    }
+
+    /**
+     * Creates iterable on the index.
+     */
+    public Iterable<Row> scan(
+        ExecutionContext<Row> ectx,
+        RelDataType rowType,
+        Predicate<Row> filter,
+        Supplier<Row> lowerBound,
+        Supplier<Row> upperBound
+    ) {
+        return new IndexScan(rowType, this, filter, lowerBound, upperBound);
+    }
+
+    /**
+     *
+     */
+    private class CursorImpl implements Cursor<Row> {
+        /** Sub map iterator. */
+        private final Iterator<Map.Entry<Row, List<Row>>> mapIt;
+
+        /** Iterator over rows with equal index keys. */
+        private Iterator<Row> listIt;
+
+        /** */
+        private Row row;
+
+        /** */
+        CursorImpl(SortedMap<Row, List<Row>> subMap) {
+            mapIt = subMap.entrySet().iterator();
+            listIt = null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Row next() throws IgniteInternalException {
+            if (!hasNext())
+                throw new NoSuchElementException();
+
+            advance();
+
+            return listIt.next();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return listIt != null && listIt.hasNext() || mapIt.hasNext();
+        }
+
+        /** */
+        private void advance() {
+            if (listIt == null || !listIt.hasNext())
+                listIt = mapIt.next().getValue().iterator();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws Exception {
+        }
+
+        /** {@inheritDoc} */
+        @NotNull @Override public Iterator<Row> iterator() {
+            return this;
+        }
+    }
+
+    /**
+     *
+     */
+    private class IndexScan extends AbstractIndexScan<Row, Row> {
+        /**
+         * @param rowType Row type.
+         * @param idx Physical index.
+         * @param filter Additional filters.
+         * @param lowerBound Lower index scan bound.
+         * @param upperBound Upper index scan bound.
+         */
+        IndexScan(
+            RelDataType rowType,
+            TreeIndex<Row> idx,
+            Predicate<Row> filter,
+            Supplier<Row> lowerBound,
+            Supplier<Row> upperBound) {
+            super(RuntimeTreeIndex.this.ectx, rowType, idx, filter, lowerBound, upperBound, null);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Row row2indexRow(Row bound) {
+            return bound;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Row indexRow2Row(Row row) {
+            return row;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableFunctionScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableFunctionScan.java
new file mode 100644
index 0000000..9744a04
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableFunctionScan.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Iterator;
+import java.util.function.Supplier;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.TransformingIterator;
+import org.jetbrains.annotations.NotNull;
+
+/** */
+public class TableFunctionScan<Row> implements Iterable<Row> {
+    /** */
+    private final Supplier<Iterable<Object[]>> dataSupplier;
+
+    /** */
+    private final RowFactory<Row> rowFactory;
+
+    /** */
+    public TableFunctionScan(
+        Supplier<Iterable<Object[]>> dataSupplier,
+        RowFactory<Row> rowFactory
+    ) {
+        this.dataSupplier = dataSupplier;
+        this.rowFactory = rowFactory;
+    }
+
+    /** {@inheritDoc} */
+    @NotNull @Override public Iterator<Row> iterator() {
+        return new TransformingIterator<>(dataSupplier.get().iterator(), rowFactory::create);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TreeIndex.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TreeIndex.java
index ce7d926..9911df0 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TreeIndex.java
@@ -14,27 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
+package org.apache.ignite.internal.processors.query.calcite.exec;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+import org.apache.ignite.internal.util.Cursor;
 
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
-
-    /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
-
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+/**
+ * Tree index interface.
+ *
+ * @param <R> Indexing row type.
+ */
+public interface TreeIndex<R> {
+    /**
+     * Index lookup method.
+     *
+     * @param lower Lower bound.
+     * @param upper Upper bound.
+     * @return Cursor over the rows within bounds.
+     */
+    Cursor<R> find(R lower, R upper);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallImplementor.java
new file mode 100644
index 0000000..6328938
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallImplementor.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexCall;
+
+/**
+ * Implements a call via given translator.
+ *
+ * @see org.apache.calcite.schema.ScalarFunction
+ * @see org.apache.calcite.schema.TableFunction
+ * @see RexImpTable
+ */
+public interface CallImplementor {
+    /**
+     * Implements a call.
+     *
+     * @param translator Translator for the call
+     * @param call Call that should be implemented
+     * @param nullAs The desired mode of {@code null} translation
+     * @return Translated call
+     */
+    Expression implement(
+        RexToLixTranslator translator,
+        RexCall call,
+        RexImpTable.NullAs nullAs);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ConverterUtils.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ConverterUtils.java
new file mode 100644
index 0000000..69cf03c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ConverterUtils.java
@@ -0,0 +1,460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+
+/** */
+public class ConverterUtils {
+    /** */
+    private ConverterUtils() {
+    }
+
+    /**
+     * In Calcite, {@code java.sql.Date} and {@code java.sql.Time} are stored as {@code Integer} type, {@code
+     * java.sql.Timestamp} is stored as {@code Long} type.
+     *
+     * @param operand Operand that should be converted.
+     * @param targetType Required type.
+     *
+     * @return New expression of required type.
+     */
+    static Expression toInternal(Expression operand, Type targetType) {
+        return toInternal(operand, operand.getType(), targetType);
+    }
+
+    /** */
+    private static Expression toInternal(Expression operand,
+        Type fromType, Type targetType) {
+        if (fromType == java.sql.Date.class) {
+            if (targetType == int.class)
+              return Expressions.call(BuiltInMethod.DATE_TO_INT.method, operand);
+            else if (targetType == Integer.class)
+              return Expressions.call(BuiltInMethod.DATE_TO_INT_OPTIONAL.method, operand);
+        }
+        else if (fromType == java.sql.Time.class) {
+            if (targetType == int.class)
+              return Expressions.call(BuiltInMethod.TIME_TO_INT.method, operand);
+            else if (targetType == Integer.class)
+              return Expressions.call(BuiltInMethod.TIME_TO_INT_OPTIONAL.method, operand);
+        }
+        else if (fromType == java.sql.Timestamp.class) {
+            if (targetType == long.class)
+              return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG.method, operand);
+            else if (targetType == Long.class)
+              return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL.method, operand);
+        }
+        return operand;
+    }
+
+    /** Converts from internal representation to JDBC representation used by
+     * arguments of user-defined functions. For example, converts date values from
+     * {@code int} to {@link java.sql.Date}. */
+    private static Expression fromInternal(Expression operand, Type targetType) {
+        return fromInternal(operand, operand.getType(), targetType);
+    }
+
+    /** */
+    private static Expression fromInternal(Expression operand,
+        Type fromType, Type targetType) {
+        if (operand == ConstantUntypedNull.INSTANCE)
+          return operand;
+        if (!(operand.getType() instanceof Class))
+          return operand;
+        if (Types.isAssignableFrom(targetType, fromType))
+          return operand;
+        if (targetType == java.sql.Date.class) {
+            // E.g. from "int" or "Integer" to "java.sql.Date",
+            // generate "SqlFunctions.internalToDate".
+            if (isA(fromType, Primitive.INT))
+              return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, operand);
+        }
+        else if (targetType == java.sql.Time.class) {
+            // E.g. from "int" or "Integer" to "java.sql.Time",
+            // generate "SqlFunctions.internalToTime".
+            if (isA(fromType, Primitive.INT))
+              return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, operand);
+        }
+        else if (targetType == java.sql.Timestamp.class) {
+            // E.g. from "long" or "Long" to "java.sql.Timestamp",
+            // generate "SqlFunctions.internalToTimestamp".
+            if (isA(fromType, Primitive.LONG))
+              return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method, operand);
+        }
+        if (Primitive.is(operand.type)
+            && Primitive.isBox(targetType)) {
+            // E.g. operand is "int", target is "Long", generate "(long) operand".
+            return Expressions.convert_(operand,
+                Primitive.ofBox(targetType).primitiveClass);
+        }
+        return operand;
+    }
+
+    /** */
+    static List<Expression> fromInternal(Class<?>[] targetTypes,
+        List<Expression> expressions) {
+        final List<Expression> list = new ArrayList<>();
+        if (targetTypes.length == expressions.size()) {
+            for (int i = 0; i < expressions.size(); i++)
+                list.add(fromInternal(expressions.get(i), targetTypes[i]));
+        } else {
+            int j = 0;
+            for (int i = 0; i < expressions.size(); i++) {
+                Class<?> type;
+                if (!targetTypes[j].isArray()) {
+                    type = targetTypes[j];
+                    j++;
+                } else
+                    type = targetTypes[j].getComponentType();
+
+                list.add(fromInternal(expressions.get(i), type));
+            }
+        }
+        return list;
+    }
+
+    /** */
+    private static Type toInternal(RelDataType type) {
+        return toInternal(type, false);
+    }
+
+    /** */
+    static Type toInternal(RelDataType type, boolean forceNotNull) {
+        switch (type.getSqlTypeName()) {
+            case DATE:
+            case TIME:
+                return type.isNullable() && !forceNotNull ? Integer.class : int.class;
+            case TIMESTAMP:
+                return type.isNullable() && !forceNotNull ? Long.class : long.class;
+            default:
+                return null; // we don't care; use the default storage type
+        }
+    }
+
+    /** */
+    static List<Type> internalTypes(List<? extends RexNode> operandList) {
+        return Util.transform(operandList, node -> toInternal(node.getType()));
+    }
+
+    /**
+     * Convert {@code operand} to target type {@code toType}.
+     *
+     * @param operand The expression to convert
+     * @param toType Target type
+     * @return A new expression with type {@code toType} or original if there is no need to convert
+     */
+    public static Expression convert(Expression operand, Type toType) {
+        final Type fromType = operand.getType();
+        return convert(operand, fromType, toType);
+    }
+
+    /**
+     * Convert {@code operand} to target type {@code toType}.
+     *
+     * @param operand The expression to convert
+     * @param fromType Field type
+     * @param toType Target type
+     * @return A new expression with type {@code toType} or original if there is no need to convert
+     */
+    public static Expression convert(Expression operand, Type fromType, Type toType) {
+        if (!Types.needTypeCast(fromType, toType))
+          return operand;
+        // E.g. from "Short" to "int".
+        // Generate "x.intValue()".
+        final Primitive toPrimitive = Primitive.of(toType);
+        final Primitive toBox = Primitive.ofBox(toType);
+        final Primitive fromBox = Primitive.ofBox(fromType);
+        final Primitive fromPrimitive = Primitive.of(fromType);
+        final boolean fromNumber = fromType instanceof Class
+            && Number.class.isAssignableFrom((Class)fromType);
+        if (fromType == String.class) {
+            if (toPrimitive != null) {
+                switch (toPrimitive) {
+                    case CHAR:
+                    case SHORT:
+                    case INT:
+                    case LONG:
+                    case FLOAT:
+                    case DOUBLE:
+                        // Generate "SqlFunctions.toShort(x)".
+                        return Expressions.call(
+                            SqlFunctions.class,
+                            "to" + SqlFunctions.initcap(toPrimitive.primitiveName),
+                            operand);
+                    default:
+                        // Generate "Short.parseShort(x)".
+                        return Expressions.call(
+                            toPrimitive.boxClass,
+                            "parse" + SqlFunctions.initcap(toPrimitive.primitiveName),
+                            operand);
+                }
+            }
+            if (toBox != null) {
+                switch (toBox) {
+                    case CHAR:
+                        // Generate "SqlFunctions.toCharBoxed(x)".
+                        return Expressions.call(
+                            SqlFunctions.class,
+                            "to" + SqlFunctions.initcap(toBox.primitiveName) + "Boxed",
+                            operand);
+                    default:
+                        // Generate "Short.valueOf(x)".
+                        return Expressions.call(
+                            toBox.boxClass,
+                            "valueOf",
+                            operand);
+                }
+            }
+        }
+        if (toPrimitive != null) {
+            if (fromPrimitive != null) {
+                // E.g. from "float" to "double"
+                return Expressions.convert_(
+                    operand, toPrimitive.primitiveClass);
+            }
+            if (fromNumber || fromBox == Primitive.CHAR) {
+                // Generate "x.shortValue()".
+                return Expressions.unbox(operand, toPrimitive);
+            }
+            else {
+                // E.g. from "Object" to "short".
+                // Generate "SqlFunctions.toShort(x)"
+                return Expressions.call(
+                    SqlFunctions.class,
+                    "to" + SqlFunctions.initcap(toPrimitive.primitiveName),
+                    operand);
+            }
+        }
+        else if (fromNumber && toBox != null) {
+            // E.g. from "Short" to "Integer"
+            // Generate "x == null ? null : Integer.valueOf(x.intValue())"
+            return Expressions.condition(
+                Expressions.equal(operand, RexImpTable.NULL_EXPR),
+                RexImpTable.NULL_EXPR,
+                Expressions.box(
+                    Expressions.unbox(operand, toBox),
+                    toBox));
+        }
+        else if (fromPrimitive != null && toBox != null) {
+            // E.g. from "int" to "Long".
+            // Generate Long.valueOf(x)
+            // Eliminate primitive casts like Long.valueOf((long) x)
+            if (operand instanceof UnaryExpression) {
+                UnaryExpression una = (UnaryExpression)operand;
+                if (una.nodeType == ExpressionType.Convert
+                    && Primitive.of(una.getType()) == toBox) {
+                    Primitive origin = Primitive.of(una.expression.type);
+                    if (origin != null && toBox.assignableFrom(origin))
+                      return Expressions.box(una.expression, toBox);
+                }
+            }
+            if (fromType == toBox.primitiveClass)
+              return Expressions.box(operand, toBox);
+            // E.g., from "int" to "Byte".
+            // Convert it first and generate "Byte.valueOf((byte)x)"
+            // Because there is no method "Byte.valueOf(int)" in Byte
+            return Expressions.box(
+                Expressions.convert_(operand, toBox.primitiveClass),
+                toBox);
+        }
+        // Convert datetime types to internal storage type:
+        // 1. java.sql.Date -> int or Integer
+        // 2. java.sql.Time -> int or Integer
+        // 3. java.sql.Timestamp -> long or Long
+        if (representAsInternalType(fromType)) {
+            final Expression internalTypedOperand =
+                toInternal(operand, fromType, toType);
+            if (operand != internalTypedOperand)
+              return internalTypedOperand;
+        }
+        // Convert internal storage type to datetime types:
+        // 1. int or Integer -> java.sql.Date
+        // 2. int or Integer -> java.sql.Time
+        // 3. long or Long -> java.sql.Timestamp
+        if (representAsInternalType(toType)) {
+            final Expression originTypedOperand =
+                fromInternal(operand, fromType, toType);
+            if (operand != originTypedOperand)
+              return originTypedOperand;
+        }
+        if (toType == BigDecimal.class) {
+            if (fromBox != null) {
+                // E.g. from "Integer" to "BigDecimal".
+                // Generate "x == null ? null : new BigDecimal(x.intValue())"
+                return Expressions.condition(
+                    Expressions.equal(operand, RexImpTable.NULL_EXPR),
+                    RexImpTable.NULL_EXPR,
+                    Expressions.new_(
+                        BigDecimal.class,
+                        Expressions.unbox(operand, fromBox)));
+            }
+            if (fromPrimitive != null) {
+                // E.g. from "int" to "BigDecimal".
+                // Generate "new BigDecimal(x)"
+                return Expressions.new_(BigDecimal.class, operand);
+            }
+            // E.g. from "Object" to "BigDecimal".
+            // Generate "x == null ? null : SqlFunctions.toBigDecimal(x)"
+            return Expressions.condition(
+                Expressions.equal(operand, RexImpTable.NULL_EXPR),
+                RexImpTable.NULL_EXPR,
+                Expressions.call(
+                    SqlFunctions.class,
+                    "toBigDecimal",
+                    operand));
+        }
+        else if (toType == String.class) {
+            if (fromPrimitive != null) {
+                switch (fromPrimitive) {
+                    case DOUBLE:
+                    case FLOAT:
+                        // E.g. from "double" to "String"
+                        // Generate "SqlFunctions.toString(x)"
+                        return Expressions.call(
+                            SqlFunctions.class,
+                            "toString",
+                            operand);
+                    default:
+                        // E.g. from "int" to "String"
+                        // Generate "Integer.toString(x)"
+                        return Expressions.call(
+                            fromPrimitive.boxClass,
+                            "toString",
+                            operand);
+                }
+            }
+            else if (fromType == BigDecimal.class) {
+                // E.g. from "BigDecimal" to "String"
+                // Generate "SqlFunctions.toString(x)"
+                return Expressions.condition(
+                    Expressions.equal(operand, RexImpTable.NULL_EXPR),
+                    RexImpTable.NULL_EXPR,
+                    Expressions.call(
+                        SqlFunctions.class,
+                        "toString",
+                        operand));
+            }
+            else {
+                Expression result;
+                try {
+                    // Avoid to generate code like:
+                    // "null.toString()" or "(xxx) null.toString()"
+                    if (operand instanceof ConstantExpression) {
+                        ConstantExpression ce = (ConstantExpression)operand;
+                        if (ce.value == null)
+                          return Expressions.convert_(operand, toType);
+                    }
+                    // Try to call "toString()" method
+                    // E.g. from "Integer" to "String"
+                    // Generate "x == null ? null : x.toString()"
+                    result = Expressions.condition(
+                        Expressions.equal(operand, RexImpTable.NULL_EXPR),
+                        RexImpTable.NULL_EXPR,
+                        Expressions.call(operand, "toString"));
+                }
+                catch (RuntimeException e) {
+                    // For some special cases, e.g., "BuiltInMethod.LESSER",
+                    // its return type is generic ("Comparable"), which contains
+                    // no "toString()" method. We fall through to "(String)x".
+                    return Expressions.convert_(operand, toType);
+                }
+                return result;
+            }
+        }
+        return Expressions.convert_(operand, toType);
+    }
+
+    /** */
+    private static boolean isA(Type fromType, Primitive primitive) {
+        return Primitive.of(fromType) == primitive
+            || Primitive.ofBox(fromType) == primitive;
+    }
+
+    /** */
+    private static boolean representAsInternalType(Type type) {
+        return type == java.sql.Date.class
+            || type == java.sql.Time.class
+            || type == java.sql.Timestamp.class;
+    }
+
+    /**
+     * In {@link org.apache.calcite.sql.type.SqlTypeAssignmentRule},
+     * some rules decide whether one type can be assignable to another type.
+     * Based on these rules, a function can accept arguments with assignable types.
+     *
+     * <p>For example, a function with Long type operand can accept Integer as input.
+     * See {@code org.apache.calcite.sql.SqlUtil#filterRoutinesByParameterType()} for details.
+     *
+     * <p>During query execution, some of the assignable types need explicit conversion
+     * to the target types. i.e., Decimal expression should be converted to Integer
+     * before it is assigned to the Integer type Lvalue(In Java, Decimal can not be assigned to
+     * Integer directly).
+     *
+     * @param targetTypes Formal operand types declared for the function arguments
+     * @param arguments Input expressions to the function
+     * @return Input expressions with probable type conversion
+     */
+    static List<Expression> convertAssignableTypes(Class<?>[] targetTypes,
+        List<Expression> arguments) {
+        final List<Expression> list = new ArrayList<>();
+        if (targetTypes.length == arguments.size()) {
+            for (int i = 0; i < arguments.size(); i++)
+                list.add(convertAssignableType(arguments.get(i), targetTypes[i]));
+        } else {
+            int j = 0;
+            for (Expression argument: arguments) {
+                Class<?> type;
+                if (!targetTypes[j].isArray()) {
+                    type = targetTypes[j];
+                    j++;
+                } else
+                    type = targetTypes[j].getComponentType();
+
+                list.add(convertAssignableType(argument, type));
+            }
+        }
+        return list;
+    }
+
+    /**
+     * Handles decimal type specifically with explicit type conversion.
+     */
+    private static Expression convertAssignableType(Expression argument, Type targetType) {
+        if (targetType != BigDecimal.class)
+            return argument;
+
+        return convert(argument, targetType);
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
new file mode 100644
index 0000000..7255955
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+
+/**
+ * Expression factory.
+ */
+public interface ExpressionFactory<Row> {
+    /** */
+    Supplier<List<AccumulatorWrapper<Row>>> accumulatorsFactory(
+        AggregateType type,
+        List<AggregateCall> calls,
+        RelDataType rowType
+    );
+
+    /**
+     * Creates a comparator for given data type and collations. Mainly used for sorted exchange.
+     *
+     * @param collations Collations.
+     * @return Row comparator.
+     */
+    Comparator<Row> comparator(RelCollation collations);
+
+    /**
+     * Creates a comparator for different rows by given field collations. Mainly used for merge join rows comparison.
+     * Note: Both list has to have the same size and matched fields collations has to have the same traits
+     * (i.e. all pairs of field collations should have the same sorting and nulls ordering).
+     *
+     * @param left Collations of left row.
+     * @param right Collations of right row.
+     * @return Rows comparator.
+     */
+    Comparator<Row> comparator(List<RelFieldCollation> left, List<RelFieldCollation> right);
+
+    /**
+     * Creates a Filter predicate.
+     * @param filter Filter expression.
+     * @param rowType Input row type.
+     * @return Filter predicate.
+     */
+    Predicate<Row> predicate(RexNode filter, RelDataType rowType);
+
+    /**
+     * Creates a Project function. Resulting function returns a row with different fields,
+     * fields order, fields types, etc.
+     * @param projects Projection expressions.
+     * @param rowType Input row type.
+     * @return Project function.
+     */
+    Function<Row, Row> project(List<RexNode> projects, RelDataType rowType);
+
+    /**
+     * Creates a Values relational node rows source.
+     *
+     * @param values Values.
+     * @param rowType Output row type.
+     * @return Values relational node rows source.
+     */
+    Iterable<Row> values(List<RexLiteral> values, RelDataType rowType);
+
+    /**
+     * Creates row from RexNodes.
+     *
+     * @param values Values.
+     * @return Row.
+     */
+    Supplier<Row> rowSource(List<RexNode> values);
+
+
+    /**
+     * Executes expression.
+     */
+    <T> Supplier<T> execute(RexNode node);
+
+    /**
+     * Creates {@link Scalar}, a code-generated expressions evaluator.
+     *
+     * @param node Expression.
+     * @param type Row type.
+     * @return Scalar.
+     */
+    default Scalar scalar(RexNode node, RelDataType type) {
+        return scalar(ImmutableList.of(node), type);
+    }
+
+    /**
+     * Creates {@link Scalar}, a code-generated expressions evaluator.
+     *
+     * @param nodes Expressions.
+     * @param type Row type.
+     * @return Scalar.
+     */
+    Scalar scalar(List<RexNode> nodes, RelDataType type);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
new file mode 100644
index 0000000..eb8260d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
@@ -0,0 +1,509 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Ordering;
+import com.google.common.primitives.Primitives;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumUtils;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.RexToLixTranslator.InputGetter;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorsFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Implements rex expression into a function object. Uses JaninoRexCompiler under the hood.
+ * Each expression compiles into a class and a wrapper over it is returned.
+ */
+public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
+//    /** */
+//    private static final Map<String, Scalar> SCALAR_CACHE = new GridBoundedConcurrentLinkedHashMap<>(1024);
+
+    /** */
+    private final IgniteTypeFactory typeFactory;
+
+    /** */
+    private final SqlConformance conformance;
+
+    /** */
+    private final RexBuilder rexBuilder;
+
+    /** */
+    private final RelDataType emptyType;
+
+    /** */
+    private final ExecutionContext<Row> ctx;
+
+    /** */
+    public ExpressionFactoryImpl(ExecutionContext<Row> ctx, IgniteTypeFactory typeFactory, SqlConformance conformance) {
+        this.ctx = ctx;
+        this.typeFactory = typeFactory;
+        this.conformance = conformance;
+
+        rexBuilder = new RexBuilder(this.typeFactory);
+        emptyType = new RelDataTypeFactory.Builder(this.typeFactory).build();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Supplier<List<AccumulatorWrapper<Row>>> accumulatorsFactory(
+        AggregateType type,
+        List<AggregateCall> calls,
+        RelDataType rowType
+    ) {
+        if (calls.isEmpty())
+            return null;
+
+        return new AccumulatorsFactory<>(ctx, type, calls, rowType);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Comparator<Row> comparator(RelCollation collation) {
+        if (collation == null || collation.getFieldCollations().isEmpty())
+            return null;
+        else if (collation.getFieldCollations().size() == 1)
+            return comparator(collation.getFieldCollations().get(0));
+        return Ordering.compound(collation.getFieldCollations()
+            .stream()
+            .map(this::comparator)
+            .collect(Collectors.toList()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Comparator<Row> comparator(List<RelFieldCollation> left, List<RelFieldCollation> right) {
+        if (nullOrEmpty(left) || nullOrEmpty(right) || left.size() != right.size())
+            throw new IllegalArgumentException("Both inputs should be non-empty and have the same size: left="
+                + (left != null ? left.size() : "null") + ", right=" + (right != null ? right.size() : "null"));
+
+        List<Comparator<Row>> comparators = new ArrayList<>();
+
+        for (int i = 0; i < left.size(); i++)
+            comparators.add(comparator(left.get(i), right.get(i)));
+
+        return Ordering.compound(comparators);
+    }
+
+    /** */
+    @SuppressWarnings("rawtypes")
+    private Comparator<Row> comparator(RelFieldCollation fieldCollation) {
+        final int nullComparison = fieldCollation.nullDirection.nullComparison;
+        final int x = fieldCollation.getFieldIndex();
+        RowHandler<Row> handler = ctx.rowHandler();
+
+        if (fieldCollation.direction == RelFieldCollation.Direction.ASCENDING) {
+            return (o1, o2) -> {
+                final Comparable c1 = (Comparable)handler.get(x, o1);
+                final Comparable c2 = (Comparable)handler.get(x, o2);
+                return RelFieldCollation.compare(c1, c2, nullComparison);
+            };
+        }
+
+        return (o1, o2) -> {
+            final Comparable c1 = (Comparable)handler.get(x, o1);
+            final Comparable c2 = (Comparable)handler.get(x, o2);
+            return RelFieldCollation.compare(c2, c1, nullComparison);
+        };
+    }
+
+    /** */
+    @SuppressWarnings("rawtypes")
+    private Comparator<Row> comparator(RelFieldCollation left, RelFieldCollation right) {
+        final int nullComparison = left.nullDirection.nullComparison;
+
+        if (nullComparison != right.nullDirection.nullComparison)
+            throw new IllegalArgumentException("Can't be compared: left=" + left + ", right=" + right);
+
+        final int lIdx = left.getFieldIndex();
+        final int rIdx = right.getFieldIndex();
+        RowHandler<Row> handler = ctx.rowHandler();
+
+        if (left.direction != right.direction)
+            throw new IllegalArgumentException("Can't be compared: left=" + left + ", right=" + right);
+
+        if (left.direction == RelFieldCollation.Direction.ASCENDING) {
+            return (o1, o2) -> {
+                final Comparable c1 = (Comparable)handler.get(lIdx, o1);
+                final Comparable c2 = (Comparable)handler.get(rIdx, o2);
+                return RelFieldCollation.compare(c1, c2, nullComparison);
+            };
+        }
+
+        return (o1, o2) -> {
+            final Comparable c1 = (Comparable)handler.get(lIdx, o1);
+            final Comparable c2 = (Comparable)handler.get(rIdx, o2);
+            return RelFieldCollation.compare(c2, c1, -nullComparison);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public Predicate<Row> predicate(RexNode filter, RelDataType rowType) {
+        return new PredicateImpl(scalar(filter, rowType));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Function<Row, Row> project(List<RexNode> projects, RelDataType rowType) {
+        return new ProjectImpl(scalar(projects, rowType), ctx.rowHandler().factory(typeFactory, RexUtil.types(projects)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Supplier<Row> rowSource(List<RexNode> values) {
+        return new ValuesImpl(scalar(values, null), ctx.rowHandler().factory(typeFactory, RexUtil.types(values)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> Supplier<T> execute(RexNode node) {
+        return new ValueImpl<T>(scalar(node, null), ctx.rowHandler().factory(typeFactory.getJavaClass(node.getType())));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterable<Row> values(List<RexLiteral> values, RelDataType rowType) {
+        RowHandler<Row> handler = ctx.rowHandler();
+        RowFactory<Row> factory = handler.factory(typeFactory, rowType);
+
+        int columns = rowType.getFieldCount();
+        assert values.size() % columns == 0;
+
+        List<Class<?>> types = new ArrayList<>(columns);
+        for (RelDataType type : RelOptUtil.getFieldTypeList(rowType))
+            types.add(Primitives.wrap((Class<?>)typeFactory.getJavaClass(type)));
+
+        List<Row> rows = new ArrayList<>(values.size() / columns);
+        Row currRow = null;
+        for (int i = 0; i < values.size(); i++) {
+            int field = i % columns;
+
+            if (field == 0)
+                rows.add(currRow = factory.create());
+
+            RexLiteral literal = values.get(i);
+
+            handler.set(field, currRow, literal.getValueAs(types.get(field)));
+        }
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Scalar scalar(List<RexNode> nodes, RelDataType type) {
+        return compile(nodes, type);
+    }
+
+    /** */
+    private Scalar compile(Iterable<RexNode> nodes, RelDataType type) {
+        if (type == null)
+            type = emptyType;
+
+        RexProgramBuilder programBuilder = new RexProgramBuilder(type, rexBuilder);
+
+        for (RexNode node : nodes)
+            programBuilder.addProject(node, null);
+
+        RexProgram program = programBuilder.getProgram();
+
+        BlockBuilder builder = new BlockBuilder();
+
+        ParameterExpression ctx_ =
+            Expressions.parameter(ExecutionContext.class, "ctx");
+
+        ParameterExpression in_ =
+            Expressions.parameter(Object.class, "in");
+
+        ParameterExpression out_ =
+            Expressions.parameter(Object.class, "out");
+
+        builder.add(
+            Expressions.declare(Modifier.FINAL, DataContext.ROOT, Expressions.convert_(ctx_, DataContext.class)));
+
+        Expression hnd_ = builder.append("hnd",
+            Expressions.call(ctx_,
+                IgniteMethod.CONTEXT_ROW_HANDLER.method()));
+
+        InputGetter inputGetter = new FieldGetter(hnd_, in_, type);
+
+        Function1<String, InputGetter> correlates = new CorrelatesBuilder(builder, ctx_, hnd_).build(nodes);
+
+        List<Expression> projects = RexToLixTranslator.translateProjects(program, typeFactory, conformance,
+            builder, null, ctx_, inputGetter, correlates);
+
+        for (int i = 0; i < projects.size(); i++) {
+            builder.add(
+                Expressions.statement(
+                    Expressions.call(hnd_,
+                        IgniteMethod.ROW_HANDLER_SET.method(),
+                            Expressions.constant(i), out_, projects.get(i))));
+        }
+
+        MethodDeclaration decl = Expressions.methodDecl(
+            Modifier.PUBLIC, void.class, IgniteMethod.SCALAR_EXECUTE.method().getName(),
+            ImmutableList.of(ctx_, in_, out_), builder.toBlock());
+
+        return Commons.compile(Scalar.class, Expressions.toString(List.of(decl), "\n", false));
+    }
+
+    /** */
+    private String digest(List<RexNode> nodes, RelDataType type) {
+        StringBuilder b = new StringBuilder();
+
+        b.append('[');
+
+        for (int i = 0; i < nodes.size(); i++) {
+            if (i > 0)
+                b.append(';');
+
+            b.append(nodes.get(i));
+        }
+
+        b.append(']');
+
+        if (type != null)
+            b.append(':').append(type.getFullTypeString());
+
+        return b.toString();
+    }
+
+    /** */
+    private class PredicateImpl implements Predicate<Row> {
+        /** */
+        private final Scalar scalar;
+
+        /** */
+        private final Row out;
+
+        /** */
+        private final RowHandler<Row> handler;
+
+        /**
+         * @param scalar Scalar.
+         */
+        private PredicateImpl(Scalar scalar) {
+            this.scalar = scalar;
+            handler = ctx.rowHandler();
+            out = handler.factory(typeFactory, typeFactory.createJavaType(Boolean.class)).create();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean test(Row r) {
+            scalar.execute(ctx, r, out);
+
+            return Boolean.TRUE == handler.get(0, out);
+        }
+    }
+
+    /** */
+    private class ProjectImpl implements Function<Row, Row> {
+        /** */
+        private final Scalar scalar;
+
+        /** */
+        private final RowFactory<Row> factory;
+
+        /**
+         * @param scalar Scalar.
+         * @param factory Row factory.
+         */
+        private ProjectImpl(Scalar scalar, RowFactory<Row> factory) {
+            this.scalar = scalar;
+            this.factory = factory;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Row apply(Row r) {
+            Row res = factory.create();
+            scalar.execute(ctx, r, res);
+
+            return res;
+        }
+    }
+
+    /** */
+    private class ValuesImpl implements Supplier<Row> {
+        /** */
+        private final Scalar scalar;
+
+        /** */
+        private final RowFactory<Row> factory;
+
+        /** */
+        private ValuesImpl(Scalar scalar, RowFactory<Row> factory) {
+            this.scalar = scalar;
+            this.factory = factory;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Row get() {
+            Row res = factory.create();
+            scalar.execute(ctx, null, res);
+
+            return res;
+        }
+    }
+
+    /** */
+    private class ValueImpl<T> implements Supplier<T> {
+        /** */
+        private final Scalar scalar;
+
+        /** */
+        private final RowFactory<Row> factory;
+
+        /** */
+        private ValueImpl(Scalar scalar, RowFactory<Row> factory) {
+            this.scalar = scalar;
+            this.factory = factory;
+        }
+
+        /** {@inheritDoc} */
+        @Override public T get() {
+            Row res = factory.create();
+            scalar.execute(ctx, null, res);
+
+            return (T)ctx.rowHandler().get(0, res);
+        }
+    }
+
+    /** */
+    private class FieldGetter implements InputGetter {
+        /** */
+        private final Expression hnd_;
+
+        /** */
+        private final Expression row_;
+
+        /** */
+        private final RelDataType rowType;
+
+        /** */
+        private FieldGetter(Expression hnd_, Expression row_, RelDataType rowType) {
+            this.hnd_ = hnd_;
+            this.row_ = row_;
+            this.rowType = rowType;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Expression field(BlockBuilder list, int index, Type desiredType) {
+            Expression row_ = list.append("row", this.row_);
+
+            Expression field = Expressions.call(hnd_,
+                IgniteMethod.ROW_HANDLER_GET.method(),
+                    Expressions.constant(index), row_);
+
+            Type fieldType = typeFactory.getJavaClass(rowType.getFieldList().get(index).getType());
+
+            if (desiredType == null) {
+                desiredType = fieldType;
+                fieldType = Object.class;
+            } else if (fieldType != java.sql.Date.class
+                && fieldType != java.sql.Time.class
+                && fieldType != java.sql.Timestamp.class) {
+                fieldType = Object.class;
+            }
+
+            return EnumUtils.convert(field, fieldType, desiredType);
+        }
+    }
+
+    /** */
+    private class CorrelatesBuilder extends RexShuttle {
+        /** */
+        private final BlockBuilder builder;
+
+        /** */
+        private final Expression ctx_;
+
+        /** */
+        private final Expression hnd_;
+
+        /** */
+        private Map<String, FieldGetter> correlates;
+
+        /** */
+        private CorrelatesBuilder(BlockBuilder builder, Expression ctx_, Expression hnd_) {
+            this.builder = builder;
+            this.hnd_ = hnd_;
+            this.ctx_ = ctx_;
+        }
+
+        /** */
+        public Function1<String, InputGetter> build(Iterable<RexNode> nodes) {
+            try {
+                for (RexNode node : nodes)
+                    node.accept(this);
+
+                return correlates == null ? null : correlates::get;
+            }
+            finally {
+                correlates = null;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public RexNode visitCorrelVariable(RexCorrelVariable variable) {
+            Expression corr_ = builder.append("corr",
+                Expressions.call(ctx_, IgniteMethod.CONTEXT_GET_CORRELATED_VALUE.method(),
+                    Expressions.constant(variable.id.getId())));
+
+            if (correlates == null)
+                correlates = new HashMap<>();
+
+            correlates.put(variable.getName(), new FieldGetter(hnd_, corr_, variable.getType()));
+
+            return variable;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteBuiltInMethod.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteBuiltInMethod.java
new file mode 100644
index 0000000..bbcc881
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteBuiltInMethod.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Method;
+
+import org.apache.calcite.linq4j.tree.Types;
+
+/**
+ * Built-in methods.
+ */
+public enum IgniteBuiltInMethod {
+    /** */
+    SYSTEM_RANGE2(IgniteSqlFunctions.class, "systemRange", Object.class, Object.class),
+
+    /** */
+    SYSTEM_RANGE3(IgniteSqlFunctions.class, "systemRange", Object.class, Object.class, Object.class);
+
+    /** */
+    public final Method method;
+
+    /** */
+    IgniteBuiltInMethod(Method method) {
+        this.method = method;
+    }
+
+    /** Defines a method. */
+    IgniteBuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
+        this(Types.lookupMethod(clazz, methodName, argumentTypes));
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteSqlFunctions.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteSqlFunctions.java
new file mode 100644
index 0000000..203c676
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteSqlFunctions.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.exp;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Ignite SQL functions.
+ */
+public class IgniteSqlFunctions {
+    /**
+     * Default constructor.
+     */
+    private IgniteSqlFunctions() {
+        // No-op.
+    }
+
+    /** SQL SYSTEM_RANGE(start, end) table function. */
+    public static ScannableTable systemRange(Object rangeStart, Object rangeEnd) {
+        return new RangeTable(rangeStart, rangeEnd, 1L);
+    }
+
+    /** SQL SYSTEM_RANGE(start, end, increment) table function. */
+    public static ScannableTable systemRange(Object rangeStart, Object rangeEnd, Object increment) {
+        return new RangeTable(rangeStart, rangeEnd, increment);
+    }
+
+    /** */
+    private static class RangeTable implements ScannableTable {
+        /** Start of the range. */
+        private final Object rangeStart;
+
+        /** End of the range. */
+        private final Object rangeEnd;
+
+        /** Increment. */
+        private final Object increment;
+
+        /**
+         * Note: {@code Object} arguments required here due to:
+         * 1. {@code NULL} arguments need to be supported, so we can't use {@code long} arguments type.
+         * 2. {@code Integer} and other numeric classes can be converted to {@code long} type by java, but can't be
+         *      converted to {@code Long} type, so we can't use {@code Long} arguments type either.
+         * Instead, we accept {@code Object} arguments type and try to convert valid types to {@code long}.
+         */
+        RangeTable(Object rangeStart, Object rangeEnd, Object increment) {
+            this.rangeStart = rangeStart;
+            this.rangeEnd = rangeEnd;
+            this.increment = increment;
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+            return typeFactory.builder().add("X", SqlTypeName.BIGINT).build();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Enumerable<@Nullable Object[]> scan(DataContext root) {
+            if (rangeStart == null || rangeEnd == null || increment == null)
+                return Linq4j.emptyEnumerable();
+
+            long rangeStart = convertToLongArg(this.rangeStart, "rangeStart");
+            long rangeEnd = convertToLongArg(this.rangeEnd, "rangeEnd");
+            long increment = convertToLongArg(this.increment, "increment");
+
+            if (increment == 0L)
+                throw new IllegalArgumentException("Increment can't be 0");
+
+            return new AbstractEnumerable<@Nullable Object[]>() {
+                @Override public Enumerator<@Nullable Object[]> enumerator() {
+                    return new Enumerator<Object[]>() {
+                        long cur = rangeStart - increment;
+
+                        @Override public Object[] current() {
+                            return new Object[] { cur };
+                        }
+
+                        @Override public boolean moveNext() {
+                            cur += increment;
+
+                            return increment > 0L ? cur <= rangeEnd : cur >= rangeEnd;
+                        }
+
+                        @Override public void reset() {
+                            cur = rangeStart - increment;
+                        }
+
+                        @Override public void close() {
+                            // No-op.
+                        }
+                    };
+                }
+            };
+        }
+
+        /** */
+        private long convertToLongArg(Object val, String name) {
+            if (val instanceof Byte || val instanceof Short || val instanceof Integer || val instanceof Long)
+                return ((Number)val).longValue();
+
+            throw new IllegalArgumentException("Unsupported argument type [arg=" + name +
+                ", type=" + val.getClass().getSimpleName() + ']');
+        }
+
+        /** {@inheritDoc} */
+        @Override public Statistic getStatistic() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Schema.TableType getJdbcTableType() {
+            return Schema.TableType.TABLE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isRolledUp(String column) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean rolledUpColumnValidInsideAgg(String column, SqlCall call,
+            SqlNode parent, CalciteConnectionConfig cfg) {
+            return true;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ImplementableFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ImplementableFunction.java
new file mode 100644
index 0000000..a30c01f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ImplementableFunction.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.schema.TableFunction;
+
+/**
+ * Function that can be translated to java code.
+ *
+ * @see ScalarFunction
+ * @see TableFunction
+ */
+public interface ImplementableFunction extends Function {
+    /**
+     * Returns implementor that translates the function to linq4j expression.
+     *
+     * @return implementor that translates the function to linq4j expression.
+     */
+    CallImplementor getImplementor();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/NotNullImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/NotNullImplementor.java
new file mode 100644
index 0000000..2ff3061
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/NotNullImplementor.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.util.List;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexCall;
+
+/**
+ * Simplified version of {@link CallImplementor} that does not know about null semantics.
+ *
+ * @see RexImpTable
+ * @see CallImplementor
+ */
+public interface NotNullImplementor {
+    /**
+     * Implements a call with assumption that all the null-checking is implemented by caller.
+     *
+     * @param translator translator to implement the code
+     * @param call call to implement
+     * @param translatedOperands arguments of a call
+     * @return expression that implements given call
+     */
+    Expression implement(
+        RexToLixTranslator translator,
+        RexCall call,
+        List<Expression> translatedOperands);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ReflectiveCallNotNullImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ReflectiveCallNotNullImplementor.java
new file mode 100644
index 0000000..857605b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ReflectiveCallNotNullImplementor.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.List;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.rex.RexCall;
+
+/**
+ * Implementation of {@link NotNullImplementor} that calls a given {@link Method}.
+ *
+ * <p>When method is not static, a new instance of the required class is
+ * created.
+ */
+public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
+    /** */
+    protected final Method method;
+
+    /**
+     * Constructor of ReflectiveCallNotNullImplementor.
+     *
+     * @param method Method that is used to implement the call
+     */
+    public ReflectiveCallNotNullImplementor(Method method) {
+        this.method = method;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Expression implement(RexToLixTranslator translator,
+        RexCall call, List<Expression> translatedOperands) {
+        translatedOperands =
+            ConverterUtils.fromInternal(method.getParameterTypes(), translatedOperands);
+        translatedOperands =
+            ConverterUtils.convertAssignableTypes(method.getParameterTypes(), translatedOperands);
+        final Expression callExpr;
+        if ((method.getModifiers() & Modifier.STATIC) != 0)
+            callExpr = Expressions.call(method, translatedOperands);
+
+        else {
+            // The UDF class must have a public zero-args constructor.
+            // Assume that the validator checked already.
+            final Expression target =
+                Expressions.new_(method.getDeclaringClass());
+            callExpr = Expressions.call(target, method, translatedOperands);
+        }
+        if (!containsCheckedException(method))
+            return callExpr;
+
+        return translator.handleMethodCheckedExceptions(callExpr);
+    }
+
+    /** */
+    private boolean containsCheckedException(Method method) {
+        Class[] exceptions = method.getExceptionTypes();
+        if (exceptions == null || exceptions.length == 0)
+            return false;
+
+        for (Class clazz : exceptions) {
+            if (!RuntimeException.class.isAssignableFrom(clazz))
+                return true;
+        }
+        return false;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexImpTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexImpTable.java
new file mode 100644
index 0000000..bf91db0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexImpTable.java
@@ -0,0 +1,2574 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.adapter.enumerable.EnumUtils;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberExpression;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.OptimizeShuttle;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlJsonEmptyOrError;
+import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.SqlTypeConstructorFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+import static org.apache.calcite.adapter.enumerable.EnumUtils.generateCollatorExpression;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Add;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Divide;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Equal;
+import static org.apache.calcite.linq4j.tree.ExpressionType.GreaterThan;
+import static org.apache.calcite.linq4j.tree.ExpressionType.GreaterThanOrEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.LessThan;
+import static org.apache.calcite.linq4j.tree.ExpressionType.LessThanOrEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Multiply;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Negate;
+import static org.apache.calcite.linq4j.tree.ExpressionType.NotEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Subtract;
+import static org.apache.calcite.linq4j.tree.ExpressionType.UnaryPlus;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.CHR;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.COMPRESS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.CONCAT2;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.CONCAT_FUNCTION;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.COSH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATE_FROM_UNIX_DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DAYNAME;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DIFFERENCE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.EXISTS_NODE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.EXTRACT_VALUE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.EXTRACT_XML;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.FROM_BASE64;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_DEPTH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_KEYS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_LENGTH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_PRETTY;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_REMOVE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_STORAGE_SIZE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_TYPE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.LEFT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.MD5;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.MONTHNAME;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.REGEXP_REPLACE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.REPEAT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.REVERSE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.RIGHT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SHA1;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SINH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SOUNDEX;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SPACE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.STRCMP;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TANH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TIMESTAMP_MICROS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TIMESTAMP_MILLIS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TIMESTAMP_SECONDS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TO_BASE64;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TRANSLATE3;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_MICROS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_MILLIS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_SECONDS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.XML_TRANSFORM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ABS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ACOS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ASCII;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ASIN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ATAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ATAN2;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CARDINALITY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CAST;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CBRT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CEIL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHARACTER_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHAR_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COALESCE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CONCAT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_CATALOG;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_PATH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_ROLE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIME;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIMESTAMP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DATETIME_PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DEFAULT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DEGREES;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ELEMENT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EQUALS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXTRACT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FLOOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.INITCAP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_A_SET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_EMPTY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_ARRAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_OBJECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_SCALAR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_A_SET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_EMPTY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_ARRAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_OBJECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_SCALAR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ITEM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_EXISTS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_QUERY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE_EXPRESSION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LAST_DAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LIKE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOCALTIME;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOCALTIMESTAMP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOG10;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MEMBER_OF;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MOD;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_EXCEPT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_EXCEPT_DISTINCT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_INTERSECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_INTERSECT_DISTINCT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_UNION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_UNION_DISTINCT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NEXT_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_EQUALS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_LIKE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_SIMILAR_TO;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_SUBMULTISET_OF;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OVERLAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PI;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POSITION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RADIANS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RAND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RAND_INTEGER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.REINTERPRET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.REPLACE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ROUND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ROW;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SESSION_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIGN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIMILAR_TO;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SLICE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.STRUCT_ACCESS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBMULTISET_OF;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBSTRING;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SYSTEM_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRIM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRUNCATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UNARY_MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UNARY_PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UPPER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.USER;
+import static org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable.LENGTH;
+import static org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable.SYSTEM_RANGE;
+
+/**
+ * Contains implementations of Rex operators as Java code.
+ */
+public class RexImpTable {
+    /** */
+    public static final RexImpTable INSTANCE = new RexImpTable();
+
+    /** */
+    public static final ConstantExpression NULL_EXPR = Expressions.constant(null);
+
+    /** */
+    public static final ConstantExpression FALSE_EXPR = Expressions.constant(false);
+
+    /** */
+    public static final ConstantExpression TRUE_EXPR = Expressions.constant(true);
+
+    /** */
+    public static final MemberExpression BOXED_FALSE_EXPR = Expressions.field(null, Boolean.class, "FALSE");
+
+    /** */
+    public static final MemberExpression BOXED_TRUE_EXPR = Expressions.field(null, Boolean.class, "TRUE");
+
+    /** */
+    private final Map<SqlOperator, RexCallImplementor> map = new HashMap<>();
+
+    /** */
+    RexImpTable() {
+        defineMethod(ROW, BuiltInMethod.ARRAY.method, NullPolicy.NONE);
+        defineMethod(UPPER, BuiltInMethod.UPPER.method, NullPolicy.STRICT);
+        defineMethod(LOWER, BuiltInMethod.LOWER.method, NullPolicy.STRICT);
+        defineMethod(INITCAP, BuiltInMethod.INITCAP.method, NullPolicy.STRICT);
+        defineMethod(TO_BASE64, BuiltInMethod.TO_BASE64.method, NullPolicy.STRICT);
+        defineMethod(FROM_BASE64, BuiltInMethod.FROM_BASE64.method, NullPolicy.STRICT);
+        defineMethod(MD5, BuiltInMethod.MD5.method, NullPolicy.STRICT);
+        defineMethod(SHA1, BuiltInMethod.SHA1.method, NullPolicy.STRICT);
+        defineMethod(SUBSTRING, BuiltInMethod.SUBSTRING.method, NullPolicy.STRICT);
+        defineMethod(LEFT, BuiltInMethod.LEFT.method, NullPolicy.ANY);
+        defineMethod(RIGHT, BuiltInMethod.RIGHT.method, NullPolicy.ANY);
+        defineMethod(REPLACE, BuiltInMethod.REPLACE.method, NullPolicy.STRICT);
+        defineMethod(TRANSLATE3, BuiltInMethod.TRANSLATE3.method, NullPolicy.STRICT);
+        defineMethod(CHR, "chr", NullPolicy.STRICT);
+        defineMethod(CHARACTER_LENGTH, BuiltInMethod.CHAR_LENGTH.method, NullPolicy.STRICT);
+        defineMethod(CHAR_LENGTH, BuiltInMethod.CHAR_LENGTH.method, NullPolicy.STRICT);
+        defineMethod(LENGTH, BuiltInMethod.CHAR_LENGTH.method, NullPolicy.STRICT);
+        defineMethod(CONCAT, BuiltInMethod.STRING_CONCAT.method, NullPolicy.STRICT);
+        defineMethod(CONCAT_FUNCTION, BuiltInMethod.MULTI_STRING_CONCAT.method, NullPolicy.STRICT);
+        defineMethod(CONCAT2, BuiltInMethod.STRING_CONCAT.method, NullPolicy.STRICT);
+        defineMethod(OVERLAY, BuiltInMethod.OVERLAY.method, NullPolicy.STRICT);
+        defineMethod(POSITION, BuiltInMethod.POSITION.method, NullPolicy.STRICT);
+        defineMethod(ASCII, BuiltInMethod.ASCII.method, NullPolicy.STRICT);
+        defineMethod(REPEAT, BuiltInMethod.REPEAT.method, NullPolicy.STRICT);
+        defineMethod(SPACE, BuiltInMethod.SPACE.method, NullPolicy.STRICT);
+        defineMethod(STRCMP, BuiltInMethod.STRCMP.method, NullPolicy.STRICT);
+        defineMethod(SOUNDEX, BuiltInMethod.SOUNDEX.method, NullPolicy.STRICT);
+        defineMethod(DIFFERENCE, BuiltInMethod.DIFFERENCE.method, NullPolicy.STRICT);
+        defineMethod(REVERSE, BuiltInMethod.REVERSE.method, NullPolicy.STRICT);
+
+        map.put(TRIM, new TrimImplementor());
+
+        // logical
+        map.put(AND, new LogicalAndImplementor());
+        map.put(OR, new LogicalOrImplementor());
+        map.put(NOT, new LogicalNotImplementor());
+
+        // comparisons
+        defineBinary(LESS_THAN, LessThan, NullPolicy.STRICT, "lt");
+        defineBinary(LESS_THAN_OR_EQUAL, LessThanOrEqual, NullPolicy.STRICT, "le");
+        defineBinary(GREATER_THAN, GreaterThan, NullPolicy.STRICT, "gt");
+        defineBinary(GREATER_THAN_OR_EQUAL, GreaterThanOrEqual, NullPolicy.STRICT,
+            "ge");
+        defineBinary(EQUALS, Equal, NullPolicy.STRICT, "eq");
+        defineBinary(NOT_EQUALS, NotEqual, NullPolicy.STRICT, "ne");
+
+        // arithmetic
+        defineBinary(PLUS, Add, NullPolicy.STRICT, "plus");
+        defineBinary(MINUS, Subtract, NullPolicy.STRICT, "minus");
+        defineBinary(MULTIPLY, Multiply, NullPolicy.STRICT, "multiply");
+        defineBinary(DIVIDE, Divide, NullPolicy.STRICT, "divide");
+        defineBinary(DIVIDE_INTEGER, Divide, NullPolicy.STRICT, "divide");
+        defineUnary(UNARY_MINUS, Negate, NullPolicy.STRICT,
+            BuiltInMethod.BIG_DECIMAL_NEGATE.getMethodName());
+        defineUnary(UNARY_PLUS, UnaryPlus, NullPolicy.STRICT, null);
+
+        defineMethod(MOD, "mod", NullPolicy.STRICT);
+        defineMethod(EXP, "exp", NullPolicy.STRICT);
+        defineMethod(POWER, "power", NullPolicy.STRICT);
+        defineMethod(LN, "ln", NullPolicy.STRICT);
+        defineMethod(LOG10, "log10", NullPolicy.STRICT);
+        defineMethod(ABS, "abs", NullPolicy.STRICT);
+
+        map.put(RAND, new RandImplementor());
+        map.put(RAND_INTEGER, new RandIntegerImplementor());
+
+        defineMethod(ACOS, "acos", NullPolicy.STRICT);
+        defineMethod(ASIN, "asin", NullPolicy.STRICT);
+        defineMethod(ATAN, "atan", NullPolicy.STRICT);
+        defineMethod(ATAN2, "atan2", NullPolicy.STRICT);
+        defineMethod(CBRT, "cbrt", NullPolicy.STRICT);
+        defineMethod(COS, "cos", NullPolicy.STRICT);
+        defineMethod(COSH, "cosh", NullPolicy.STRICT);
+        defineMethod(COT, "cot", NullPolicy.STRICT);
+        defineMethod(DEGREES, "degrees", NullPolicy.STRICT);
+        defineMethod(RADIANS, "radians", NullPolicy.STRICT);
+        defineMethod(ROUND, "sround", NullPolicy.STRICT);
+        defineMethod(SIGN, "sign", NullPolicy.STRICT);
+        defineMethod(SIN, "sin", NullPolicy.STRICT);
+        defineMethod(SINH, "sinh", NullPolicy.STRICT);
+        defineMethod(TAN, "tan", NullPolicy.STRICT);
+        defineMethod(TANH, "tanh", NullPolicy.STRICT);
+        defineMethod(TRUNCATE, "struncate", NullPolicy.STRICT);
+
+        map.put(PI, new PiImplementor());
+
+        // datetime
+        map.put(DATETIME_PLUS, new DatetimeArithmeticImplementor());
+        map.put(MINUS_DATE, new DatetimeArithmeticImplementor());
+        map.put(EXTRACT, new ExtractImplementor());
+        map.put(FLOOR,
+            new FloorImplementor(BuiltInMethod.FLOOR.method.getName(),
+                BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method,
+                BuiltInMethod.UNIX_DATE_FLOOR.method));
+        map.put(CEIL,
+            new FloorImplementor(BuiltInMethod.CEIL.method.getName(),
+                BuiltInMethod.UNIX_TIMESTAMP_CEIL.method,
+                BuiltInMethod.UNIX_DATE_CEIL.method));
+
+        defineMethod(LAST_DAY, "lastDay", NullPolicy.STRICT);
+        map.put(DAYNAME,
+            new PeriodNameImplementor("dayName",
+                BuiltInMethod.DAYNAME_WITH_TIMESTAMP,
+                BuiltInMethod.DAYNAME_WITH_DATE));
+        map.put(MONTHNAME,
+            new PeriodNameImplementor("monthName",
+                BuiltInMethod.MONTHNAME_WITH_TIMESTAMP,
+                BuiltInMethod.MONTHNAME_WITH_DATE));
+        defineMethod(TIMESTAMP_SECONDS, "timestampSeconds", NullPolicy.STRICT);
+        defineMethod(TIMESTAMP_MILLIS, "timestampMillis", NullPolicy.STRICT);
+        defineMethod(TIMESTAMP_MICROS, "timestampMicros", NullPolicy.STRICT);
+        defineMethod(UNIX_SECONDS, "unixSeconds", NullPolicy.STRICT);
+        defineMethod(UNIX_MILLIS, "unixMillis", NullPolicy.STRICT);
+        defineMethod(UNIX_MICROS, "unixMicros", NullPolicy.STRICT);
+        defineMethod(DATE_FROM_UNIX_DATE, "dateFromUnixDate", NullPolicy.STRICT);
+        defineMethod(UNIX_DATE, "unixDate", NullPolicy.STRICT);
+
+        map.put(IS_NULL, new IsNullImplementor());
+        map.put(IS_NOT_NULL, new IsNotNullImplementor());
+        map.put(IS_TRUE, new IsTrueImplementor());
+        map.put(IS_NOT_TRUE, new IsNotTrueImplementor());
+        map.put(IS_FALSE, new IsFalseImplementor());
+        map.put(IS_NOT_FALSE, new IsNotFalseImplementor());
+
+        // LIKE and SIMILAR
+        final MethodImplementor likeImplementor =
+            new MethodImplementor(BuiltInMethod.LIKE.method, NullPolicy.STRICT,
+                false);
+        map.put(LIKE, likeImplementor);
+        map.put(NOT_LIKE, likeImplementor);
+        final MethodImplementor similarImplementor =
+            new MethodImplementor(BuiltInMethod.SIMILAR.method, NullPolicy.STRICT,
+                false);
+        map.put(SIMILAR_TO, similarImplementor);
+        map.put(NOT_SIMILAR_TO, NotImplementor.of(similarImplementor));
+
+        // POSIX REGEX
+        final MethodImplementor posixRegexImplementor =
+            new MethodImplementor(BuiltInMethod.POSIX_REGEX.method,
+                NullPolicy.STRICT, false);
+        map.put(SqlStdOperatorTable.POSIX_REGEX_CASE_INSENSITIVE,
+            posixRegexImplementor);
+        map.put(SqlStdOperatorTable.POSIX_REGEX_CASE_SENSITIVE,
+            posixRegexImplementor);
+        map.put(SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_INSENSITIVE,
+            NotImplementor.of(posixRegexImplementor));
+        map.put(SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_SENSITIVE,
+            NotImplementor.of(posixRegexImplementor));
+        map.put(REGEXP_REPLACE, new RegexpReplaceImplementor());
+
+        // Multisets & arrays
+        defineMethod(CARDINALITY, BuiltInMethod.COLLECTION_SIZE.method,
+            NullPolicy.STRICT);
+        defineMethod(SLICE, BuiltInMethod.SLICE.method, NullPolicy.NONE);
+        defineMethod(ELEMENT, BuiltInMethod.ELEMENT.method, NullPolicy.STRICT);
+        defineMethod(STRUCT_ACCESS, BuiltInMethod.STRUCT_ACCESS.method, NullPolicy.ANY);
+        defineMethod(MEMBER_OF, BuiltInMethod.MEMBER_OF.method, NullPolicy.NONE);
+        final MethodImplementor isEmptyImplementor =
+            new MethodImplementor(BuiltInMethod.IS_EMPTY.method, NullPolicy.NONE,
+                false);
+        map.put(IS_EMPTY, isEmptyImplementor);
+        map.put(IS_NOT_EMPTY, NotImplementor.of(isEmptyImplementor));
+        final MethodImplementor isASetImplementor =
+            new MethodImplementor(BuiltInMethod.IS_A_SET.method, NullPolicy.NONE,
+                false);
+        map.put(IS_A_SET, isASetImplementor);
+        map.put(IS_NOT_A_SET, NotImplementor.of(isASetImplementor));
+        defineMethod(MULTISET_INTERSECT_DISTINCT,
+            BuiltInMethod.MULTISET_INTERSECT_DISTINCT.method, NullPolicy.NONE);
+        defineMethod(MULTISET_INTERSECT,
+            BuiltInMethod.MULTISET_INTERSECT_ALL.method, NullPolicy.NONE);
+        defineMethod(MULTISET_EXCEPT_DISTINCT,
+            BuiltInMethod.MULTISET_EXCEPT_DISTINCT.method, NullPolicy.NONE);
+        defineMethod(MULTISET_EXCEPT, BuiltInMethod.MULTISET_EXCEPT_ALL.method, NullPolicy.NONE);
+        defineMethod(MULTISET_UNION_DISTINCT,
+            BuiltInMethod.MULTISET_UNION_DISTINCT.method, NullPolicy.NONE);
+        defineMethod(MULTISET_UNION, BuiltInMethod.MULTISET_UNION_ALL.method, NullPolicy.NONE);
+        final MethodImplementor subMultisetImplementor =
+            new MethodImplementor(BuiltInMethod.SUBMULTISET_OF.method, NullPolicy.NONE, false);
+        map.put(SUBMULTISET_OF, subMultisetImplementor);
+        map.put(NOT_SUBMULTISET_OF, NotImplementor.of(subMultisetImplementor));
+
+        map.put(COALESCE, new CoalesceImplementor());
+        map.put(CAST, new CastImplementor());
+        map.put(DATE, new CastImplementor());
+
+        map.put(REINTERPRET, new ReinterpretImplementor());
+
+        final RexCallImplementor value = new ValueConstructorImplementor();
+        map.put(MAP_VALUE_CONSTRUCTOR, value);
+        map.put(ARRAY_VALUE_CONSTRUCTOR, value);
+        map.put(ITEM, new ItemImplementor());
+
+        map.put(DEFAULT, new DefaultImplementor());
+
+        // Sequences
+        defineMethod(CURRENT_VALUE, BuiltInMethod.SEQUENCE_CURRENT_VALUE.method,
+            NullPolicy.STRICT);
+        defineMethod(NEXT_VALUE, BuiltInMethod.SEQUENCE_NEXT_VALUE.method,
+            NullPolicy.STRICT);
+
+        // Compression Operators
+        defineMethod(COMPRESS, BuiltInMethod.COMPRESS.method, NullPolicy.ARG0);
+
+        // Xml Operators
+        defineMethod(EXTRACT_VALUE, BuiltInMethod.EXTRACT_VALUE.method, NullPolicy.ARG0);
+        defineMethod(XML_TRANSFORM, BuiltInMethod.XML_TRANSFORM.method, NullPolicy.ARG0);
+        defineMethod(EXTRACT_XML, BuiltInMethod.EXTRACT_XML.method, NullPolicy.ARG0);
+        defineMethod(EXISTS_NODE, BuiltInMethod.EXISTS_NODE.method, NullPolicy.ARG0);
+
+        // Json Operators
+        defineMethod(JSON_VALUE_EXPRESSION,
+            BuiltInMethod.JSON_VALUE_EXPRESSION.method, NullPolicy.STRICT);
+        defineMethod(JSON_EXISTS, BuiltInMethod.JSON_EXISTS.method, NullPolicy.ARG0);
+        map.put(JSON_VALUE,
+            new JsonValueImplementor(BuiltInMethod.JSON_VALUE.method));
+        defineMethod(JSON_QUERY, BuiltInMethod.JSON_QUERY.method, NullPolicy.ARG0);
+        defineMethod(JSON_TYPE, BuiltInMethod.JSON_TYPE.method, NullPolicy.ARG0);
+        defineMethod(JSON_DEPTH, BuiltInMethod.JSON_DEPTH.method, NullPolicy.ARG0);
+        defineMethod(JSON_KEYS, BuiltInMethod.JSON_KEYS.method, NullPolicy.ARG0);
+        defineMethod(JSON_PRETTY, BuiltInMethod.JSON_PRETTY.method, NullPolicy.ARG0);
+        defineMethod(JSON_LENGTH, BuiltInMethod.JSON_LENGTH.method, NullPolicy.ARG0);
+        defineMethod(JSON_REMOVE, BuiltInMethod.JSON_REMOVE.method, NullPolicy.ARG0);
+        defineMethod(JSON_STORAGE_SIZE, BuiltInMethod.JSON_STORAGE_SIZE.method, NullPolicy.ARG0);
+        defineMethod(JSON_OBJECT, BuiltInMethod.JSON_OBJECT.method, NullPolicy.NONE);
+        defineMethod(JSON_ARRAY, BuiltInMethod.JSON_ARRAY.method, NullPolicy.NONE);
+        map.put(IS_JSON_VALUE,
+            new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method,
+                NullPolicy.NONE, false));
+        map.put(IS_JSON_OBJECT,
+            new MethodImplementor(BuiltInMethod.IS_JSON_OBJECT.method,
+                NullPolicy.NONE, false));
+        map.put(IS_JSON_ARRAY,
+            new MethodImplementor(BuiltInMethod.IS_JSON_ARRAY.method,
+                NullPolicy.NONE, false));
+        map.put(IS_JSON_SCALAR,
+            new MethodImplementor(BuiltInMethod.IS_JSON_SCALAR.method,
+                NullPolicy.NONE, false));
+        map.put(IS_NOT_JSON_VALUE,
+            NotImplementor.of(
+                new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method,
+                    NullPolicy.NONE, false)));
+        map.put(IS_NOT_JSON_OBJECT,
+            NotImplementor.of(
+                new MethodImplementor(BuiltInMethod.IS_JSON_OBJECT.method,
+                    NullPolicy.NONE, false)));
+        map.put(IS_NOT_JSON_ARRAY,
+            NotImplementor.of(
+                new MethodImplementor(BuiltInMethod.IS_JSON_ARRAY.method,
+                    NullPolicy.NONE, false)));
+        map.put(IS_NOT_JSON_SCALAR,
+            NotImplementor.of(
+                new MethodImplementor(BuiltInMethod.IS_JSON_SCALAR.method,
+                    NullPolicy.NONE, false)));
+
+        // System functions
+        final SystemFunctionImplementor systemFunctionImplementor = new SystemFunctionImplementor();
+        map.put(USER, systemFunctionImplementor);
+        map.put(CURRENT_USER, systemFunctionImplementor);
+        map.put(SESSION_USER, systemFunctionImplementor);
+        map.put(SYSTEM_USER, systemFunctionImplementor);
+        map.put(CURRENT_PATH, systemFunctionImplementor);
+        map.put(CURRENT_ROLE, systemFunctionImplementor);
+        map.put(CURRENT_CATALOG, systemFunctionImplementor);
+        map.put(SYSTEM_RANGE, systemFunctionImplementor);
+
+        // Current time functions
+        map.put(CURRENT_TIME, systemFunctionImplementor);
+        map.put(CURRENT_TIMESTAMP, systemFunctionImplementor);
+        map.put(CURRENT_DATE, systemFunctionImplementor);
+        map.put(LOCALTIME, systemFunctionImplementor);
+        map.put(LOCALTIMESTAMP, systemFunctionImplementor);
+    }
+
+    /** */
+    private <T> Supplier<T> constructorSupplier(Class<T> klass) {
+        final Constructor<T> constructor;
+        try {
+            constructor = klass.getDeclaredConstructor();
+        }
+        catch (NoSuchMethodException e) {
+            throw new IllegalArgumentException(
+                klass + " should implement zero arguments constructor");
+        }
+        return () -> {
+            try {
+                return constructor.newInstance();
+            }
+            catch (InstantiationException | IllegalAccessException
+                | InvocationTargetException e) {
+                throw new IllegalStateException(
+                    "Error while creating aggregate implementor " + constructor, e);
+            }
+        };
+    }
+
+    /** */
+    private void defineMethod(SqlOperator operator, String functionName, NullPolicy nullPolicy) {
+        map.put(operator, new MethodNameImplementor(functionName, nullPolicy, false));
+    }
+
+    /** */
+    private void defineMethod(SqlOperator operator, Method method, NullPolicy nullPolicy) {
+        map.put(operator, new MethodImplementor(method, nullPolicy, false));
+    }
+
+    /** */
+    private void defineUnary(SqlOperator operator, ExpressionType expressionType,
+        NullPolicy nullPolicy, String backupMethodName) {
+        map.put(operator, new UnaryImplementor(expressionType, nullPolicy, backupMethodName));
+    }
+
+    /** */
+    private void defineBinary(SqlOperator operator, ExpressionType expressionType,
+        NullPolicy nullPolicy, String backupMethodName) {
+        map.put(operator,
+            new BinaryImplementor(nullPolicy, true, expressionType,
+                backupMethodName));
+    }
+
+    /** */
+    private static RexCallImplementor wrapAsRexCallImplementor(
+        final CallImplementor implementor) {
+        return new AbstractRexCallImplementor(NullPolicy.NONE, false) {
+            @Override String getVariableName() {
+                return "udf";
+            }
+
+            @Override Expression implementSafe(RexToLixTranslator translator,
+                RexCall call, List<Expression> argValueList) {
+                return implementor.implement(translator, call, RexImpTable.NullAs.NULL);
+            }
+        };
+    }
+
+    /** */
+    public RexCallImplementor get(final SqlOperator operator) {
+        if (operator instanceof SqlUserDefinedFunction) {
+            org.apache.calcite.schema.Function udf =
+                ((SqlUserDefinedFunction)operator).getFunction();
+            if (!(udf instanceof ImplementableFunction)) {
+                throw new IllegalStateException("User defined function " + operator
+                    + " must implement ImplementableFunction");
+            }
+            CallImplementor implementor = ((ImplementableFunction)udf).getImplementor();
+            return wrapAsRexCallImplementor(implementor);
+        }
+        else if (operator instanceof SqlTypeConstructorFunction)
+            return map.get(SqlStdOperatorTable.ROW);
+
+        return map.get(operator);
+    }
+
+    /** */
+    static Expression optimize(Expression expression) {
+        return expression.accept(new OptimizeShuttle());
+    }
+
+    /** */
+    static Expression optimize2(Expression operand, Expression expression) {
+        if (Primitive.is(operand.getType())) {
+            // Primitive values cannot be null
+            return optimize(expression);
+        }
+
+        return optimize(
+            Expressions.condition(
+                Expressions.equal(operand, NULL_EXPR),
+                NULL_EXPR,
+                expression));
+    }
+
+    /** */
+    private static RelDataType toSql(RelDataTypeFactory typeFactory,
+        RelDataType type) {
+        if (type instanceof RelDataTypeFactoryImpl.JavaType) {
+            final SqlTypeName typeName = type.getSqlTypeName();
+            if (typeName != null && typeName != SqlTypeName.OTHER) {
+                return typeFactory.createTypeWithNullability(
+                    typeFactory.createSqlType(typeName),
+                    type.isNullable());
+            }
+        }
+        return type;
+    }
+
+    /** */
+    private static <E> boolean allSame(List<E> list) {
+        E prev = null;
+        for (E e : list) {
+            if (prev != null && !prev.equals(e))
+                return false;
+
+            prev = e;
+        }
+        return true;
+    }
+
+    /**
+     * Strategy what an operator should return if one of its arguments is null.
+     */
+    public enum NullAs {
+        /**
+         * The most common policy among the SQL built-in operators. If one of the arguments is null, returns null.
+         */
+        NULL,
+
+        /**
+         * If one of the arguments is null, the function returns false. Example: {@code IS NOT NULL}.
+         */
+        FALSE,
+
+        /**
+         * If one of the arguments is null, the function returns true. Example: {@code IS NULL}.
+         */
+        TRUE,
+
+        /**
+         * It is not possible for any of the arguments to be null.  If the argument type is nullable, the enclosing code
+         * will already have performed a not-null check. This may allow the operator implementor to generate a more
+         * efficient implementation, for example, by avoiding boxing or unboxing.
+         */
+        NOT_POSSIBLE,
+
+        /** Return false if result is not null, true if result is null. */
+        IS_NULL,
+
+        /** Return true if result is not null, false if result is null. */
+        IS_NOT_NULL;
+
+        public static NullAs of(boolean nullable) {
+            return nullable ? NULL : NOT_POSSIBLE;
+        }
+
+        /**
+         * Adapts an expression with "normal" result to one that adheres to this particular policy.
+         */
+        public Expression handle(Expression x) {
+            switch (Primitive.flavor(x.getType())) {
+                case PRIMITIVE:
+                    // Expression cannot be null. We can skip any runtime checks.
+                    switch (this) {
+                        case NULL:
+                        case NOT_POSSIBLE:
+                        case FALSE:
+                        case TRUE:
+                            return x;
+                        case IS_NULL:
+                            return FALSE_EXPR;
+                        case IS_NOT_NULL:
+                            return TRUE_EXPR;
+                        default:
+                            throw new AssertionError();
+                    }
+                case BOX:
+                    switch (this) {
+                        case NOT_POSSIBLE:
+                            return ConverterUtils.convert(x,
+                                Primitive.ofBox(x.getType()).primitiveClass);
+                    }
+                    // fall through
+            }
+            switch (this) {
+                case NULL:
+                case NOT_POSSIBLE:
+                    return x;
+                case FALSE:
+                    return Expressions.call(BuiltInMethod.IS_TRUE.method, x);
+                case TRUE:
+                    return Expressions.call(BuiltInMethod.IS_NOT_FALSE.method, x);
+                case IS_NULL:
+                    return Expressions.equal(x, NULL_EXPR);
+                case IS_NOT_NULL:
+                    return Expressions.notEqual(x, NULL_EXPR);
+                default:
+                    throw new AssertionError();
+            }
+        }
+    }
+
+    /** */
+    static Expression getDefaultValue(Type type) {
+        if (Primitive.is(type)) {
+            Primitive p = Primitive.of(type);
+            return Expressions.constant(p.defaultValue, type);
+        }
+        return Expressions.constant(null, type);
+    }
+
+    /**
+     * Multiplies an expression by a constant and divides by another constant, optimizing appropriately.
+     *
+     * <p>For example, {@code multiplyDivide(e, 10, 1000)} returns
+     * {@code e / 100}.
+     */
+    public static Expression multiplyDivide(Expression e, BigDecimal multiplier,
+        BigDecimal divider) {
+        if (multiplier.equals(BigDecimal.ONE)) {
+            if (divider.equals(BigDecimal.ONE))
+                return e;
+
+            return Expressions.divide(e,
+                Expressions.constant(divider.intValueExact()));
+        }
+        final BigDecimal x =
+            multiplier.divide(divider, RoundingMode.UNNECESSARY);
+        switch (x.compareTo(BigDecimal.ONE)) {
+            case 0:
+                return e;
+            case 1:
+                return Expressions.multiply(e, Expressions.constant(x.intValueExact()));
+            case -1:
+                return multiplyDivide(e, BigDecimal.ONE, x);
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    /** Implementor for the {@code TRIM} function. */
+    private static class TrimImplementor extends AbstractRexCallImplementor {
+        /** */
+        TrimImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "trim";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            final boolean strict = !translator.conformance.allowExtendedTrim();
+            final Object value = translator.getLiteralValue(argValueList.get(0));
+            SqlTrimFunction.Flag flag = (SqlTrimFunction.Flag)value;
+            return Expressions.call(
+                BuiltInMethod.TRIM.method,
+                Expressions.constant(
+                    flag == SqlTrimFunction.Flag.BOTH
+                        || flag == SqlTrimFunction.Flag.LEADING),
+                Expressions.constant(
+                    flag == SqlTrimFunction.Flag.BOTH
+                        || flag == SqlTrimFunction.Flag.TRAILING),
+                argValueList.get(1),
+                argValueList.get(2),
+                Expressions.constant(strict));
+        }
+    }
+
+    /**
+     * Implementor for the {@code MONTHNAME} and {@code DAYNAME} functions. Each takes a {@link java.util.Locale}
+     * argument.
+     */
+    private static class PeriodNameImplementor extends MethodNameImplementor {
+        /** */
+        private final BuiltInMethod timestampMethod;
+
+        /** */
+        private final BuiltInMethod dateMethod;
+
+        /** */
+        PeriodNameImplementor(String methodName, BuiltInMethod timestampMethod,
+            BuiltInMethod dateMethod) {
+            super(methodName, NullPolicy.STRICT, false);
+            this.timestampMethod = timestampMethod;
+            this.dateMethod = dateMethod;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "periodName";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            Expression operand = argValueList.get(0);
+            final RelDataType type = call.operands.get(0).getType();
+            switch (type.getSqlTypeName()) {
+                case TIMESTAMP:
+                    return getExpression(translator, operand, timestampMethod);
+                case DATE:
+                    return getExpression(translator, operand, dateMethod);
+                default:
+                    throw new AssertionError("unknown type " + type);
+            }
+        }
+
+        /** */
+        protected Expression getExpression(RexToLixTranslator translator,
+            Expression operand, BuiltInMethod builtInMethod) {
+            final MethodCallExpression locale =
+                Expressions.call(BuiltInMethod.LOCALE.method, translator.getRoot());
+            return Expressions.call(builtInMethod.method.getDeclaringClass(),
+                builtInMethod.method.getName(), operand, locale);
+        }
+    }
+
+    /** Implementor for the {@code FLOOR} and {@code CEIL} functions. */
+    private static class FloorImplementor extends MethodNameImplementor {
+        /** */
+        final Method timestampMethod;
+
+        /** */
+        final Method dateMethod;
+
+        /** */
+        FloorImplementor(String methodName, Method timestampMethod, Method dateMethod) {
+            super(methodName, NullPolicy.STRICT, false);
+            this.timestampMethod = timestampMethod;
+            this.dateMethod = dateMethod;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "floor";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            switch (call.getOperands().size()) {
+                case 1:
+                    switch (call.getType().getSqlTypeName()) {
+                        case BIGINT:
+                        case INTEGER:
+                        case SMALLINT:
+                        case TINYINT:
+                            return argValueList.get(0);
+                        default:
+                            return super.implementSafe(translator, call, argValueList);
+                    }
+
+                case 2:
+                    final Type type;
+                    final Method floorMethod;
+                    final boolean preFloor;
+                    Expression operand = argValueList.get(0);
+                    switch (call.getType().getSqlTypeName()) {
+                        case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                            operand = Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+                            // fall through
+                        case TIMESTAMP:
+                            type = long.class;
+                            floorMethod = timestampMethod;
+                            preFloor = true;
+                            break;
+                        default:
+                            type = int.class;
+                            floorMethod = dateMethod;
+                            preFloor = false;
+                    }
+                    final TimeUnitRange timeUnitRange =
+                        (TimeUnitRange)translator.getLiteralValue(argValueList.get(1));
+                    switch (timeUnitRange) {
+                        case YEAR:
+                        case QUARTER:
+                        case MONTH:
+                        case WEEK:
+                        case DAY:
+                            final Expression operand1 =
+                                preFloor ? call(operand, type, TimeUnit.DAY) : operand;
+                            return Expressions.call(floorMethod,
+                                translator.getLiteral(argValueList.get(1)), operand1);
+                        case NANOSECOND:
+                        default:
+                            return call(operand, type, timeUnitRange.startUnit);
+                    }
+
+                default:
+                    throw new AssertionError();
+            }
+        }
+
+        /** */
+        private Expression call(Expression operand, Type type,
+            TimeUnit timeUnit) {
+            return Expressions.call(SqlFunctions.class, methodName,
+                ConverterUtils.convert(operand, type),
+                ConverterUtils.convert(
+                    Expressions.constant(timeUnit.multiplier), type));
+        }
+    }
+
+    /** Implementor for a function that generates calls to a given method. */
+    private static class MethodImplementor extends AbstractRexCallImplementor {
+        /** */
+        protected final Method method;
+
+        /** */
+        MethodImplementor(Method method, NullPolicy nullPolicy, boolean harmonize) {
+            super(nullPolicy, harmonize);
+            this.method = method;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "method_call";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList) {
+            final Expression expression;
+            Class clazz = method.getDeclaringClass();
+            if (Modifier.isStatic(method.getModifiers()))
+                expression = EnumUtils.call(clazz, method.getName(), argValueList);
+            else {
+                expression = EnumUtils.call(clazz, method.getName(),
+                    Util.skip(argValueList, 1), argValueList.get(0));
+            }
+            return expression;
+        }
+    }
+
+    /**
+     * Implementor for JSON_VALUE function, convert to solid format "JSON_VALUE(json_doc, path, empty_behavior,
+     * empty_default, error_behavior, error default)" in order to simplify the runtime implementation.
+     *
+     * <p>We should avoid this when we support
+     * variable arguments function.
+     */
+    private static class JsonValueImplementor extends MethodImplementor {
+        /** */
+        JsonValueImplementor(Method method) {
+            super(method, NullPolicy.ARG0, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList) {
+            final List<Expression> newOperands = new ArrayList<>();
+            newOperands.add(argValueList.get(0));
+            newOperands.add(argValueList.get(1));
+            List<Expression> leftExprs = Util.skip(argValueList, 2);
+            // Default value for JSON_VALUE behaviors.
+            Expression emptyBehavior = Expressions.constant(SqlJsonValueEmptyOrErrorBehavior.NULL);
+            Expression defaultValueOnEmpty = Expressions.constant(null);
+            Expression errorBehavior = Expressions.constant(SqlJsonValueEmptyOrErrorBehavior.NULL);
+            Expression defaultValueOnError = Expressions.constant(null);
+            // Patched up with user defines.
+            if (!leftExprs.isEmpty()) {
+                for (int i = 0; i < leftExprs.size(); i++) {
+                    Expression expr = leftExprs.get(i);
+                    final Object exprVal = translator.getLiteralValue(expr);
+                    if (exprVal != null) {
+                        int defaultSymbolIdx = i - 2;
+                        if (exprVal == SqlJsonEmptyOrError.EMPTY) {
+                            if (defaultSymbolIdx >= 0
+                                && translator.getLiteralValue(leftExprs.get(defaultSymbolIdx))
+                                == SqlJsonValueEmptyOrErrorBehavior.DEFAULT) {
+                                defaultValueOnEmpty = leftExprs.get(i - 1);
+                                emptyBehavior = leftExprs.get(defaultSymbolIdx);
+                            }
+                            else
+                                emptyBehavior = leftExprs.get(i - 1);
+                        }
+                        else if (exprVal == SqlJsonEmptyOrError.ERROR) {
+                            if (defaultSymbolIdx >= 0
+                                && translator.getLiteralValue(leftExprs.get(defaultSymbolIdx))
+                                == SqlJsonValueEmptyOrErrorBehavior.DEFAULT) {
+                                defaultValueOnError = leftExprs.get(i - 1);
+                                errorBehavior = leftExprs.get(defaultSymbolIdx);
+                            }
+                            else
+                                errorBehavior = leftExprs.get(i - 1);
+                        }
+                    }
+                }
+            }
+            newOperands.add(emptyBehavior);
+            newOperands.add(defaultValueOnEmpty);
+            newOperands.add(errorBehavior);
+            newOperands.add(defaultValueOnError);
+            Class clazz = method.getDeclaringClass();
+            final Expression expression = EnumUtils.call(clazz, method.getName(), newOperands);
+
+            final Type returnType =
+                translator.typeFactory.getJavaClass(call.getType());
+            return EnumUtils.convert(expression, returnType);
+        }
+    }
+
+    /**
+     * Implementor for SQL functions that generates calls to a given method name.
+     *
+     * <p>Use this, as opposed to {@link MethodImplementor}, if the SQL function
+     * is overloaded; then you can use one implementor for several overloads.
+     */
+    private static class MethodNameImplementor extends AbstractRexCallImplementor {
+        /** */
+        protected final String methodName;
+
+        /** */
+        MethodNameImplementor(String methodName,
+            NullPolicy nullPolicy, boolean harmonize) {
+            super(nullPolicy, harmonize);
+            this.methodName = methodName;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "method_name_call";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList) {
+            return EnumUtils.call(
+                SqlFunctions.class,
+                methodName,
+                argValueList);
+        }
+    }
+
+    /** Implementor for binary operators. */
+    private static class BinaryImplementor extends AbstractRexCallImplementor {
+        /**
+         * Types that can be arguments to comparison operators such as {@code <}.
+         */
+        private static final List<Primitive> COMP_OP_TYPES =
+            ImmutableList.of(
+                Primitive.BYTE,
+                Primitive.CHAR,
+                Primitive.SHORT,
+                Primitive.INT,
+                Primitive.LONG,
+                Primitive.FLOAT,
+                Primitive.DOUBLE);
+
+        /** */
+        private static final List<SqlBinaryOperator> COMPARISON_OPERATORS =
+            ImmutableList.of(
+                SqlStdOperatorTable.LESS_THAN,
+                SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+                SqlStdOperatorTable.GREATER_THAN,
+                SqlStdOperatorTable.GREATER_THAN_OR_EQUAL);
+
+        /** */
+        private static final List<SqlBinaryOperator> EQUALS_OPERATORS =
+            ImmutableList.of(
+                SqlStdOperatorTable.EQUALS,
+                SqlStdOperatorTable.NOT_EQUALS);
+
+        /** */
+        public static final String METHOD_POSTFIX_FOR_ANY_TYPE = "Any";
+
+        /** */
+        private final ExpressionType expressionType;
+
+        /** */
+        private final String backupMethodName;
+
+        /** */
+        BinaryImplementor(NullPolicy nullPolicy, boolean harmonize,
+            ExpressionType expressionType, String backupMethodName) {
+            super(nullPolicy, harmonize);
+            this.expressionType = expressionType;
+            this.backupMethodName = backupMethodName;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "binary_call";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(
+            final RexToLixTranslator translator,
+            final RexCall call,
+            final List<Expression> argValueList) {
+            // neither nullable:
+            //   return x OP y
+            // x nullable
+            //   null_returns_null
+            //     return x == null ? null : x OP y
+            //   ignore_null
+            //     return x == null ? null : y
+            // x, y both nullable
+            //   null_returns_null
+            //     return x == null || y == null ? null : x OP y
+            //   ignore_null
+            //     return x == null ? y : y == null ? x : x OP y
+            if (backupMethodName != null) {
+                // If one or both operands have ANY type, use the late-binding backup
+                // method.
+                if (anyAnyOperands(call))
+                    return callBackupMethodAnyType(argValueList);
+
+                final Type type0 = argValueList.get(0).getType();
+                final Type type1 = argValueList.get(1).getType();
+                final SqlBinaryOperator op = (SqlBinaryOperator)call.getOperator();
+                final RelDataType relDataType0 = call.getOperands().get(0).getType();
+                final Expression fieldComparator = generateCollatorExpression(relDataType0.getCollation());
+                if (fieldComparator != null)
+                    argValueList.add(fieldComparator);
+
+                final Primitive primitive = Primitive.ofBoxOr(type0);
+                if (primitive == null
+                    || type1 == BigDecimal.class
+                    || COMPARISON_OPERATORS.contains(op)
+                    && !COMP_OP_TYPES.contains(primitive)) {
+                    return Expressions.call(SqlFunctions.class, backupMethodName,
+                        argValueList);
+                }
+                // When checking equals or not equals on two primitive boxing classes
+                // (i.e. Long x, Long y), we should fall back to call `SqlFunctions.eq(x, y)`
+                // or `SqlFunctions.ne(x, y)`, rather than `x == y`
+                final Primitive boxPrimitive0 = Primitive.ofBox(type0);
+                final Primitive boxPrimitive1 = Primitive.ofBox(type1);
+                if (EQUALS_OPERATORS.contains(op)
+                    && boxPrimitive0 != null && boxPrimitive1 != null) {
+                    return Expressions.call(SqlFunctions.class, backupMethodName,
+                        argValueList);
+                }
+            }
+            return Expressions.makeBinary(expressionType,
+                argValueList.get(0), argValueList.get(1));
+        }
+
+        /** Returns whether any of a call's operands have ANY type. */
+        private boolean anyAnyOperands(RexCall call) {
+            for (RexNode operand : call.operands) {
+                if (operand.getType().getSqlTypeName() == SqlTypeName.ANY)
+                    return true;
+            }
+            return false;
+        }
+
+        /** */
+        private Expression callBackupMethodAnyType(List<Expression> expressions) {
+            final String backupMethodNameForAnyType =
+                backupMethodName + METHOD_POSTFIX_FOR_ANY_TYPE;
+
+            // one or both of parameter(s) is(are) ANY type
+            final Expression expression0 = maybeBox(expressions.get(0));
+            final Expression expression1 = maybeBox(expressions.get(1));
+            return Expressions.call(SqlFunctions.class, backupMethodNameForAnyType,
+                expression0, expression1);
+        }
+
+        /** */
+        private Expression maybeBox(Expression expression) {
+            final Primitive primitive = Primitive.of(expression.getType());
+            if (primitive != null)
+                expression = Expressions.box(expression, primitive);
+
+            return expression;
+        }
+    }
+
+    /** Implementor for unary operators. */
+    private static class UnaryImplementor extends AbstractRexCallImplementor {
+        /** */
+        private final ExpressionType expressionType;
+
+        /** */
+        private final String backupMethodName;
+
+        /** */
+        UnaryImplementor(ExpressionType expressionType, NullPolicy nullPolicy,
+            String backupMethodName) {
+            super(nullPolicy, false);
+            this.expressionType = expressionType;
+            this.backupMethodName = backupMethodName;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "unary_call";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList) {
+            final Expression argValue = argValueList.get(0);
+
+            final Expression e;
+            //Special case for implementing unary minus with BigDecimal type
+            //for other data type(except BigDecimal) '-' operator is OK, but for
+            //BigDecimal, we should call negate method of BigDecimal
+            if (expressionType == ExpressionType.Negate && argValue.type == BigDecimal.class
+                && null != backupMethodName)
+                e = Expressions.call(argValue, backupMethodName);
+            else
+                e = Expressions.makeUnary(expressionType, argValue);
+
+            if (e.type.equals(argValue.type))
+                return e;
+            // Certain unary operators do not preserve type. For example, the "-"
+            // operator applied to a "byte" expression returns an "int".
+            return Expressions.convert_(e, argValue.type);
+        }
+    }
+
+    /** Implementor for the {@code EXTRACT(unit FROM datetime)} function. */
+    private static class ExtractImplementor extends AbstractRexCallImplementor {
+        /** */
+        ExtractImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "extract";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            final TimeUnitRange timeUnitRange =
+                (TimeUnitRange)translator.getLiteralValue(argValueList.get(0));
+            final TimeUnit unit = timeUnitRange.startUnit;
+            Expression operand = argValueList.get(1);
+            final SqlTypeName sqlTypeName =
+                call.operands.get(1).getType().getSqlTypeName();
+            switch (unit) {
+                case MILLENNIUM:
+                case CENTURY:
+                case YEAR:
+                case QUARTER:
+                case MONTH:
+                case DAY:
+                case DOW:
+                case DECADE:
+                case DOY:
+                case ISODOW:
+                case ISOYEAR:
+                case WEEK:
+                    switch (sqlTypeName) {
+                        case INTERVAL_YEAR:
+                        case INTERVAL_YEAR_MONTH:
+                        case INTERVAL_MONTH:
+                        case INTERVAL_DAY:
+                        case INTERVAL_DAY_HOUR:
+                        case INTERVAL_DAY_MINUTE:
+                        case INTERVAL_DAY_SECOND:
+                        case INTERVAL_HOUR:
+                        case INTERVAL_HOUR_MINUTE:
+                        case INTERVAL_HOUR_SECOND:
+                        case INTERVAL_MINUTE:
+                        case INTERVAL_MINUTE_SECOND:
+                        case INTERVAL_SECOND:
+                            break;
+                        case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                            operand = Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+                            // fall through
+                        case TIMESTAMP:
+                            operand = Expressions.call(BuiltInMethod.FLOOR_DIV.method,
+                                operand, Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+                            // fall through
+                        case DATE:
+                            return Expressions.call(BuiltInMethod.UNIX_DATE_EXTRACT.method,
+                                argValueList.get(0), operand);
+                        default:
+                            throw new AssertionError("unexpected " + sqlTypeName);
+                    }
+                    break;
+                case MILLISECOND:
+                case MICROSECOND:
+                case NANOSECOND:
+                    if (sqlTypeName == SqlTypeName.DATE)
+                        return Expressions.constant(0L);
+
+                    operand = mod(operand, TimeUnit.MINUTE.multiplier.longValue());
+                    return Expressions.multiply(
+                        operand, Expressions.constant((long)(1 / unit.multiplier.doubleValue())));
+                case EPOCH:
+                    switch (sqlTypeName) {
+                        case DATE:
+                            // convert to milliseconds
+                            operand = Expressions.multiply(operand,
+                                Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+                            // fall through
+                        case TIMESTAMP:
+                            // convert to seconds
+                            return Expressions.divide(operand,
+                                Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+                        case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                            operand = Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+                            return Expressions.divide(operand,
+                                Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+                        case INTERVAL_YEAR:
+                        case INTERVAL_YEAR_MONTH:
+                        case INTERVAL_MONTH:
+                        case INTERVAL_DAY:
+                        case INTERVAL_DAY_HOUR:
+                        case INTERVAL_DAY_MINUTE:
+                        case INTERVAL_DAY_SECOND:
+                        case INTERVAL_HOUR:
+                        case INTERVAL_HOUR_MINUTE:
+                        case INTERVAL_HOUR_SECOND:
+                        case INTERVAL_MINUTE:
+                        case INTERVAL_MINUTE_SECOND:
+                        case INTERVAL_SECOND:
+                            // no convertlet conversion, pass it as extract
+                            throw new AssertionError("unexpected " + sqlTypeName);
+                    }
+                    break;
+                case HOUR:
+                case MINUTE:
+                case SECOND:
+                    switch (sqlTypeName) {
+                        case DATE:
+                            return Expressions.multiply(operand, Expressions.constant(0L));
+                    }
+                    break;
+            }
+
+            operand = mod(operand, getFactor(unit));
+            if (unit == TimeUnit.QUARTER)
+                operand = Expressions.subtract(operand, Expressions.constant(1L));
+
+            operand = Expressions.divide(operand,
+                Expressions.constant(unit.multiplier.longValue()));
+            if (unit == TimeUnit.QUARTER)
+                operand = Expressions.add(operand, Expressions.constant(1L));
+
+            return operand;
+        }
+    }
+
+    /** */
+    private static Expression mod(Expression operand, long factor) {
+        if (factor == 1L)
+            return operand;
+        else {
+            return Expressions.call(BuiltInMethod.FLOOR_MOD.method,
+                operand, Expressions.constant(factor));
+        }
+    }
+
+    /** */
+    private static long getFactor(TimeUnit unit) {
+        switch (unit) {
+            case DAY:
+                return 1L;
+            case HOUR:
+                return TimeUnit.DAY.multiplier.longValue();
+            case MINUTE:
+                return TimeUnit.HOUR.multiplier.longValue();
+            case SECOND:
+                return TimeUnit.MINUTE.multiplier.longValue();
+            case MILLISECOND:
+                return TimeUnit.SECOND.multiplier.longValue();
+            case MONTH:
+                return TimeUnit.YEAR.multiplier.longValue();
+            case QUARTER:
+                return TimeUnit.YEAR.multiplier.longValue();
+            case YEAR:
+            case DECADE:
+            case CENTURY:
+            case MILLENNIUM:
+                return 1L;
+            default:
+                throw Util.unexpected(unit);
+        }
+    }
+
+    /** Implementor for the SQL {@code COALESCE} operator. */
+    private static class CoalesceImplementor extends AbstractRexCallImplementor {
+        /** */
+        CoalesceImplementor() {
+            super(NullPolicy.NONE, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "coalesce";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return implementRecurse(translator, argValueList);
+        }
+
+        /** */
+        private Expression implementRecurse(RexToLixTranslator translator,
+            final List<Expression> argValueList) {
+            if (argValueList.size() == 1)
+                return argValueList.get(0);
+            else {
+                return Expressions.condition(
+                    translator.checkNotNull(argValueList.get(0)),
+                    argValueList.get(0),
+                    implementRecurse(translator, Util.skip(argValueList)));
+            }
+        }
+    }
+
+    /** Implementor for the SQL {@code CAST} operator. */
+    private static class CastImplementor extends AbstractRexCallImplementor {
+        /** */
+        CastImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "cast";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            assert call.getOperands().size() == 1;
+            final RelDataType sourceType = call.getOperands().get(0).getType();
+
+            // Short-circuit if no cast is required
+            RexNode arg = call.getOperands().get(0);
+            if (call.getType().equals(sourceType)) {
+                // No cast required, omit cast
+                return argValueList.get(0);
+            }
+            if (SqlTypeUtil.equalSansNullability(translator.typeFactory,
+                call.getType(), arg.getType())
+                && translator.deref(arg) instanceof RexLiteral) {
+                return RexToLixTranslator.translateLiteral(
+                    (RexLiteral)translator.deref(arg), call.getType(),
+                    translator.typeFactory, NullAs.NULL);
+            }
+            final RelDataType targetType =
+                nullifyType(translator.typeFactory, call.getType(), false);
+            return translator.translateCast(sourceType,
+                targetType, argValueList.get(0));
+        }
+
+        /** */
+        private RelDataType nullifyType(JavaTypeFactory typeFactory,
+            final RelDataType type, final boolean nullable) {
+            if (type instanceof RelDataTypeFactoryImpl.JavaType) {
+                final Primitive primitive = Primitive.ofBox(
+                    ((RelDataTypeFactoryImpl.JavaType)type).getJavaClass());
+                if (primitive != null)
+                    return typeFactory.createJavaType(primitive.primitiveClass);
+            }
+            return typeFactory.createTypeWithNullability(type, nullable);
+        }
+    }
+
+    /** Implementor for the {@code REINTERPRET} internal SQL operator. */
+    private static class ReinterpretImplementor extends AbstractRexCallImplementor {
+        /** */
+        ReinterpretImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "reInterpret";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            assert call.getOperands().size() == 1;
+            return argValueList.get(0);
+        }
+    }
+
+    /** Implementor for a value-constructor. */
+    private static class ValueConstructorImplementor
+        extends AbstractRexCallImplementor {
+
+        /** */
+        ValueConstructorImplementor() {
+            super(NullPolicy.NONE, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "value_constructor";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            SqlKind kind = call.getOperator().getKind();
+            final BlockBuilder blockBuilder = translator.getBlockBuilder();
+            switch (kind) {
+                case MAP_VALUE_CONSTRUCTOR:
+                    Expression map =
+                        blockBuilder.append("map", Expressions.new_(LinkedHashMap.class),
+                            false);
+                    for (int i = 0; i < argValueList.size(); i++) {
+                        Expression key = argValueList.get(i++);
+                        Expression value = argValueList.get(i);
+                        blockBuilder.add(
+                            Expressions.statement(
+                                Expressions.call(map, BuiltInMethod.MAP_PUT.method,
+                                    Expressions.box(key), Expressions.box(value))));
+                    }
+                    return map;
+                case ARRAY_VALUE_CONSTRUCTOR:
+                    Expression lyst =
+                        blockBuilder.append("list", Expressions.new_(ArrayList.class),
+                            false);
+                    for (Expression value : argValueList) {
+                        blockBuilder.add(
+                            Expressions.statement(
+                                Expressions.call(lyst, BuiltInMethod.COLLECTION_ADD.method,
+                                    Expressions.box(value))));
+                    }
+                    return lyst;
+                default:
+                    throw new AssertionError("unexpected: " + kind);
+            }
+        }
+    }
+
+    /** Implementor for the {@code ITEM} SQL operator. */
+    private static class ItemImplementor extends AbstractRexCallImplementor {
+        /** */
+        ItemImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "item";
+        }
+
+        // Since we follow PostgreSQL's semantics that an out-of-bound reference
+        // returns NULL, x[y] can return null even if x and y are both NOT NULL.
+        // (In SQL standard semantics, an out-of-bound reference to an array
+        // throws an exception.)
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            final MethodImplementor implementor =
+                getImplementor(call.getOperands().get(0).getType().getSqlTypeName());
+            return implementor.implementSafe(translator, call, argValueList);
+        }
+
+        /** */
+        private MethodImplementor getImplementor(SqlTypeName sqlTypeName) {
+            switch (sqlTypeName) {
+                case ARRAY:
+                    return new MethodImplementor(BuiltInMethod.ARRAY_ITEM.method, nullPolicy, false);
+                case MAP:
+                    return new MethodImplementor(BuiltInMethod.MAP_ITEM.method, nullPolicy, false);
+                default:
+                    return new MethodImplementor(BuiltInMethod.ANY_ITEM.method, nullPolicy, false);
+            }
+        }
+    }
+
+    /**
+     * Implementor for SQL system functions.
+     *
+     * <p>Several of these are represented internally as constant values, set
+     * per execution.
+     */
+    private static class SystemFunctionImplementor
+        extends AbstractRexCallImplementor {
+        /** */
+        SystemFunctionImplementor() {
+            super(NullPolicy.NONE, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "system_func";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            final SqlOperator op = call.getOperator();
+            final Expression root = translator.getRoot();
+            if (op == CURRENT_USER
+                || op == SESSION_USER
+                || op == USER)
+                return Expressions.call(BuiltInMethod.USER.method, root);
+            else if (op == SYSTEM_USER)
+                return Expressions.call(BuiltInMethod.SYSTEM_USER.method, root);
+            else if (op == CURRENT_PATH
+                || op == CURRENT_ROLE
+                || op == CURRENT_CATALOG) {
+                // By default, the CURRENT_ROLE and CURRENT_CATALOG functions return the
+                // empty string because a role or a catalog has to be set explicitly.
+                return Expressions.constant("");
+            }
+            else if (op == CURRENT_TIMESTAMP)
+                return Expressions.call(BuiltInMethod.CURRENT_TIMESTAMP.method, root);
+            else if (op == CURRENT_TIME)
+                return Expressions.call(BuiltInMethod.CURRENT_TIME.method, root);
+            else if (op == CURRENT_DATE)
+                return Expressions.call(BuiltInMethod.CURRENT_DATE.method, root);
+            else if (op == LOCALTIMESTAMP)
+                return Expressions.call(BuiltInMethod.LOCAL_TIMESTAMP.method, root);
+            else if (op == LOCALTIME)
+                return Expressions.call(BuiltInMethod.LOCAL_TIME.method, root);
+            else if (op == SYSTEM_RANGE) {
+                if (call.getOperands().size() == 2)
+                    return createTableFunctionImplementor(IgniteMethod.SYSTEM_RANGE2.method())
+                        .implement(translator, call, NullAs.NULL);
+
+                if (call.getOperands().size() == 3)
+                    return createTableFunctionImplementor(IgniteMethod.SYSTEM_RANGE3.method())
+                        .implement(translator, call, NullAs.NULL);
+            }
+
+            throw new AssertionError("unknown function " + op);
+        }
+    }
+
+    /** Implementor for the {@code NOT} operator. */
+    private static class NotImplementor extends AbstractRexCallImplementor {
+        /** */
+        private AbstractRexCallImplementor implementor;
+
+        /** */
+        private NotImplementor(AbstractRexCallImplementor implementor) {
+            super(null, false);
+            this.implementor = implementor;
+        }
+
+        /** */
+        static AbstractRexCallImplementor of(AbstractRexCallImplementor implementor) {
+            return new NotImplementor(implementor);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "not";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            final Expression expression =
+                implementor.implementSafe(translator, call, argValueList);
+            return Expressions.not(expression);
+        }
+    }
+
+    /** Implementor for various datetime arithmetic. */
+    private static class DatetimeArithmeticImplementor
+        extends AbstractRexCallImplementor {
+        /** */
+        DatetimeArithmeticImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "dateTime_arithmetic";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            final RexNode operand0 = call.getOperands().get(0);
+            Expression trop0 = argValueList.get(0);
+            final SqlTypeName typeName1 =
+                call.getOperands().get(1).getType().getSqlTypeName();
+            Expression trop1 = argValueList.get(1);
+            final SqlTypeName typeName = call.getType().getSqlTypeName();
+            switch (operand0.getType().getSqlTypeName()) {
+                case DATE:
+                    switch (typeName) {
+                        case TIMESTAMP:
+                            trop0 = Expressions.convert_(
+                                Expressions.multiply(trop0,
+                                    Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                                long.class);
+                            break;
+                        default:
+                            switch (typeName1) {
+                                case INTERVAL_DAY:
+                                case INTERVAL_DAY_HOUR:
+                                case INTERVAL_DAY_MINUTE:
+                                case INTERVAL_DAY_SECOND:
+                                case INTERVAL_HOUR:
+                                case INTERVAL_HOUR_MINUTE:
+                                case INTERVAL_HOUR_SECOND:
+                                case INTERVAL_MINUTE:
+                                case INTERVAL_MINUTE_SECOND:
+                                case INTERVAL_SECOND:
+                                    trop1 = Expressions.convert_(
+                                        Expressions.divide(trop1,
+                                            Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                                        int.class);
+                            }
+                    }
+                    break;
+                case TIME:
+                    trop1 = Expressions.convert_(trop1, int.class);
+                    break;
+            }
+            switch (typeName1) {
+                case INTERVAL_YEAR:
+                case INTERVAL_YEAR_MONTH:
+                case INTERVAL_MONTH:
+                    switch (call.getKind()) {
+                        case MINUS:
+                            trop1 = Expressions.negate(trop1);
+                    }
+                    switch (typeName) {
+                        case TIME:
+                            return Expressions.convert_(trop0, long.class);
+                        default:
+                            final BuiltInMethod method =
+                                operand0.getType().getSqlTypeName() == SqlTypeName.TIMESTAMP
+                                    ? BuiltInMethod.ADD_MONTHS
+                                    : BuiltInMethod.ADD_MONTHS_INT;
+                            return Expressions.call(method.method, trop0, trop1);
+                    }
+
+                case INTERVAL_DAY:
+                case INTERVAL_DAY_HOUR:
+                case INTERVAL_DAY_MINUTE:
+                case INTERVAL_DAY_SECOND:
+                case INTERVAL_HOUR:
+                case INTERVAL_HOUR_MINUTE:
+                case INTERVAL_HOUR_SECOND:
+                case INTERVAL_MINUTE:
+                case INTERVAL_MINUTE_SECOND:
+                case INTERVAL_SECOND:
+                    switch (call.getKind()) {
+                        case MINUS:
+                            return normalize(typeName, Expressions.subtract(trop0, trop1));
+                        default:
+                            return normalize(typeName, Expressions.add(trop0, trop1));
+                    }
+
+                default:
+                    switch (call.getKind()) {
+                        case MINUS:
+                            switch (typeName) {
+                                case INTERVAL_YEAR:
+                                case INTERVAL_YEAR_MONTH:
+                                case INTERVAL_MONTH:
+                                    return Expressions.call(BuiltInMethod.SUBTRACT_MONTHS.method,
+                                        trop0, trop1);
+                            }
+                            TimeUnit fromUnit =
+                                typeName1 == SqlTypeName.DATE ? TimeUnit.DAY : TimeUnit.MILLISECOND;
+                            TimeUnit toUnit = TimeUnit.MILLISECOND;
+                            return multiplyDivide(
+                                Expressions.convert_(Expressions.subtract(trop0, trop1),
+                                    (Class)long.class),
+                                fromUnit.multiplier, toUnit.multiplier);
+                        default:
+                            throw new AssertionError(call);
+                    }
+            }
+        }
+
+        /** Normalizes a TIME value into 00:00:00..23:59:39. */
+        private Expression normalize(SqlTypeName typeName, Expression e) {
+            switch (typeName) {
+                case TIME:
+                    return Expressions.call(BuiltInMethod.FLOOR_MOD.method, e,
+                        Expressions.constant(DateTimeUtils.MILLIS_PER_DAY));
+                default:
+                    return e;
+            }
+        }
+    }
+
+    /** Null-safe implementor of {@code RexCall}s. */
+    public interface RexCallImplementor {
+        /** */
+        RexToLixTranslator.Result implement(
+            RexToLixTranslator translator,
+            RexCall call,
+            List<RexToLixTranslator.Result> arguments);
+    }
+
+    /**
+     * Abstract implementation of the {@link RexCallImplementor} interface.
+     *
+     * <p>It is not always safe to execute the {@link RexCall} directly due to
+     * the special null arguments. Therefore, the generated code logic is conditional correspondingly.
+     *
+     * <p>For example, {@code a + b} will generate two declaration statements:
+     *
+     * <blockquote>
+     * <code>
+     * final Integer xxx_value = (a_isNull || b_isNull) ? null : plus(a, b);<br> final boolean xxx_isNull = xxx_value ==
+     * null;
+     * </code>
+     * </blockquote>
+     */
+    private abstract static class AbstractRexCallImplementor
+        implements RexCallImplementor {
+        /** */
+        final NullPolicy nullPolicy;
+
+        /** */
+        private final boolean harmonize;
+
+        /** */
+        AbstractRexCallImplementor(NullPolicy nullPolicy, boolean harmonize) {
+            this.nullPolicy = nullPolicy;
+            this.harmonize = harmonize;
+        }
+
+        /** {@inheritDoc} */
+        @Override public RexToLixTranslator.Result implement(
+            final RexToLixTranslator translator,
+            final RexCall call,
+            final List<RexToLixTranslator.Result> arguments) {
+            final List<Expression> argIsNullList = new ArrayList<>();
+            final List<Expression> argValueList = new ArrayList<>();
+            for (RexToLixTranslator.Result result : arguments) {
+                argIsNullList.add(result.isNullVariable);
+                argValueList.add(result.valueVariable);
+            }
+            final Expression condition = getCondition(argIsNullList);
+            final ParameterExpression valueVariable =
+                genValueStatement(translator, call, argValueList, condition);
+            final ParameterExpression isNullVariable =
+                genIsNullStatement(translator, valueVariable);
+            return new RexToLixTranslator.Result(isNullVariable, valueVariable);
+        }
+
+        /** */
+        // Variable name facilitates reasoning about issues when necessary
+        abstract String getVariableName();
+
+        /** Figures out conditional expression according to NullPolicy. */
+        Expression getCondition(final List<Expression> argIsNullList) {
+            if (argIsNullList.isEmpty()
+                || nullPolicy == null
+                || nullPolicy == NullPolicy.NONE)
+                return FALSE_EXPR;
+
+            if (nullPolicy == NullPolicy.ARG0)
+                return argIsNullList.get(0);
+
+            return Expressions.foldOr(argIsNullList);
+        }
+
+        /** */
+        // E.g., "final Integer xxx_value = (a_isNull || b_isNull) ? null : plus(a, b)"
+        private ParameterExpression genValueStatement(
+            final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList,
+            final Expression condition) {
+            List<Expression> optimizedArgValueList = argValueList;
+            if (harmonize) {
+                optimizedArgValueList =
+                    harmonize(optimizedArgValueList, translator, call);
+            }
+            optimizedArgValueList = unboxIfNecessary(optimizedArgValueList);
+
+            final Expression callValue =
+                implementSafe(translator, call, optimizedArgValueList);
+
+            // In general, RexCall's type is correct for code generation
+            // and thus we should ensure the consistency.
+            // However, for some special cases (e.g., TableFunction),
+            // the implementation's type is correct, we can't convert it.
+            final SqlOperator op = call.getOperator();
+            final Type returnType = translator.typeFactory.getJavaClass(call.getType());
+            final boolean noConvert = (returnType == null)
+                || (returnType == callValue.getType())
+                || (op instanceof SqlTableFunction)
+                || (op instanceof SqlUserDefinedTableMacro)
+                || (op instanceof SqlUserDefinedTableFunction);
+            final Expression convertedCallValue =
+                noConvert
+                    ? callValue
+                    : ConverterUtils.convert(callValue, returnType);
+
+            final Expression valueExpression =
+                Expressions.condition(condition,
+                    getIfTrue(convertedCallValue.getType()),
+                    convertedCallValue);
+            final ParameterExpression value =
+                Expressions.parameter(convertedCallValue.getType(),
+                    translator.getBlockBuilder().newName(getVariableName() + "_value"));
+            translator.getBlockBuilder().add(
+                Expressions.declare(Modifier.FINAL, value, valueExpression));
+            return value;
+        }
+
+        /** */
+        Expression getIfTrue(Type type) {
+            return getDefaultValue(type);
+        }
+
+        /** */
+        // E.g., "final boolean xxx_isNull = xxx_value == null"
+        private ParameterExpression genIsNullStatement(
+            final RexToLixTranslator translator, final ParameterExpression value) {
+            final ParameterExpression isNullVariable =
+                Expressions.parameter(Boolean.TYPE,
+                    translator.getBlockBuilder().newName(getVariableName() + "_isNull"));
+            final Expression isNullExpression = translator.checkNull(value);
+            translator.getBlockBuilder().add(
+                Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+            return isNullVariable;
+        }
+
+        /** Ensures that operands have identical type. */
+        private List<Expression> harmonize(final List<Expression> argValueList,
+            final RexToLixTranslator translator, final RexCall call) {
+            int nullCount = 0;
+            final List<RelDataType> types = new ArrayList<>();
+            final RelDataTypeFactory typeFactory =
+                translator.builder.getTypeFactory();
+            for (RexNode operand : call.getOperands()) {
+                RelDataType type = operand.getType();
+                type = toSql(typeFactory, type);
+                if (translator.isNullable(operand))
+                    ++nullCount;
+                else
+                    type = typeFactory.createTypeWithNullability(type, false);
+
+                types.add(type);
+            }
+            if (allSame(types)) {
+                // Operands have the same nullability and type. Return them
+                // unchanged.
+                return argValueList;
+            }
+            final RelDataType type = typeFactory.leastRestrictive(types);
+            if (type == null) {
+                // There is no common type. Presumably this is a binary operator with
+                // asymmetric arguments (e.g. interval / integer) which is not intended
+                // to be harmonized.
+                return argValueList;
+            }
+            assert (nullCount > 0) == type.isNullable();
+            final Type javaClass =
+                translator.typeFactory.getJavaClass(type);
+            final List<Expression> harmonizedArgValues = new ArrayList<>();
+            for (Expression argValue : argValueList) {
+                harmonizedArgValues.add(
+                    EnumUtils.convert(argValue, javaClass));
+            }
+            return harmonizedArgValues;
+        }
+
+        /**
+         * Under null check, it is safe to unbox the operands before entering the implementor.
+         */
+        private List<Expression> unboxIfNecessary(final List<Expression> argValueList) {
+            List<Expression> unboxValueList = argValueList;
+            if (nullPolicy == NullPolicy.STRICT || nullPolicy == NullPolicy.ANY
+                || nullPolicy == NullPolicy.SEMI_STRICT) {
+                unboxValueList = argValueList.stream()
+                    .map(this::unboxExpression)
+                    .collect(Collectors.toList());
+            }
+            if (nullPolicy == NullPolicy.ARG0 && !argValueList.isEmpty()) {
+                final Expression unboxArg0 = unboxExpression(unboxValueList.get(0));
+                unboxValueList.set(0, unboxArg0);
+            }
+            return unboxValueList;
+        }
+
+        /** */
+        private Expression unboxExpression(final Expression argValue) {
+            Primitive fromBox = Primitive.ofBox(argValue.getType());
+            if (fromBox == null || fromBox == Primitive.VOID)
+                return argValue;
+
+            // Optimization: for "long x";
+            // "Long.valueOf(x)" generates "x"
+            if (argValue instanceof MethodCallExpression) {
+                MethodCallExpression mce = (MethodCallExpression)argValue;
+                if ("valueOf".equals(mce.method.getName()) && mce.expressions.size() == 1) {
+                    Expression originArg = mce.expressions.get(0);
+                    if (Primitive.of(originArg.type) == fromBox)
+                        return originArg;
+                }
+            }
+            return NullAs.NOT_POSSIBLE.handle(argValue);
+        }
+
+        /** */
+        abstract Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList);
+    }
+
+    /**
+     * Implementor for the {@code AND} operator.
+     *
+     * <p>If any of the arguments are false, result is false;
+     * else if any arguments are null, result is null; else true.
+     */
+    private static class LogicalAndImplementor extends AbstractRexCallImplementor {
+        /** */
+        LogicalAndImplementor() {
+            super(NullPolicy.NONE, true);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "logical_and";
+        }
+
+        /** {@inheritDoc} */
+        @Override public RexToLixTranslator.Result implement(final RexToLixTranslator translator,
+            final RexCall call, final List<RexToLixTranslator.Result> arguments) {
+            final List<Expression> argIsNullList = new ArrayList<>();
+            for (RexToLixTranslator.Result result : arguments)
+                argIsNullList.add(result.isNullVariable);
+
+            final List<Expression> nullAsTrue =
+                arguments.stream()
+                    .map(result ->
+                        Expressions.condition(result.isNullVariable, TRUE_EXPR,
+                            result.valueVariable))
+                    .collect(Collectors.toList());
+            final Expression hasFalse =
+                Expressions.not(Expressions.foldAnd(nullAsTrue));
+            final Expression hasNull = Expressions.foldOr(argIsNullList);
+            final Expression callExpression =
+                Expressions.condition(hasFalse, BOXED_FALSE_EXPR,
+                    Expressions.condition(hasNull, NULL_EXPR, BOXED_TRUE_EXPR));
+            final RexImpTable.NullAs nullAs = translator.isNullable(call)
+                ? RexImpTable.NullAs.NULL : RexImpTable.NullAs.NOT_POSSIBLE;
+            final Expression valueExpression = nullAs.handle(callExpression);
+            final ParameterExpression valueVariable =
+                Expressions.parameter(valueExpression.getType(),
+                    translator.getBlockBuilder().newName(getVariableName() + "_value"));
+            final Expression isNullExpression = translator.checkNull(valueVariable);
+            final ParameterExpression isNullVariable =
+                Expressions.parameter(Boolean.TYPE,
+                    translator.getBlockBuilder().newName(getVariableName() + "_isNull"));
+            translator.getBlockBuilder().add(
+                Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+            translator.getBlockBuilder().add(
+                Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+            return new RexToLixTranslator.Result(isNullVariable, valueVariable);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return null;
+        }
+    }
+
+    /**
+     * Implementor for the {@code OR} operator.
+     *
+     * <p>If any of the arguments are true, result is true;
+     * else if any arguments are null, result is null; else false.
+     */
+    private static class LogicalOrImplementor extends AbstractRexCallImplementor {
+        /** */
+        LogicalOrImplementor() {
+            super(NullPolicy.NONE, true);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "logical_or";
+        }
+
+        /** {@inheritDoc} */
+        @Override public RexToLixTranslator.Result implement(final RexToLixTranslator translator,
+            final RexCall call, final List<RexToLixTranslator.Result> arguments) {
+            final List<Expression> argIsNullList = new ArrayList<>();
+            for (RexToLixTranslator.Result result : arguments)
+                argIsNullList.add(result.isNullVariable);
+
+            final List<Expression> nullAsFalse =
+                arguments.stream()
+                    .map(result ->
+                        Expressions.condition(result.isNullVariable, FALSE_EXPR,
+                            result.valueVariable))
+                    .collect(Collectors.toList());
+            final Expression hasTrue = Expressions.foldOr(nullAsFalse);
+            final Expression hasNull = Expressions.foldOr(argIsNullList);
+            final Expression callExpression =
+                Expressions.condition(hasTrue, BOXED_TRUE_EXPR,
+                    Expressions.condition(hasNull, NULL_EXPR, BOXED_FALSE_EXPR));
+            final RexImpTable.NullAs nullAs = translator.isNullable(call)
+                ? RexImpTable.NullAs.NULL : RexImpTable.NullAs.NOT_POSSIBLE;
+            final Expression valueExpression = nullAs.handle(callExpression);
+            final ParameterExpression valueVariable =
+                Expressions.parameter(valueExpression.getType(),
+                    translator.getBlockBuilder().newName(getVariableName() + "_value"));
+            final Expression isNullExpression = translator.checkNull(valueExpression);
+            final ParameterExpression isNullVariable =
+                Expressions.parameter(Boolean.TYPE,
+                    translator.getBlockBuilder().newName(getVariableName() + "_isNull"));
+            translator.getBlockBuilder().add(
+                Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+            translator.getBlockBuilder().add(
+                Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+            return new RexToLixTranslator.Result(isNullVariable, valueVariable);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return null;
+        }
+    }
+
+    /**
+     * Implementor for the {@code NOT} operator.
+     *
+     * <p>If any of the arguments are false, result is true;
+     * else if any arguments are null, result is null; else false.
+     */
+    private static class LogicalNotImplementor extends AbstractRexCallImplementor {
+        /** */
+        LogicalNotImplementor() {
+            super(NullPolicy.NONE, true);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "logical_not";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.call(BuiltInMethod.NOT.method, argValueList);
+        }
+    }
+
+    /**
+     * Implementation that calls a given {@link Method}.
+     *
+     * <p>When method is not static, a new instance of the required class is
+     * created.
+     */
+    private static class ReflectiveImplementor extends AbstractRexCallImplementor {
+        /** */
+        protected final Method method;
+
+        /** */
+        ReflectiveImplementor(Method method, NullPolicy nullPolicy) {
+            super(nullPolicy, false);
+            this.method = method;
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "reflective_" + method.getName();
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList) {
+            List<Expression> argValueList0 = ConverterUtils.fromInternal(method.getParameterTypes(), argValueList);
+            if ((method.getModifiers() & Modifier.STATIC) != 0)
+                return Expressions.call(method, argValueList0);
+
+            // The UDF class must have a public zero-args constructor.
+            // Assume that the validator checked already.
+            final Expression target = Expressions.new_(method.getDeclaringClass());
+            return Expressions.call(target, method, argValueList0);
+        }
+    }
+
+    /** Implementor for the {@code RAND} function. */
+    private static class RandImplementor extends AbstractRexCallImplementor {
+        /** */
+        private final AbstractRexCallImplementor[] implementors = {
+            new ReflectiveImplementor(BuiltInMethod.RAND.method, nullPolicy),
+            new ReflectiveImplementor(BuiltInMethod.RAND_SEED.method, nullPolicy)
+        };
+
+        /** */
+        RandImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "rand";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return implementors[call.getOperands().size()]
+                .implementSafe(translator, call, argValueList);
+        }
+    }
+
+    /** Implementor for the {@code RAND_INTEGER} function. */
+    private static class RandIntegerImplementor extends AbstractRexCallImplementor {
+        /** */
+        private final AbstractRexCallImplementor[] implementors = {
+            null,
+            new ReflectiveImplementor(BuiltInMethod.RAND_INTEGER.method, nullPolicy),
+            new ReflectiveImplementor(BuiltInMethod.RAND_INTEGER_SEED.method, nullPolicy)
+        };
+
+        /** */
+        RandIntegerImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "rand_integer";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return implementors[call.getOperands().size()]
+                .implementSafe(translator, call, argValueList);
+        }
+    }
+
+    /** Implementor for the {@code PI} operator. */
+    private static class PiImplementor extends AbstractRexCallImplementor {
+        /** */
+        PiImplementor() {
+            super(NullPolicy.NONE, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "pi";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.constant(Math.PI);
+        }
+    }
+
+    /** Implementor for the {@code IS FALSE} SQL operator. */
+    private static class IsFalseImplementor extends AbstractRexCallImplementor {
+        /** */
+        IsFalseImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "is_false";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression getIfTrue(Type type) {
+            return Expressions.constant(false, type);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.equal(argValueList.get(0), FALSE_EXPR);
+        }
+    }
+
+    /** Implementor for the {@code IS NOT FALSE} SQL operator. */
+    private static class IsNotFalseImplementor extends AbstractRexCallImplementor {
+        /** */
+        IsNotFalseImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "is_not_false";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression getIfTrue(Type type) {
+            return Expressions.constant(true, type);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.notEqual(argValueList.get(0), FALSE_EXPR);
+        }
+    }
+
+    /** Implementor for the {@code IS NOT NULL} SQL operator. */
+    private static class IsNotNullImplementor extends AbstractRexCallImplementor {
+        /** */
+        IsNotNullImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "is_not_null";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression getIfTrue(Type type) {
+            return Expressions.constant(false, type);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.notEqual(argValueList.get(0), NULL_EXPR);
+        }
+    }
+
+    /** Implementor for the {@code IS NOT TRUE} SQL operator. */
+    private static class IsNotTrueImplementor extends AbstractRexCallImplementor {
+        /** */
+        IsNotTrueImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "is_not_true";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression getIfTrue(Type type) {
+            return Expressions.constant(true, type);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.notEqual(argValueList.get(0), TRUE_EXPR);
+        }
+    }
+
+    /** Implementor for the {@code IS NULL} SQL operator. */
+    private static class IsNullImplementor extends AbstractRexCallImplementor {
+        /** */
+        IsNullImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "is_null";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression getIfTrue(Type type) {
+            return Expressions.constant(true, type);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.equal(argValueList.get(0), NULL_EXPR);
+        }
+    }
+
+    /** Implementor for the {@code IS TRUE} SQL operator. */
+    private static class IsTrueImplementor extends AbstractRexCallImplementor {
+        /** */
+        IsTrueImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "is_true";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression getIfTrue(Type type) {
+            return Expressions.constant(false, type);
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.equal(argValueList.get(0), TRUE_EXPR);
+        }
+    }
+
+    /** Implementor for the {@code REGEXP_REPLACE} function. */
+    private static class RegexpReplaceImplementor extends AbstractRexCallImplementor {
+        private final AbstractRexCallImplementor[] implementors = {
+            new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE3.method, nullPolicy),
+            new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE4.method, nullPolicy),
+            new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE5.method, nullPolicy),
+            new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE6.method, nullPolicy),
+        };
+
+        /** */
+        RegexpReplaceImplementor() {
+            super(NullPolicy.STRICT, false);
+        }
+
+        @Override String getVariableName() {
+            return "regexp_replace";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(RexToLixTranslator translator,
+            RexCall call, List<Expression> argValueList) {
+            return implementors[call.getOperands().size() - 3]
+                .implementSafe(translator, call, argValueList);
+        }
+    }
+
+    /** Implementor for the {@code DEFAULT} function. */
+    private static class DefaultImplementor extends AbstractRexCallImplementor {
+        /** */
+        DefaultImplementor() {
+            super(NullPolicy.NONE, false);
+        }
+
+        /** {@inheritDoc} */
+        @Override String getVariableName() {
+            return "default";
+        }
+
+        /** {@inheritDoc} */
+        @Override Expression implementSafe(final RexToLixTranslator translator,
+            final RexCall call, final List<Expression> argValueList) {
+            return Expressions.constant(null);
+        }
+    }
+
+    /** */
+    private static CallImplementor createTableFunctionImplementor(final Method method) {
+        return createImplementor(
+            new ReflectiveCallNotNullImplementor(method) {
+                @Override public Expression implement(RexToLixTranslator translator,
+                    RexCall call, List<Expression> translatedOperands) {
+                    Expression expr = super.implement(translator, call,
+                        translatedOperands);
+                    final Class<?> returnType = method.getReturnType();
+                    if (QueryableTable.class.isAssignableFrom(returnType)) {
+                        Expression queryable = Expressions.call(
+                            Expressions.convert_(expr, QueryableTable.class),
+                            BuiltInMethod.QUERYABLE_TABLE_AS_QUERYABLE.method,
+                            Expressions.call(translator.getRoot(),
+                                BuiltInMethod.DATA_CONTEXT_GET_QUERY_PROVIDER.method),
+                            Expressions.constant(null, SchemaPlus.class),
+                            Expressions.constant(call.getOperator().getName(), String.class));
+                        expr = Expressions.call(queryable,
+                            BuiltInMethod.QUERYABLE_AS_ENUMERABLE.method);
+                    } else {
+                        expr = Expressions.call(expr,
+                            BuiltInMethod.SCANNABLE_TABLE_SCAN.method,
+                            translator.getRoot());
+                    }
+                    return expr;
+                }
+            }, NullPolicy.NONE, false);
+    }
+
+    /** */
+    public static CallImplementor createImplementor(
+        final NotNullImplementor implementor,
+        final NullPolicy nullPolicy,
+        final boolean harmonize) {
+        return (translator, call, nullAs) -> {
+            final RexImpTable.RexCallImplementor rexCallImplementor
+                = createRexCallImplementor(implementor, nullPolicy, harmonize);
+            final List<RexToLixTranslator.Result> arguments = translator.getCallOperandResult(call);
+            assert arguments != null;
+            final RexToLixTranslator.Result result = rexCallImplementor.implement(translator, call, arguments);
+            return nullAs.handle(result.valueVariable);
+        };
+    }
+
+    /** */
+    private static RexCallImplementor createRexCallImplementor(
+        final NotNullImplementor implementor,
+        final NullPolicy nullPolicy,
+        final boolean harmonize) {
+        return new AbstractRexCallImplementor(nullPolicy, harmonize) {
+            @Override String getVariableName() {
+                return "not_null_udf";
+            }
+
+            @Override Expression implementSafe(RexToLixTranslator translator,
+                RexCall call, List<Expression> argValueList) {
+                return implementor.implement(translator, call, argValueList);
+            }
+        };
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToLixTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToLixTranslator.java
new file mode 100644
index 0000000..b980735
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToLixTranslator.java
@@ -0,0 +1,1312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.CatchBlock;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Statement;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVariable;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.Pair;
+
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CASE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SEARCH;
+
+/**
+ * Translates {@link RexNode REX expressions} to {@link Expression linq4j expressions}.
+ */
+public class RexToLixTranslator implements RexVisitor<RexToLixTranslator.Result> {
+    /** */
+    final JavaTypeFactory typeFactory;
+
+    /** */
+    final RexBuilder builder;
+
+    /** */
+    private final RexProgram program;
+
+    /** */
+    final SqlConformance conformance;
+
+    /** */
+    private final Expression root;
+
+    /** */
+    final RexToLixTranslator.InputGetter inputGetter;
+
+    /** */
+    private final BlockBuilder list;
+
+    /** */
+    private final Function1<String, InputGetter> correlates;
+
+    /**
+     * Map from RexLiteral's variable name to its literal, which is often a ({@link ConstantExpression})) It is used in
+     * the some {@code RexCall}'s implementors, such as {@code ExtractImplementor}.
+     *
+     * @see #getLiteral
+     * @see #getLiteralValue
+     */
+    private final Map<Expression, Expression> literalMap = new HashMap<>();
+
+    /**
+     * For {@code RexCall}, keep the list of its operand's {@code Result}. It is useful when creating a {@code
+     * CallImplementor}.
+     */
+    private final Map<RexCall, List<Result>> callOperandResultMap = new HashMap<>();
+
+    /**
+     * Map from RexNode under specific storage type to its Result, to avoid generating duplicate code. For {@code
+     * RexInputRef}, {@code RexDynamicParam} and {@code RexFieldAccess}.
+     */
+    private final Map<Pair<RexNode, Type>, Result> rexWithStorageTypeResultMap = new HashMap<>();
+
+    /**
+     * Map from RexNode to its Result, to avoid generating duplicate code. For {@code RexLiteral} and {@code RexCall}.
+     */
+    private final Map<RexNode, Result> rexResultMap = new HashMap<>();
+
+    /** */
+    private Type currentStorageType;
+
+    /** */
+    private RexToLixTranslator(RexProgram program,
+        JavaTypeFactory typeFactory,
+        Expression root,
+        InputGetter inputGetter,
+        BlockBuilder list,
+        RexBuilder builder,
+        SqlConformance conformance,
+        Function1<String, InputGetter> correlates) {
+        this.program = program; // may be null
+        this.typeFactory = Objects.requireNonNull(typeFactory);
+        this.conformance = Objects.requireNonNull(conformance);
+        this.root = Objects.requireNonNull(root);
+        this.inputGetter = inputGetter;
+        this.list = Objects.requireNonNull(list);
+        this.builder = Objects.requireNonNull(builder);
+        this.correlates = correlates; // may be null
+    }
+
+    /**
+     * Translates a {@link RexProgram} to a sequence of expressions and declarations.
+     *
+     * @param program Program to be translated
+     * @param typeFactory Type factory
+     * @param conformance SQL conformance
+     * @param list List of statements, populated with declarations
+     * @param outputPhysType Output type, or null
+     * @param root Root expression
+     * @param inputGetter Generates expressions for inputs
+     * @param correlates Provider of references to the values of correlated variables
+     * @return Sequence of expressions, optional condition
+     */
+    public static List<Expression> translateProjects(RexProgram program,
+        JavaTypeFactory typeFactory, SqlConformance conformance,
+        BlockBuilder list, PhysType outputPhysType, Expression root,
+        InputGetter inputGetter, Function1<String, InputGetter> correlates) {
+        List<Type> storageTypes = null;
+        if (outputPhysType != null) {
+            final RelDataType rowType = outputPhysType.getRowType();
+            storageTypes = new ArrayList<>(rowType.getFieldCount());
+            for (int i = 0; i < rowType.getFieldCount(); i++)
+                storageTypes.add(outputPhysType.getJavaFieldType(i));
+        }
+        return new RexToLixTranslator(program, typeFactory, root, inputGetter,
+            list, new RexBuilder(typeFactory), conformance, null)
+            .setCorrelates(correlates)
+            .translateList(program.getProjectList(), storageTypes);
+    }
+
+    /** */
+    Expression translate(RexNode expr) {
+        final RexImpTable.NullAs nullAs =
+            RexImpTable.NullAs.of(isNullable(expr));
+        return translate(expr, nullAs);
+    }
+
+    /** */
+    Expression translate(RexNode expr, RexImpTable.NullAs nullAs) {
+        return translate(expr, nullAs, null);
+    }
+
+    /** */
+    Expression translate(RexNode expr, Type storageType) {
+        final RexImpTable.NullAs nullAs =
+            RexImpTable.NullAs.of(isNullable(expr));
+        return translate(expr, nullAs, storageType);
+    }
+
+    /** */
+    Expression translate(RexNode expr, RexImpTable.NullAs nullAs,
+        Type storageType) {
+        currentStorageType = storageType;
+        final Result result = expr.accept(this);
+        final Expression translated =
+            ConverterUtils.toInternal(result.valueVariable, storageType);
+        assert translated != null;
+        // When we asked for not null input that would be stored as box, avoid unboxing
+        if (RexImpTable.NullAs.NOT_POSSIBLE == nullAs
+            && translated.type.equals(storageType))
+            return translated;
+
+        return nullAs.handle(translated);
+    }
+
+    /** */
+    Expression translateCast(
+        RelDataType sourceType,
+        RelDataType targetType,
+        Expression operand) {
+        Expression convert = null;
+        switch (targetType.getSqlTypeName()) {
+            case ANY:
+                convert = operand;
+                break;
+            case DATE:
+                switch (sourceType.getSqlTypeName()) {
+                    case CHAR:
+                    case VARCHAR:
+                        convert =
+                            Expressions.call(BuiltInMethod.STRING_TO_DATE.method, operand);
+                        break;
+                    case TIMESTAMP:
+                        convert = Expressions.convert_(
+                            Expressions.call(BuiltInMethod.FLOOR_DIV.method,
+                                operand, Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                            int.class);
+                        break;
+                    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_DATE.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                }
+                break;
+            case TIME:
+                switch (sourceType.getSqlTypeName()) {
+                    case CHAR:
+                    case VARCHAR:
+                        convert =
+                            Expressions.call(BuiltInMethod.STRING_TO_TIME.method, operand);
+                        break;
+                    case TIME_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_TIME.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                        break;
+                    case TIMESTAMP:
+                        convert = Expressions.convert_(
+                            Expressions.call(
+                                BuiltInMethod.FLOOR_MOD.method,
+                                operand,
+                                Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                            int.class);
+                        break;
+                    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIME.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                }
+                break;
+            case TIME_WITH_LOCAL_TIME_ZONE:
+                switch (sourceType.getSqlTypeName()) {
+                    case CHAR:
+                    case VARCHAR:
+                        convert =
+                            Expressions.call(BuiltInMethod.STRING_TO_TIME_WITH_LOCAL_TIME_ZONE.method, operand);
+                        break;
+                    case TIME:
+                        convert = Expressions.call(
+                            BuiltInMethod.TIME_STRING_TO_TIME_WITH_LOCAL_TIME_ZONE.method,
+                            RexImpTable.optimize2(
+                                operand,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_TIME_TO_STRING.method,
+                                    operand)),
+                            Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+                        break;
+                    case TIMESTAMP:
+                        convert = Expressions.call(
+                            BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+                            RexImpTable.optimize2(
+                                operand,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+                                    operand)),
+                            Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+                        break;
+                    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIME_WITH_LOCAL_TIME_ZONE.method,
+                                operand));
+                }
+                break;
+            case TIMESTAMP:
+                switch (sourceType.getSqlTypeName()) {
+                    case CHAR:
+                    case VARCHAR:
+                        convert =
+                            Expressions.call(BuiltInMethod.STRING_TO_TIMESTAMP.method, operand);
+                        break;
+                    case DATE:
+                        convert = Expressions.multiply(
+                            Expressions.convert_(operand, long.class),
+                            Expressions.constant(DateTimeUtils.MILLIS_PER_DAY));
+                        break;
+                    case TIME:
+                        convert =
+                            Expressions.add(
+                                Expressions.multiply(
+                                    Expressions.convert_(
+                                        Expressions.call(BuiltInMethod.CURRENT_DATE.method, root),
+                                        long.class),
+                                    Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                                Expressions.convert_(operand, long.class));
+                        break;
+                    case TIME_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_DATE_TO_STRING.method,
+                                    Expressions.call(BuiltInMethod.CURRENT_DATE.method, root)),
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                        break;
+                    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                }
+                break;
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                switch (sourceType.getSqlTypeName()) {
+                    case CHAR:
+                    case VARCHAR:
+                        convert =
+                            Expressions.call(
+                                BuiltInMethod.STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+                                operand);
+                        break;
+                    case DATE:
+                        convert = Expressions.call(
+                            BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+                            RexImpTable.optimize2(
+                                operand,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+                                    Expressions.multiply(
+                                        Expressions.convert_(operand, long.class),
+                                        Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)))),
+                            Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+                        break;
+                    case TIME:
+                        convert = Expressions.call(
+                            BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+                            RexImpTable.optimize2(
+                                operand,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+                                    Expressions.add(
+                                        Expressions.multiply(
+                                            Expressions.convert_(
+                                                Expressions.call(BuiltInMethod.CURRENT_DATE.method, root),
+                                                long.class),
+                                            Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                                        Expressions.convert_(operand, long.class)))),
+                            Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+                        break;
+                    case TIME_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_DATE_TO_STRING.method,
+                                    Expressions.call(BuiltInMethod.CURRENT_DATE.method, root)),
+                                operand));
+                        break;
+                    case TIMESTAMP:
+                        convert = Expressions.call(
+                            BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+                            RexImpTable.optimize2(
+                                operand,
+                                Expressions.call(
+                                    BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+                                    operand)),
+                            Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+                }
+                break;
+            case BOOLEAN:
+                switch (sourceType.getSqlTypeName()) {
+                    case CHAR:
+                    case VARCHAR:
+                        convert = Expressions.call(
+                            BuiltInMethod.STRING_TO_BOOLEAN.method,
+                            operand);
+                }
+                break;
+            case CHAR:
+            case VARCHAR:
+                final SqlIntervalQualifier interval =
+                    sourceType.getIntervalQualifier();
+                switch (sourceType.getSqlTypeName()) {
+                    case DATE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.UNIX_DATE_TO_STRING.method,
+                                operand));
+                        break;
+                    case TIME:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.UNIX_TIME_TO_STRING.method,
+                                operand));
+                        break;
+                    case TIME_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_STRING.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                        break;
+                    case TIMESTAMP:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+                                operand));
+                        break;
+                    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_STRING.method,
+                                operand,
+                                Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+                        break;
+                    case INTERVAL_YEAR:
+                    case INTERVAL_YEAR_MONTH:
+                    case INTERVAL_MONTH:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.INTERVAL_YEAR_MONTH_TO_STRING.method,
+                                operand,
+                                Expressions.constant(interval.timeUnitRange)));
+                        break;
+                    case INTERVAL_DAY:
+                    case INTERVAL_DAY_HOUR:
+                    case INTERVAL_DAY_MINUTE:
+                    case INTERVAL_DAY_SECOND:
+                    case INTERVAL_HOUR:
+                    case INTERVAL_HOUR_MINUTE:
+                    case INTERVAL_HOUR_SECOND:
+                    case INTERVAL_MINUTE:
+                    case INTERVAL_MINUTE_SECOND:
+                    case INTERVAL_SECOND:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.INTERVAL_DAY_TIME_TO_STRING.method,
+                                operand,
+                                Expressions.constant(interval.timeUnitRange),
+                                Expressions.constant(
+                                    interval.getFractionalSecondPrecision(
+                                        typeFactory.getTypeSystem()))));
+                        break;
+                    case BOOLEAN:
+                        convert = RexImpTable.optimize2(
+                            operand,
+                            Expressions.call(
+                                BuiltInMethod.BOOLEAN_TO_STRING.method,
+                                operand));
+                        break;
+                }
+        }
+        if (convert == null)
+            convert = ConverterUtils.convert(operand, typeFactory.getJavaClass(targetType));
+
+        // Going from anything to CHAR(n) or VARCHAR(n), make sure value is no
+        // longer than n.
+        boolean pad = false;
+        boolean truncate = true;
+        switch (targetType.getSqlTypeName()) {
+            case CHAR:
+            case BINARY:
+                pad = true;
+                // fall through
+            case VARCHAR:
+            case VARBINARY:
+                final int targetPrecision = targetType.getPrecision();
+                if (targetPrecision >= 0) {
+                    switch (sourceType.getSqlTypeName()) {
+                        case CHAR:
+                        case VARCHAR:
+                        case BINARY:
+                        case VARBINARY:
+                            // If this is a widening cast, no need to truncate.
+                            final int sourcePrecision = sourceType.getPrecision();
+                            if (SqlTypeUtil.comparePrecision(sourcePrecision, targetPrecision) <= 0)
+                                truncate = false;
+
+                            // If this is a widening cast, no need to pad.
+                            if (SqlTypeUtil.comparePrecision(sourcePrecision, targetPrecision) >= 0)
+                                pad = false;
+
+                            // fall through
+                        default:
+                            if (truncate || pad) {
+                                convert =
+                                    Expressions.call(
+                                        pad
+                                            ? BuiltInMethod.TRUNCATE_OR_PAD.method
+                                            : BuiltInMethod.TRUNCATE.method,
+                                        convert,
+                                        Expressions.constant(targetPrecision));
+                            }
+                    }
+                }
+                break;
+            case TIMESTAMP:
+                int targetScale = targetType.getScale();
+                if (targetScale == RelDataType.SCALE_NOT_SPECIFIED)
+                    targetScale = 0;
+
+                if (targetScale < sourceType.getScale()) {
+                    convert =
+                        Expressions.call(
+                            BuiltInMethod.ROUND_LONG.method,
+                            convert,
+                            Expressions.constant(
+                                (long)Math.pow(10, 3 - targetScale)));
+                }
+                break;
+            case INTERVAL_YEAR:
+            case INTERVAL_YEAR_MONTH:
+            case INTERVAL_MONTH:
+            case INTERVAL_DAY:
+            case INTERVAL_DAY_HOUR:
+            case INTERVAL_DAY_MINUTE:
+            case INTERVAL_DAY_SECOND:
+            case INTERVAL_HOUR:
+            case INTERVAL_HOUR_MINUTE:
+            case INTERVAL_HOUR_SECOND:
+            case INTERVAL_MINUTE:
+            case INTERVAL_MINUTE_SECOND:
+            case INTERVAL_SECOND:
+                switch (sourceType.getSqlTypeName().getFamily()) {
+                    case NUMERIC:
+                        final BigDecimal multiplier = targetType.getSqlTypeName().getEndUnit().multiplier;
+                        final BigDecimal divider = BigDecimal.ONE;
+                        convert = RexImpTable.multiplyDivide(convert, multiplier, divider);
+                }
+        }
+        return scaleIntervalToNumber(sourceType, targetType, convert);
+    }
+
+    /**
+     * Dereferences an expression if it is a {@link RexLocalRef}.
+     */
+    public RexNode deref(RexNode expr) {
+        if (expr instanceof RexLocalRef) {
+            RexLocalRef ref = (RexLocalRef)expr;
+            final RexNode e2 = program.getExprList().get(ref.getIndex());
+            assert ref.getType().equals(e2.getType());
+            return e2;
+        }
+
+        return expr;
+    }
+
+    /**
+     * Translates a literal.
+     *
+     * @throws ControlFlowException if literal is null but {@code nullAs} is {@link RexImpTable.NullAs#NOT_POSSIBLE}.
+     */
+    public static Expression translateLiteral(
+        RexLiteral literal,
+        RelDataType type,
+        JavaTypeFactory typeFactory,
+        RexImpTable.NullAs nullAs) {
+        if (literal.isNull()) {
+            switch (nullAs) {
+                case TRUE:
+                case IS_NULL:
+                    return RexImpTable.TRUE_EXPR;
+                case FALSE:
+                case IS_NOT_NULL:
+                    return RexImpTable.FALSE_EXPR;
+                case NOT_POSSIBLE:
+                    throw new ControlFlowException();
+                case NULL:
+                default:
+                    return RexImpTable.NULL_EXPR;
+            }
+        }
+        else {
+            switch (nullAs) {
+                case IS_NOT_NULL:
+                    return RexImpTable.TRUE_EXPR;
+                case IS_NULL:
+                    return RexImpTable.FALSE_EXPR;
+            }
+        }
+        Type javaClass = typeFactory.getJavaClass(type);
+        final Object value2;
+        switch (literal.getType().getSqlTypeName()) {
+            case DECIMAL:
+                final BigDecimal bd = literal.getValueAs(BigDecimal.class);
+                if (javaClass == float.class)
+                    return Expressions.constant(bd, javaClass);
+                else if (javaClass == double.class)
+                    return Expressions.constant(bd, javaClass);
+                assert javaClass == BigDecimal.class;
+                return Expressions.new_(BigDecimal.class,
+                    Expressions.constant(bd.toString()));
+            case DATE:
+            case TIME:
+            case TIME_WITH_LOCAL_TIME_ZONE:
+            case INTERVAL_YEAR:
+            case INTERVAL_YEAR_MONTH:
+            case INTERVAL_MONTH:
+                value2 = literal.getValueAs(Integer.class);
+                javaClass = int.class;
+                break;
+            case TIMESTAMP:
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+            case INTERVAL_DAY:
+            case INTERVAL_DAY_HOUR:
+            case INTERVAL_DAY_MINUTE:
+            case INTERVAL_DAY_SECOND:
+            case INTERVAL_HOUR:
+            case INTERVAL_HOUR_MINUTE:
+            case INTERVAL_HOUR_SECOND:
+            case INTERVAL_MINUTE:
+            case INTERVAL_MINUTE_SECOND:
+            case INTERVAL_SECOND:
+                value2 = literal.getValueAs(Long.class);
+                javaClass = long.class;
+                break;
+            case CHAR:
+            case VARCHAR:
+                value2 = literal.getValueAs(String.class);
+                break;
+            case BINARY:
+            case VARBINARY:
+                return Expressions.new_(
+                    ByteString.class,
+                    Expressions.constant(
+                        literal.getValueAs(byte[].class),
+                        byte[].class));
+            case GEOMETRY:
+                final Geometries.Geom geom = literal.getValueAs(Geometries.Geom.class);
+                final String wkt = GeoFunctions.ST_AsWKT(geom);
+                return Expressions.call(null, BuiltInMethod.ST_GEOM_FROM_TEXT.method,
+                    Expressions.constant(wkt));
+            case SYMBOL:
+                value2 = literal.getValueAs(Enum.class);
+                javaClass = value2.getClass();
+                break;
+            default:
+                final Primitive primitive = Primitive.ofBoxOr(javaClass);
+                final Comparable value = literal.getValueAs(Comparable.class);
+
+                value2 = primitive != null && value instanceof Number ? primitive.number((Number)value) : value;
+        }
+        return Expressions.constant(value2, javaClass);
+    }
+
+    /** */
+    public List<Expression> translateList(
+        List<RexNode> operandList,
+        RexImpTable.NullAs nullAs) {
+        return translateList(operandList, nullAs,
+            ConverterUtils.internalTypes(operandList));
+    }
+
+    /** */
+    public List<Expression> translateList(
+        List<RexNode> operandList,
+        RexImpTable.NullAs nullAs,
+        List<? extends Type> storageTypes) {
+        final List<Expression> list = new ArrayList<>();
+        for (Pair<RexNode, ? extends Type> e : Pair.zip(operandList, storageTypes))
+            list.add(translate(e.left, nullAs, e.right));
+
+        return list;
+    }
+
+    /**
+     * Translates the list of {@code RexNode}, using the default output types. This might be suboptimal in terms of
+     * additional box-unbox when you use the translation later. If you know the java class that will be used to store
+     * the results, use {@link RexToLixTranslator#translateList(List, List)} version.
+     *
+     * @param operandList list of RexNodes to translate
+     * @return translated expressions
+     */
+    public List<Expression> translateList(List<? extends RexNode> operandList) {
+        return translateList(operandList, ConverterUtils.internalTypes(operandList));
+    }
+
+    /**
+     * Translates the list of {@code RexNode}, while optimizing for output storage. For instance, if the result of
+     * translation is going to be stored in {@code Object[]}, and the input is {@code Object[]} as well, then translator
+     * will avoid casting, boxing, etc.
+     *
+     * @param operandList list of RexNodes to translate
+     * @param storageTypes hints of the java classes that will be used to store translation results. Use null to use
+     * default storage type
+     * @return translated expressions
+     */
+    public List<Expression> translateList(List<? extends RexNode> operandList,
+        List<? extends Type> storageTypes) {
+        final List<Expression> list = new ArrayList<>(operandList.size());
+
+        for (int i = 0; i < operandList.size(); i++) {
+            RexNode rex = operandList.get(i);
+            Type desiredType = null;
+            if (storageTypes != null)
+                desiredType = storageTypes.get(i);
+
+            final Expression translate = translate(rex, desiredType);
+            list.add(translate);
+            // desiredType is still a hint, thus we might get any kind of output
+            // (boxed or not) when hint was provided.
+            // It is favourable to get the type matching desired type
+            if (desiredType == null && !isNullable(rex)) {
+                assert !Primitive.isBox(translate.getType())
+                    : "Not-null boxed primitive should come back as primitive: "
+                    + rex + ", " + translate.getType();
+            }
+        }
+        return list;
+    }
+
+    /**
+     * Returns whether an expression is nullable.
+     *
+     * @param e Expression
+     * @return Whether expression is nullable
+     */
+    public boolean isNullable(RexNode e) {
+        return e.getType().isNullable();
+    }
+
+    /** */
+    public RexToLixTranslator setBlock(BlockBuilder block) {
+        if (block == list)
+            return this;
+
+        return new RexToLixTranslator(program, typeFactory, root, inputGetter,
+            block, builder, conformance, correlates);
+    }
+
+    /** */
+    public RexToLixTranslator setCorrelates(
+        Function1<String, InputGetter> correlates) {
+        if (this.correlates == correlates)
+            return this;
+
+        return new RexToLixTranslator(program, typeFactory, root, inputGetter, list,
+            builder, conformance, correlates);
+    }
+
+    /** */
+    public Expression getRoot() {
+        return root;
+    }
+
+    /** */
+    private static Expression scaleIntervalToNumber(
+        RelDataType sourceType,
+        RelDataType targetType,
+        Expression operand) {
+        switch (targetType.getSqlTypeName().getFamily()) {
+            case NUMERIC:
+                switch (sourceType.getSqlTypeName()) {
+                    case INTERVAL_YEAR:
+                    case INTERVAL_YEAR_MONTH:
+                    case INTERVAL_MONTH:
+                    case INTERVAL_DAY:
+                    case INTERVAL_DAY_HOUR:
+                    case INTERVAL_DAY_MINUTE:
+                    case INTERVAL_DAY_SECOND:
+                    case INTERVAL_HOUR:
+                    case INTERVAL_HOUR_MINUTE:
+                    case INTERVAL_HOUR_SECOND:
+                    case INTERVAL_MINUTE:
+                    case INTERVAL_MINUTE_SECOND:
+                    case INTERVAL_SECOND:
+                        // Scale to the given field.
+                        final BigDecimal multiplier = BigDecimal.ONE;
+                        final BigDecimal divider =
+                            sourceType.getSqlTypeName().getEndUnit().multiplier;
+                        return RexImpTable.multiplyDivide(operand, multiplier, divider);
+                }
+        }
+        return operand;
+    }
+
+    /**
+     * Visit {@code RexInputRef}. If it has never been visited under current storage type before, {@code
+     * RexToLixTranslator} generally produces three lines of code. For example, when visiting a column (named
+     * commission) in table Employee, the generated code snippet is: {@code final Employee current =(Employee)
+     * inputEnumerator.current(); final Integer input_value = current.commission; final boolean input_isNull =
+     * input_value == null; }
+     */
+    @Override public Result visitInputRef(RexInputRef inputRef) {
+        final Pair<RexNode, Type> key = Pair.of(inputRef, currentStorageType);
+        // If the RexInputRef has been visited under current storage type already,
+        // it is not necessary to visit it again, just return the result.
+        if (rexWithStorageTypeResultMap.containsKey(key))
+            return rexWithStorageTypeResultMap.get(key);
+
+        // Generate one line of code to get the input, e.g.,
+        // "final Employee current =(Employee) inputEnumerator.current();"
+        final Expression valueExpression = inputGetter.field(
+            list, inputRef.getIndex(), currentStorageType);
+
+        // Generate one line of code for the value of RexInputRef, e.g.,
+        // "final Integer input_value = current.commission;"
+        final ParameterExpression valueVariable =
+            Expressions.parameter(
+                valueExpression.getType(), list.newName("input_value"));
+        list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+
+        // Generate one line of code to check whether RexInputRef is null, e.g.,
+        // "final boolean input_isNull = input_value == null;"
+        final Expression isNullExpression = checkNull(valueVariable);
+        final ParameterExpression isNullVariable =
+            Expressions.parameter(
+                Boolean.TYPE, list.newName("input_isNull"));
+        list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+
+        final Result result = new Result(isNullVariable, valueVariable);
+
+        // Cache <RexInputRef, currentStorageType>'s result
+        rexWithStorageTypeResultMap.put(key, result);
+
+        return new Result(isNullVariable, valueVariable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitLocalRef(RexLocalRef localRef) {
+        return deref(localRef).accept(this);
+    }
+
+    /**
+     * Visit {@code RexLiteral}. If it has never been visited before, {@code RexToLixTranslator} will generate two lines
+     * of code. For example, when visiting a primitive int (10), the generated code snippet is: {@code final int
+     * literal_value = 10; final boolean literal_isNull = false; }
+     */
+    @Override public Result visitLiteral(RexLiteral literal) {
+        // If the RexLiteral has been visited already, just return the result
+        if (rexResultMap.containsKey(literal))
+            return rexResultMap.get(literal);
+
+        // Generate one line of code for the value of RexLiteral, e.g.,
+        // "final int literal_value = 10;"
+        final Expression valueExpression = literal.isNull()
+            // Note: even for null literal, we can't loss its type information
+            ? getTypedNullLiteral(literal)
+            : translateLiteral(literal, literal.getType(),
+            typeFactory, RexImpTable.NullAs.NOT_POSSIBLE);
+        final ParameterExpression valueVariable =
+            Expressions.parameter(valueExpression.getType(),
+                list.newName("literal_value"));
+        list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+
+        // Generate one line of code to check whether RexLiteral is null, e.g.,
+        // "final boolean literal_isNull = false;"
+        final Expression isNullExpression =
+            literal.isNull() ? RexImpTable.TRUE_EXPR : RexImpTable.FALSE_EXPR;
+        final ParameterExpression isNullVariable = Expressions.parameter(
+            Boolean.TYPE, list.newName("literal_isNull"));
+        list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+
+        // Maintain the map from valueVariable (ParameterExpression) to real Expression
+        literalMap.put(valueVariable, valueExpression);
+        final Result result = new Result(isNullVariable, valueVariable);
+        // Cache RexLiteral's result
+        rexResultMap.put(literal, result);
+        return result;
+    }
+
+    /**
+     * Returns an {@code Expression} for null literal without losing its type information.
+     */
+    private ConstantExpression getTypedNullLiteral(RexLiteral literal) {
+        assert literal.isNull();
+        Type javaClass = typeFactory.getJavaClass(literal.getType());
+        switch (literal.getType().getSqlTypeName()) {
+            case DATE:
+            case TIME:
+            case TIME_WITH_LOCAL_TIME_ZONE:
+            case INTERVAL_YEAR:
+            case INTERVAL_YEAR_MONTH:
+            case INTERVAL_MONTH:
+                javaClass = Integer.class;
+                break;
+            case TIMESTAMP:
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+            case INTERVAL_DAY:
+            case INTERVAL_DAY_HOUR:
+            case INTERVAL_DAY_MINUTE:
+            case INTERVAL_DAY_SECOND:
+            case INTERVAL_HOUR:
+            case INTERVAL_HOUR_MINUTE:
+            case INTERVAL_HOUR_SECOND:
+            case INTERVAL_MINUTE:
+            case INTERVAL_MINUTE_SECOND:
+            case INTERVAL_SECOND:
+                javaClass = Long.class;
+                break;
+        }
+        return javaClass == null || javaClass == Void.class
+            ? RexImpTable.NULL_EXPR
+            : Expressions.constant(null, javaClass);
+    }
+
+    /**
+     * Visit {@code RexCall}. For most {@code SqlOperator}s, we can get the implementor from {@code RexImpTable}.
+     * Several operators (e.g., CaseWhen) with special semantics need to be implemented separately.
+     */
+    @Override public Result visitCall(RexCall call) {
+        if (rexResultMap.containsKey(call))
+            return rexResultMap.get(call);
+
+        final SqlOperator operator = call.getOperator();
+        if (operator == CASE)
+            return implementCaseWhen(call);
+
+        if (operator == SEARCH)
+            return RexUtil.expandSearch(builder, program, call).accept(this);
+
+        final RexImpTable.RexCallImplementor implementor =
+            RexImpTable.INSTANCE.get(operator);
+        if (implementor == null)
+            throw new RuntimeException("cannot translate call " + call);
+
+        final List<RexNode> operandList = call.getOperands();
+        final List<Type> storageTypes = ConverterUtils.internalTypes(operandList);
+        final List<Result> operandResults = new ArrayList<>();
+        for (int i = 0; i < operandList.size(); i++) {
+            final Result operandResult =
+                implementCallOperand(operandList.get(i), storageTypes.get(i), this);
+            operandResults.add(operandResult);
+        }
+        callOperandResultMap.put(call, operandResults);
+        final Result result = implementor.implement(this, call, operandResults);
+        rexResultMap.put(call, result);
+        return result;
+    }
+
+    /** */
+    private static Result implementCallOperand(final RexNode operand,
+        final Type storageType, final RexToLixTranslator translator) {
+        final Type originalStorageType = translator.currentStorageType;
+        translator.currentStorageType = storageType;
+        Result operandResult = operand.accept(translator);
+        if (storageType != null)
+          operandResult = translator.toInnerStorageType(operandResult, storageType);
+        translator.currentStorageType = originalStorageType;
+        return operandResult;
+    }
+
+    /** */
+    private static Expression implementCallOperand2(final RexNode operand,
+        final Type storageType, final RexToLixTranslator translator) {
+        final Type originalStorageType = translator.currentStorageType;
+        translator.currentStorageType = storageType;
+        final Expression result = translator.translate(operand);
+        translator.currentStorageType = originalStorageType;
+        return result;
+    }
+
+    /**
+     * The CASE operator is SQL’s way of handling if/then logic. Different with other {@code RexCall}s, it is not safe
+     * to implement its operands first. For example: {@code select case when s=0 then false else 100/s > 0 end from
+     * (values (1),(0)) ax(s); }
+     */
+    private Result implementCaseWhen(RexCall call) {
+        final Type returnType = typeFactory.getJavaClass(call.getType());
+        final ParameterExpression valueVariable =
+            Expressions.parameter(returnType,
+                list.newName("case_when_value"));
+        list.add(Expressions.declare(0, valueVariable, null));
+        final List<RexNode> operandList = call.getOperands();
+        implementRecursively(this, operandList, valueVariable, 0);
+        final Expression isNullExpression = checkNull(valueVariable);
+        final ParameterExpression isNullVariable =
+            Expressions.parameter(
+                Boolean.TYPE, list.newName("case_when_isNull"));
+        list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+        final Result result = new Result(isNullVariable, valueVariable);
+        rexResultMap.put(call, result);
+        return result;
+    }
+
+    /**
+     * Case statements of the form: {@code CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END}. When {@code a = true},
+     * returns {@code b}; when {@code c = true}, returns {@code d}; else returns {@code e}.
+     *
+     * <p>We generate code that looks like:
+     *
+     * <blockquote><pre>
+     *      int case_when_value;
+     *      ......code for a......
+     *      if (!a_isNull && a_value) {
+     *          ......code for b......
+     *          case_when_value = res(b_isNull, b_value);
+     *      } else {
+     *          ......code for c......
+     *          if (!c_isNull && c_value) {
+     *              ......code for d......
+     *              case_when_value = res(d_isNull, d_value);
+     *          } else {
+     *              ......code for e......
+     *              case_when_value = res(e_isNull, e_value);
+     *          }
+     *      }
+     * </pre></blockquote>
+     */
+    private void implementRecursively(final RexToLixTranslator currentTranslator,
+        final List<RexNode> operandList, final ParameterExpression valueVariable, int pos) {
+        final BlockBuilder currentBlockBuilder = currentTranslator.getBlockBuilder();
+        final List<Type> storageTypes = ConverterUtils.internalTypes(operandList);
+        // [ELSE] clause
+        if (pos == operandList.size() - 1) {
+            Expression res = implementCallOperand2(operandList.get(pos),
+                storageTypes.get(pos), currentTranslator);
+            currentBlockBuilder.add(
+                Expressions.statement(
+                    Expressions.assign(valueVariable,
+                        ConverterUtils.convert(res, valueVariable.getType()))));
+            return;
+        }
+        // Condition code: !a_isNull && a_value
+        final RexNode testerNode = operandList.get(pos);
+        final Result testerResult = implementCallOperand(testerNode,
+            storageTypes.get(pos), currentTranslator);
+        final Expression tester = Expressions.andAlso(
+            Expressions.not(testerResult.isNullVariable),
+            testerResult.valueVariable);
+        // Code for {if} branch
+        final RexNode ifTrueNode = operandList.get(pos + 1);
+        final BlockBuilder ifTrueBlockBuilder =
+            new BlockBuilder(true, currentBlockBuilder);
+        final RexToLixTranslator ifTrueTranslator =
+            currentTranslator.setBlock(ifTrueBlockBuilder);
+        final Expression ifTrueRes = implementCallOperand2(ifTrueNode,
+            storageTypes.get(pos + 1), ifTrueTranslator);
+        // Assign the value: case_when_value = ifTrueRes
+        ifTrueBlockBuilder.add(
+            Expressions.statement(
+                Expressions.assign(valueVariable,
+                    ConverterUtils.convert(ifTrueRes, valueVariable.getType()))));
+        final BlockStatement ifTrue = ifTrueBlockBuilder.toBlock();
+        // There is no [ELSE] clause
+        if (pos + 1 == operandList.size() - 1) {
+            currentBlockBuilder.add(
+                Expressions.ifThen(tester, ifTrue));
+            return;
+        }
+        // Generate code for {else} branch recursively
+        final BlockBuilder ifFalseBlockBuilder =
+            new BlockBuilder(true, currentBlockBuilder);
+        final RexToLixTranslator ifFalseTranslator =
+            currentTranslator.setBlock(ifFalseBlockBuilder);
+        implementRecursively(ifFalseTranslator, operandList, valueVariable, pos + 2);
+        final BlockStatement ifFalse = ifFalseBlockBuilder.toBlock();
+        currentBlockBuilder.add(
+            Expressions.ifThenElse(tester, ifTrue, ifFalse));
+    }
+
+    /** */
+    private Result toInnerStorageType(final Result result, final Type storageType) {
+        final Expression valueExpression =
+            ConverterUtils.toInternal(result.valueVariable, storageType);
+        if (valueExpression.equals(result.valueVariable))
+            return result;
+
+        final ParameterExpression valueVariable =
+            Expressions.parameter(
+                valueExpression.getType(),
+                list.newName(result.valueVariable.name + "_inner_type"));
+        list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+        final ParameterExpression isNullVariable = result.isNullVariable;
+        return new Result(isNullVariable, valueVariable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitDynamicParam(RexDynamicParam dynamicParam) {
+        final Pair<RexNode, Type> key = Pair.of(dynamicParam, currentStorageType);
+        if (rexWithStorageTypeResultMap.containsKey(key))
+            return rexWithStorageTypeResultMap.get(key);
+
+        final Type storageType = currentStorageType != null
+            ? currentStorageType : typeFactory.getJavaClass(dynamicParam.getType());
+        final Expression valueExpression = ConverterUtils.convert(
+            Expressions.call(root, BuiltInMethod.DATA_CONTEXT_GET.method,
+                Expressions.constant("?" + dynamicParam.getIndex())),
+            storageType);
+        final ParameterExpression valueVariable =
+            Expressions.parameter(valueExpression.getType(), list.newName("value_dynamic_param"));
+        list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+        final ParameterExpression isNullVariable =
+            Expressions.parameter(Boolean.TYPE, list.newName("isNull_dynamic_param"));
+        list.add(Expressions.declare(Modifier.FINAL, isNullVariable, checkNull(valueVariable)));
+        final Result result = new Result(isNullVariable, valueVariable);
+        rexWithStorageTypeResultMap.put(key, result);
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitFieldAccess(RexFieldAccess fieldAccess) {
+        final Pair<RexNode, Type> key = Pair.of(fieldAccess, currentStorageType);
+        if (rexWithStorageTypeResultMap.containsKey(key))
+            return rexWithStorageTypeResultMap.get(key);
+
+        final RexNode target = deref(fieldAccess.getReferenceExpr());
+        int fieldIndex = fieldAccess.getField().getIndex();
+        String fieldName = fieldAccess.getField().getName();
+        switch (target.getKind()) {
+            case CORREL_VARIABLE:
+                if (correlates == null) {
+                    throw new RuntimeException("Cannot translate " + fieldAccess
+                        + " since correlate variables resolver is not defined");
+                }
+                final RexToLixTranslator.InputGetter getter =
+                    correlates.apply(((RexVariable)target).getName());
+                final Expression input = getter.field(
+                    list, fieldIndex, currentStorageType);
+                final Expression condition = checkNull(input);
+                final ParameterExpression valueVariable =
+                    Expressions.parameter(input.getType(), list.newName("corInp_value"));
+                list.add(Expressions.declare(Modifier.FINAL, valueVariable, input));
+                final ParameterExpression isNullVariable =
+                    Expressions.parameter(Boolean.TYPE, list.newName("corInp_isNull"));
+                final Expression isNullExpression = Expressions.condition(
+                    condition,
+                    RexImpTable.TRUE_EXPR,
+                    checkNull(valueVariable));
+                list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+                final Result result1 = new Result(isNullVariable, valueVariable);
+                rexWithStorageTypeResultMap.put(key, result1);
+                return result1;
+            default:
+                RexNode rxIndex =
+                    builder.makeLiteral(fieldIndex, typeFactory.createType(int.class), true);
+                RexNode rxName =
+                    builder.makeLiteral(fieldName, typeFactory.createType(String.class), true);
+                RexCall accessCall = (RexCall)builder.makeCall(
+                    fieldAccess.getType(), SqlStdOperatorTable.STRUCT_ACCESS,
+                    ImmutableList.of(target, rxIndex, rxName));
+                final Result result2 = accessCall.accept(this);
+                rexWithStorageTypeResultMap.put(key, result2);
+                return result2;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitOver(RexOver over) {
+        throw new RuntimeException("cannot translate expression " + over);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitCorrelVariable(RexCorrelVariable correlVariable) {
+        throw new RuntimeException("Cannot translate " + correlVariable
+            + ". Correlated variables should always be referenced by field access");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitRangeRef(RexRangeRef rangeRef) {
+        throw new RuntimeException("cannot translate expression " + rangeRef);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitSubQuery(RexSubQuery subQuery) {
+        throw new RuntimeException("cannot translate expression " + subQuery);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitTableInputRef(RexTableInputRef fieldRef) {
+        throw new RuntimeException("cannot translate expression " + fieldRef);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Result visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+        return visitInputRef(fieldRef);
+    }
+
+    /** */
+    Expression checkNull(Expression expr) {
+        if (Primitive.flavor(expr.getType()) == Primitive.Flavor.PRIMITIVE)
+            return RexImpTable.FALSE_EXPR;
+
+        return Expressions.equal(expr, RexImpTable.NULL_EXPR);
+    }
+
+    /** */
+    Expression checkNotNull(Expression expr) {
+        if (Primitive.flavor(expr.getType()) == Primitive.Flavor.PRIMITIVE)
+            return RexImpTable.TRUE_EXPR;
+
+        return Expressions.notEqual(expr, RexImpTable.NULL_EXPR);
+    }
+
+    /** */
+    BlockBuilder getBlockBuilder() {
+        return list;
+    }
+
+    /** */
+    Expression getLiteral(Expression literalVariable) {
+        return literalMap.get(literalVariable);
+    }
+
+    /** Returns the value of a literal. */
+    Object getLiteralValue(Expression expr) {
+        if (expr instanceof ParameterExpression) {
+            final Expression constantExpr = literalMap.get(expr);
+            return getLiteralValue(constantExpr);
+        }
+        if (expr instanceof ConstantExpression)
+            return ((ConstantExpression)expr).value;
+
+        return null;
+    }
+
+    /** */
+    List<Result> getCallOperandResult(RexCall call) {
+        return callOperandResultMap.get(call);
+    }
+
+    /** Translates a field of an input to an expression. */
+    public interface InputGetter {
+        /** */
+        Expression field(BlockBuilder list, int index, Type storageType);
+    }
+
+    /** Result of translating a {@code RexNode}. */
+    public static class Result {
+        /** */
+        final ParameterExpression isNullVariable;
+
+        /** */
+        final ParameterExpression valueVariable;
+
+        /** */
+        public Result(ParameterExpression isNullVariable,
+            ParameterExpression valueVariable) {
+            this.isNullVariable = isNullVariable;
+            this.valueVariable = valueVariable;
+        }
+    }
+
+    /**
+     * Handle checked Exceptions declared in Method. In such case,
+     * method call should be wrapped in a try...catch block.
+     * "
+     *      final Type method_call;
+     *      try {
+     *        method_call = callExpr
+     *      } catch (Exception e) {
+     *        throw new RuntimeException(e);
+     *      }
+     * "
+     */
+    Expression handleMethodCheckedExceptions(Expression callExpr) {
+        // Try statement
+        ParameterExpression methodCall = Expressions.parameter(
+            callExpr.getType(), list.newName("method_call"));
+        list.add(Expressions.declare(Modifier.FINAL, methodCall, null));
+        Statement st = Expressions.statement(Expressions.assign(methodCall, callExpr));
+        // Catch Block, wrap checked exception in unchecked exception
+        ParameterExpression e = Expressions.parameter(0, Exception.class, "e");
+        Expression uncheckedException = Expressions.new_(RuntimeException.class, e);
+        CatchBlock cb = Expressions.catch_(e, Expressions.throw_(uncheckedException));
+        list.add(Expressions.tryCatch(st, cb));
+        return methodCall;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
similarity index 60%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
index ce7d926..9fc3d9e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
@@ -14,27 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
-
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
 
-    /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
 
+/** */
+public interface Scalar {
     /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+    void execute(ExecutionContext ctx, Object in, Object out);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
similarity index 61%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
index ce7d926..b4b9b91 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
@@ -14,27 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/** */
+public interface Accumulator extends Serializable {
+    /** */
+    void add(Object... args);
+
     /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+    void apply(Accumulator other);
 
     /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+    Object end();
 
     /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+    List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory);
 
     /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+    RelDataType returnType(IgniteTypeFactory typeFactory);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
index ce7d926..7d7702b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
@@ -14,27 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+/** */
+public interface AccumulatorWrapper<Row> {
     /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+    void add(Row row);
 
     /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+    Object end();
 
     /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+    void apply(Accumulator accumulator);
 
     /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+    Accumulator accumulator();
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
new file mode 100644
index 0000000..4c61423
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
@@ -0,0 +1,1068 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+import static org.apache.calcite.sql.type.SqlTypeName.ANY;
+import static org.apache.calcite.sql.type.SqlTypeName.BIGINT;
+import static org.apache.calcite.sql.type.SqlTypeName.DECIMAL;
+import static org.apache.calcite.sql.type.SqlTypeName.DOUBLE;
+import static org.apache.calcite.sql.type.SqlTypeName.INTEGER;
+import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR;
+import static org.apache.ignite.internal.util.ArrayUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class Accumulators {
+    /** */
+    public static Supplier<Accumulator> accumulatorFactory(AggregateCall call) {
+        if (!call.isDistinct())
+            return accumulatorFunctionFactory(call);
+
+        Supplier<Accumulator> fac = accumulatorFunctionFactory(call);
+
+        return () -> new DistinctAccumulator(fac);
+    }
+
+    /** */
+    public static Supplier<Accumulator> accumulatorFunctionFactory(AggregateCall call) {
+        switch (call.getAggregation().getName()) {
+            case "COUNT":
+                return LongCount.FACTORY;
+            case "AVG":
+                return avgFactory(call);
+            case "SUM":
+                return sumFactory(call);
+            case "$SUM0":
+                return sumEmptyIsZeroFactory(call);
+            case "MIN":
+                return minFactory(call);
+            case "MAX":
+                return maxFactory(call);
+            case "SINGLE_VALUE":
+                return SingleVal.FACTORY;
+            default:
+                throw new AssertionError(call.getAggregation().getName());
+        }
+    }
+
+    /** */
+    private static Supplier<Accumulator> avgFactory(AggregateCall call) {
+        switch (call.type.getSqlTypeName()) {
+            case BIGINT:
+            case DECIMAL:
+                return DecimalAvg.FACTORY;
+            case DOUBLE:
+            case REAL:
+            case FLOAT:
+            case INTEGER:
+            default:
+                return DoubleAvg.FACTORY;
+        }
+    }
+
+    /** */
+    private static Supplier<Accumulator> sumFactory(AggregateCall call) {
+        switch (call.type.getSqlTypeName()) {
+            case DOUBLE:
+            case REAL:
+            case FLOAT:
+                return DoubleSum.FACTORY;
+            case DECIMAL:
+                return DecimalSum.FACTORY;
+            case INTEGER:
+                return IntSum.FACTORY;
+            case BIGINT:
+            default:
+                return LongSum.FACTORY;
+        }
+    }
+
+    /** */
+    private static Supplier<Accumulator> sumEmptyIsZeroFactory(AggregateCall call) {
+        switch (call.type.getSqlTypeName()) {
+            case DOUBLE:
+            case REAL:
+            case FLOAT:
+                return DoubleSumEmptyIsZero.FACTORY;
+            case DECIMAL:
+                return DecimalSumEmptyIsZero.FACTORY;
+            case INTEGER:
+                return IntSumEmptyIsZero.FACTORY;
+            case BIGINT:
+            default:
+                return LongSumEmptyIsZero.FACTORY;
+        }
+    }
+
+    /** */
+    private static Supplier<Accumulator> minFactory(AggregateCall call) {
+        switch (call.type.getSqlTypeName()) {
+            case DOUBLE:
+            case REAL:
+            case FLOAT:
+                return DoubleMinMax.MIN_FACTORY;
+            case DECIMAL:
+                return DecimalMinMax.MIN_FACTORY;
+            case INTEGER:
+                return IntMinMax.MIN_FACTORY;
+            case CHAR:
+            case VARCHAR:
+                return VarCharMinMax.MIN_FACTORY;
+            case BIGINT:
+            default:
+                return LongMinMax.MIN_FACTORY;
+        }
+    }
+
+    /** */
+    private static Supplier<Accumulator> maxFactory(AggregateCall call) {
+        switch (call.type.getSqlTypeName()) {
+            case DOUBLE:
+            case REAL:
+            case FLOAT:
+                return DoubleMinMax.MAX_FACTORY;
+            case DECIMAL:
+                return DecimalMinMax.MAX_FACTORY;
+            case INTEGER:
+                return IntMinMax.MAX_FACTORY;
+            case CHAR:
+            case VARCHAR:
+                return VarCharMinMax.MAX_FACTORY;
+            case BIGINT:
+            default:
+                return LongMinMax.MAX_FACTORY;
+        }
+    }
+
+    /** */
+    private static class SingleVal implements Accumulator {
+        /** */
+        private Object holder;
+
+        /** */
+        private boolean touched;
+
+        /** */
+        public static final Supplier<Accumulator> FACTORY = SingleVal::new;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            assert args.length == 1 : args.length;
+
+            if (touched)
+                throw new IllegalArgumentException("Subquery returned more than 1 value.");
+
+            touched = true;
+
+            holder = args[0];
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            assert holder == null : "sudden apply for: " + other + " on SingleVal";
+
+            holder = ((SingleVal)other).holder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return holder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(ANY), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createSqlType(ANY);
+        }
+    }
+
+    /** */
+    public static class DecimalAvg implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = DecimalAvg::new;
+        
+        /** */
+        private BigDecimal sum = BigDecimal.ZERO;
+
+        /** */
+        private BigDecimal cnt = BigDecimal.ZERO;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            BigDecimal in = (BigDecimal)args[0];
+
+            if (in == null)
+                return;
+
+            sum = sum.add(in);
+            cnt = cnt.add(BigDecimal.ONE);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DecimalAvg other0 = (DecimalAvg)other;
+
+            sum = sum.add(other0.sum);
+            cnt = cnt.add(other0.cnt);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return cnt.compareTo(BigDecimal.ZERO) == 0 ? null : sum.divide(cnt, MathContext.DECIMAL64);
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+        }
+    }
+
+    /** */
+    public static class DoubleAvg implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = DoubleAvg::new;
+
+        /** */
+        private double sum;
+
+        /** */
+        private long cnt;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Double in = (Double)args[0];
+
+            if (in == null)
+                return;
+
+            sum += in;
+            cnt++;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DoubleAvg other0 = (DoubleAvg)other;
+
+            sum += other0.sum;
+            cnt += other0.cnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return cnt > 0 ? sum / cnt : null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+        }
+    }
+
+    /** */
+    private static class LongCount implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = LongCount::new;
+
+        /** */
+        private long cnt;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            assert nullOrEmpty(args) || args.length == 1;
+
+            if (nullOrEmpty(args) || args[0] != null)
+                cnt++;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            LongCount other0 = (LongCount)other;
+            cnt += other0.cnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return cnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(ANY), false));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createSqlType(BIGINT);
+        }
+    }
+
+    /** */
+    private static class DoubleSum implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = DoubleSum::new;
+
+        /** */
+        private double sum;
+
+        /** */
+        private boolean empty = true;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Double in = (Double)args[0];
+
+            if (in == null)
+                return;
+
+            empty = false;
+            sum += in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DoubleSum other0 = (DoubleSum)other;
+
+            if (other0.empty)
+                return;
+
+            empty = false;
+            sum += other0.sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+        }
+    }
+
+    /** */
+    private static class IntSum implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = IntSum::new;
+
+        /** */
+        private int sum;
+
+        /** */
+        private boolean empty = true;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Integer in = (Integer)args[0];
+
+            if (in == null)
+                return;
+
+            empty = false;
+            sum += in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            IntSum other0 = (IntSum)other;
+
+            if (other0.empty)
+                return;
+
+            empty = false;
+            sum += other0.sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true);
+        }
+    }
+
+    /** */
+    private static class LongSum implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = LongSum::new;
+
+        /** */
+        private long sum;
+
+        /** */
+        private boolean empty = true;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Long in = (Long)args[0];
+
+            if (in == null)
+                return;
+
+            empty = false;
+            sum += in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            LongSum other0 = (LongSum)other;
+
+            if (other0.empty)
+                return;
+
+            empty = false;
+            sum += other0.sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true);
+        }
+    }
+
+    /** */
+    private static class DecimalSum implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = DecimalSum::new;
+
+        /** */
+        private BigDecimal sum;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            BigDecimal in = (BigDecimal)args[0];
+
+            if (in == null)
+                return;
+
+            sum = sum == null ? in : sum.add(in);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DecimalSum other0 = (DecimalSum)other;
+
+            if (other0.sum == null)
+                return;
+
+            sum = sum == null ? other0.sum : sum.add(other0.sum);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+        }
+    }
+
+    /** */
+    private static class DoubleSumEmptyIsZero implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = DoubleSumEmptyIsZero::new;
+
+        /** */
+        private double sum;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Double in = (Double)args[0];
+
+            if (in == null)
+                return;
+
+            sum += in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DoubleSumEmptyIsZero other0 = (DoubleSumEmptyIsZero)other;
+
+            sum += other0.sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+        }
+    }
+
+    /** */
+    private static class IntSumEmptyIsZero implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = IntSumEmptyIsZero::new;
+
+        /** */
+        private int sum;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Integer in = (Integer)args[0];
+
+            if (in == null)
+                return;
+
+            sum += in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            IntSumEmptyIsZero other0 = (IntSumEmptyIsZero)other;
+
+            sum += other0.sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true);
+        }
+    }
+
+    /** */
+    private static class LongSumEmptyIsZero implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = LongSumEmptyIsZero::new;
+
+        /** */
+        private long sum;
+
+        /** */
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Long in = (Long)args[0];
+
+            if (in == null)
+                return;
+
+            sum += in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            LongSumEmptyIsZero other0 = (LongSumEmptyIsZero)other;
+
+            sum += other0.sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return sum;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true);
+        }
+    }
+
+    /** */
+    private static class DecimalSumEmptyIsZero implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> FACTORY = DecimalSumEmptyIsZero::new;
+
+        /** */
+        private BigDecimal sum;
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            BigDecimal in = (BigDecimal)args[0];
+
+            if (in == null)
+                return;
+
+            sum = sum == null ? in : sum.add(in);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DecimalSumEmptyIsZero other0 = (DecimalSumEmptyIsZero)other;
+
+            sum = sum == null ? other0.sum : sum.add(other0.sum);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return sum != null ? sum : BigDecimal.ZERO;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+        }
+    }
+
+    /** */
+    private static class DoubleMinMax implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> MIN_FACTORY = () -> new DoubleMinMax(true);
+
+        /** */
+        public static final Supplier<Accumulator> MAX_FACTORY = () -> new DoubleMinMax(false);
+
+        /** */
+        private final boolean min;
+
+        /** */
+        private double val;
+
+        /** */
+        private boolean empty = true;
+
+        /** */
+        private DoubleMinMax(boolean min) {
+            this.min = min;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Double in = (Double)args[0];
+
+            if (in == null)
+                return;
+
+            val = empty ? in : min ? Math.min(val, in) : Math.max(val, in);
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DoubleMinMax other0 = (DoubleMinMax)other;
+
+            if (other0.empty)
+                return;
+
+            val = empty ? other0.val : min ? Math.min(val, other0.val) : Math.max(val, other0.val);
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+        }
+    }
+
+    /** */
+    private static class VarCharMinMax implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> MIN_FACTORY = () -> new VarCharMinMax(true);
+
+        /** */
+        public static final Supplier<Accumulator> MAX_FACTORY = () -> new VarCharMinMax(false);
+
+        /** */
+        private final boolean min;
+
+        /** */
+        private CharSequence val;
+
+        /** */
+        private boolean empty = true;
+
+        /** */
+        private VarCharMinMax(boolean min) {
+            this.min = min;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            CharSequence in = (CharSequence)args[0];
+
+            if (in == null)
+                return;
+
+            val = empty ? in : min ?
+                (CharSeqComparator.INSTANCE.compare(val, in) < 0 ? val : in) :
+                (CharSeqComparator.INSTANCE.compare(val, in) < 0 ? in : val);
+
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            VarCharMinMax other0 = (VarCharMinMax)other;
+
+            if (other0.empty)
+                return;
+
+            val = empty ? other0.val : min ?
+                (CharSeqComparator.INSTANCE.compare(val, other0.val) < 0 ? val : other0.val) :
+                (CharSeqComparator.INSTANCE.compare(val, other0.val) < 0 ? other0.val : val);
+
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true);
+        }
+
+        /** */
+        @SuppressWarnings("ComparatorNotSerializable")
+        private static class CharSeqComparator implements Comparator<CharSequence> {
+            /** */
+            private static final CharSeqComparator INSTANCE = new CharSeqComparator();
+
+            /** */
+            @Override public int compare(CharSequence s1, CharSequence s2) {
+                int len = Math.min(s1.length(), s2.length());
+
+                // find the first difference and return
+                for (int i = 0; i < len; i += 1) {
+                    int cmp = Character.compare(s1.charAt(i), s2.charAt(i));
+                    if (cmp != 0)
+                        return cmp;
+                }
+
+                // if there are no differences, then the shorter seq is first
+                return Integer.compare(s1.length(), s2.length());
+            }
+        }
+    }
+
+    /** */
+    private static class IntMinMax implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> MIN_FACTORY = () -> new IntMinMax(true);
+
+        /** */
+        public static final Supplier<Accumulator> MAX_FACTORY = () -> new IntMinMax(false);
+
+        /** */
+        private final boolean min;
+
+        /** */
+        private int val;
+
+        /** */
+        private boolean empty = true;
+
+        /** */
+        private IntMinMax(boolean min) {
+            this.min = min;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Integer in = (Integer)args[0];
+
+            if (in == null)
+                return;
+
+            val = empty ? in : min ? Math.min(val, in) : Math.max(val, in);
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            IntMinMax other0 = (IntMinMax)other;
+
+            if (other0.empty)
+                return;
+
+            val = empty ? other0.val : min ? Math.min(val, other0.val) : Math.max(val, other0.val);
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true);
+        }
+    }
+
+    /** */
+    private static class LongMinMax implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> MIN_FACTORY = () -> new LongMinMax(true);
+
+        /** */
+        public static final Supplier<Accumulator> MAX_FACTORY = () -> new LongMinMax(false);
+
+        /** */
+        private final boolean min;
+
+        /** */
+        private long val;
+
+        /** */
+        private boolean empty = true;
+
+        /** */
+        private LongMinMax(boolean min) {
+            this.min = min;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Long in = (Long)args[0];
+
+            if (in == null)
+                return;
+
+            val = empty ? in : min ? Math.min(val, in) : Math.max(val, in);
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            LongMinMax other0 = (LongMinMax)other;
+
+            if (other0.empty)
+                return;
+
+            val = empty ? other0.val : min ? Math.min(val, other0.val) : Math.max(val, other0.val);
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true);
+        }
+    }
+
+    /** */
+    private static class DecimalMinMax implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> MIN_FACTORY = () -> new DecimalMinMax(true);
+
+        /** */
+        public static final Supplier<Accumulator> MAX_FACTORY = () -> new DecimalMinMax(false);
+
+        /** */
+        private final boolean min;
+
+        /** */
+        private BigDecimal val;
+
+        /** */
+        private DecimalMinMax(boolean min) {
+            this.min = min;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            BigDecimal in = (BigDecimal)args[0];
+
+            if (in == null)
+                return;
+
+            val = val == null ? in : min ? val.min(in) : val.max(in);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DecimalMinMax other0 = (DecimalMinMax)other;
+
+            if (other0.val == null)
+                return;
+
+            val = val == null ? other0.val : min ? val.min(other0.val) : val.max(other0.val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+        }
+    }
+
+    /** */
+    private static class DistinctAccumulator implements Accumulator {
+        /** */
+        private final Accumulator acc;
+
+        /** */
+        private final Set<Object> set = new HashSet<>();
+
+        /** */
+        private DistinctAccumulator(Supplier<Accumulator> accSup) {
+            this.acc = accSup.get();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            Object in = args[0];
+
+            if (in == null)
+                return;
+
+            set.add(in);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            DistinctAccumulator other0 = (DistinctAccumulator)other;
+
+            set.addAll(other0.set);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            for (Object o : set)
+                acc.add(o);
+
+            return acc.end();
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return acc.argumentTypes(typeFactory);
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return acc.returnType(typeFactory);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
new file mode 100644
index 0000000..4626623
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
@@ -0,0 +1,330 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.lang.reflect.Modifier;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+import com.google.common.primitives.Primitives;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumUtils;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.createRowType;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/** */
+public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrapper<Row>>> {
+    /** */
+    private static final LoadingCache<Pair<RelDataType, RelDataType>, Function<Object, Object>> CACHE =
+        CacheBuilder.newBuilder().build(CacheLoader.from(AccumulatorsFactory::cast0));
+
+    /** */
+    public static interface CastFunction extends Function<Object, Object> {
+        @Override Object apply(Object o);
+    }
+
+    /** */
+    private static Function<Object, Object> cast(RelDataType from, RelDataType to) {
+        assert !from.isStruct();
+        assert !to.isStruct();
+
+        return cast(Pair.of(from, to));
+    }
+
+    /** */
+    private static Function<Object, Object> cast(Pair<RelDataType, RelDataType> types) {
+        try {
+            return CACHE.get(types);
+        }
+        catch (ExecutionException e) {
+            throw new IgniteInternalException(e);
+        }
+    }
+
+    /** */
+    private static Function<Object, Object> cast0(Pair<RelDataType, RelDataType> types) {
+        IgniteTypeFactory typeFactory = PlanningContext.empty().typeFactory();
+
+        RelDataType from = types.left;
+        RelDataType to = types.right;
+
+        Class<?> fromType = Primitives.wrap((Class<?>)typeFactory.getJavaClass(from));
+        Class<?> toType = Primitives.wrap((Class<?>)typeFactory.getJavaClass(to));
+
+        if (toType.isAssignableFrom(fromType))
+            return Function.identity();
+
+        if (Void.class == toType)
+            return o -> null;
+
+        return compileCast(typeFactory, from, to);
+    }
+
+    /** */
+    private static Function<Object, Object> compileCast(IgniteTypeFactory typeFactory, RelDataType from,
+        RelDataType to) {
+        RelDataType rowType = createRowType(typeFactory, from);
+        ParameterExpression in_ = Expressions.parameter(Object.class, "in");
+
+        RexToLixTranslator.InputGetter getter =
+            new RexToLixTranslator.InputGetterImpl(
+                ImmutableList.of(
+                    Pair.of(EnumUtils.convert(in_, Object.class, typeFactory.getJavaClass(from)),
+                        PhysTypeImpl.of(typeFactory, rowType,
+                            JavaRowFormat.SCALAR, false))));
+
+        RexBuilder builder = new RexBuilder(typeFactory);
+        RexProgramBuilder programBuilder = new RexProgramBuilder(rowType, builder);
+        RexNode cast = builder.makeCast(to, builder.makeInputRef(from, 0));
+        programBuilder.addProject(cast, null);
+        RexProgram program = programBuilder.getProgram();
+        BlockBuilder list = new BlockBuilder();
+        List<Expression> projects = RexToLixTranslator.translateProjects(program, typeFactory, SqlConformanceEnum.DEFAULT,
+            list, null, DataContext.ROOT, getter, null);
+        list.add(projects.get(0));
+
+        MethodDeclaration decl = Expressions.methodDecl(
+            Modifier.PUBLIC, Object.class, "apply", ImmutableList.of(in_), list.toBlock());
+
+        return Commons.compile(CastFunction.class, Expressions.toString(List.of(decl), "\n", false));
+    }
+
+    /** */
+    private final ExecutionContext<Row> ctx;
+
+    /** */
+    private final AggregateType type;
+
+    /** */
+    private final RelDataType inputRowType;
+
+    /** */
+    private final List<WrapperPrototype> prototypes;
+
+    /** */
+    public AccumulatorsFactory(
+        ExecutionContext<Row> ctx,
+        AggregateType type,
+        List<AggregateCall> aggCalls,
+        RelDataType inputRowType
+    ) {
+        this.ctx = ctx;
+        this.type = type;
+        this.inputRowType = inputRowType;
+
+        prototypes = Commons.transform(aggCalls, WrapperPrototype::new);
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<AccumulatorWrapper<Row>> get() {
+        return Commons.transform(prototypes, WrapperPrototype::get);
+    }
+
+    /** */
+    private final class WrapperPrototype implements Supplier<AccumulatorWrapper<Row>> {
+        /** */
+        private Supplier<Accumulator> accFactory;
+
+        /** */
+        private final AggregateCall call;
+
+        /** */
+        private Function<Object[], Object[]> inAdapter;
+
+        /** */
+        private Function<Object, Object> outAdapter;
+
+        /** */
+        private WrapperPrototype(AggregateCall call) {
+            this.call = call;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AccumulatorWrapper<Row> get() {
+            Accumulator accumulator = accumulator();
+
+            return new AccumulatorWrapperImpl(accumulator, call, inAdapter, outAdapter);
+        }
+
+        /** */
+        @NotNull private Accumulator accumulator() {
+            if (accFactory != null)
+                return accFactory.get();
+
+            // init factory and adapters
+            accFactory = Accumulators.accumulatorFactory(call);
+            Accumulator accumulator = accFactory.get();
+
+            inAdapter = createInAdapter(accumulator);
+            outAdapter = createOutAdapter(accumulator);
+
+            return accumulator;
+        }
+
+        /** */
+        @NotNull private Function<Object[], Object[]> createInAdapter(Accumulator accumulator) {
+            if (type == AggregateType.REDUCE || nullOrEmpty(call.getArgList()))
+                return Function.identity();
+
+            List<RelDataType> inTypes = SqlTypeUtil.projectTypes(inputRowType, call.getArgList());
+            List<RelDataType> outTypes = accumulator.argumentTypes(ctx.getTypeFactory());
+
+            if (call.ignoreNulls())
+                inTypes = Commons.transform(inTypes, this::nonNull);
+
+            List<Function<Object, Object>> casts =
+                Commons.transform(Pair.zip(inTypes, outTypes), AccumulatorsFactory::cast);
+
+            return new Function<Object[], Object[]>() {
+                @Override public Object[] apply(Object[] args) {
+                    for (int i = 0; i < args.length; i++)
+                        args[i] = casts.get(i).apply(args[i]);
+                    return args;
+                }
+            };
+        }
+
+        /** */
+        @NotNull private Function<Object, Object> createOutAdapter(Accumulator accumulator) {
+            if (type == AggregateType.MAP)
+                return Function.identity();
+
+            RelDataType inType = accumulator.returnType(ctx.getTypeFactory());
+            RelDataType outType = call.getType();
+
+            return cast(inType, outType);
+        }
+
+        /** */
+        private RelDataType nonNull(RelDataType type) {
+            return ctx.getTypeFactory().createTypeWithNullability(type, false);
+        }
+    }
+
+    /** */
+    private final class AccumulatorWrapperImpl implements AccumulatorWrapper<Row> {
+        /** */
+        private final Accumulator accumulator;
+
+        /** */
+        private final Function<Object[], Object[]> inAdapter;
+
+        /** */
+        private final Function<Object, Object> outAdapter;
+
+        /** */
+        private final List<Integer> argList;
+
+        /** */
+        private final int filterArg;
+
+        /** */
+        private final boolean ignoreNulls;
+
+        /** */
+        private final RowHandler<Row> handler;
+
+        /** */
+        AccumulatorWrapperImpl(
+            Accumulator accumulator,
+            AggregateCall call,
+            Function<Object[], Object[]> inAdapter,
+            Function<Object, Object> outAdapter
+        ) {
+            this.accumulator = accumulator;
+            this.inAdapter = inAdapter;
+            this.outAdapter = outAdapter;
+
+            argList = call.getArgList();
+            ignoreNulls = call.ignoreNulls();
+            filterArg = call.hasFilter() ? call.filterArg : -1;
+
+            handler = ctx.rowHandler();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Row row) {
+            assert type != AggregateType.REDUCE;
+
+            if (filterArg >= 0 && Boolean.TRUE != handler.get(filterArg, row))
+                return;
+
+            Object[] args = new Object[argList.size()];
+            for (int i = 0; i < argList.size(); i++) {
+                args[i] = handler.get(argList.get(i), row);
+
+                if (ignoreNulls && args[i] == null)
+                    return;
+            }
+
+            accumulator.add(inAdapter.apply(args));
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            assert type != AggregateType.MAP;
+
+            return outAdapter.apply(accumulator.end());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator accumulator) {
+            assert type == AggregateType.REDUCE;
+
+            this.accumulator.apply(accumulator);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Accumulator accumulator() {
+            assert type == AggregateType.MAP;
+
+            return accumulator;
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AggregateType.java
similarity index 60%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AggregateType.java
index ce7d926..c743e2a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AggregateType.java
@@ -14,27 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
 
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+/**
+ *
+ */
+public enum AggregateType {
+    /** Map phase. */
+    MAP,
 
-    /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+    /** Reduce phase. */
+    REDUCE,
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+    /** Single phase aggregate. */
+    SINGLE
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java
new file mode 100644
index 0000000..9606754
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.util.Arrays;
+
+import static org.apache.ignite.internal.util.ArrayUtils.OBJECT_EMPTY_ARRAY;
+
+/**
+ *
+ */
+public class GroupKey {
+    /** */
+    public static final GroupKey EMPTY_GRP_KEY = new GroupKey(OBJECT_EMPTY_ARRAY);
+
+    /** */
+    private final Object[] fields;
+
+    /** */
+    public GroupKey(Object[] fields) {
+        this.fields = fields;
+    }
+
+    /** */
+    public Object field(int idx) {
+        return fields[idx];
+    }
+
+    /** */
+    public int fieldsCount() {
+        return fields.length;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        GroupKey groupKey = (GroupKey) o;
+
+        return Arrays.equals(fields, groupKey.fields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return Arrays.hashCode(fields);
+    }
+
+    /** */
+    public static Builder builder(int rowLen) {
+        return new Builder(rowLen);
+    }
+
+    /** */
+    public static class Builder {
+        /** */
+        private final Object[] fields;
+
+        /** */
+        private int idx;
+
+        /** */
+        private Builder(int rowLen) {
+            fields = new Object[rowLen];
+        }
+
+        /** */
+        public Builder add(Object val) {
+            if (idx == fields.length)
+                throw new IndexOutOfBoundsException();
+
+            fields[idx++] = val;
+
+            return this;
+        }
+
+        /** */
+        public GroupKey build() {
+            assert idx == fields.length;
+
+            return new GroupKey(fields);
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
new file mode 100644
index 0000000..9325885
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteLogger;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Abstract node of execution tree.
+ */
+public abstract class AbstractNode<Row> implements Node<Row> {
+    /** */
+    protected static final int MODIFY_BATCH_SIZE = 100;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_BATCH_SIZE", 100);
+
+    /** */
+    protected static final int IO_BATCH_SIZE = 256;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IO_BATCH_SIZE", 256);
+
+    /** */
+    protected static final int IO_BATCH_CNT = 4;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IO_BATCH_CNT", 4);
+
+    /** */
+    protected final int inBufSize = Commons.IN_BUFFER_SIZE;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IN_BUFFER_SIZE", 2);
+
+    /** */
+    protected final IgniteLogger log = IgniteLogger.forClass(getClass());
+
+    /** for debug purpose */
+    private volatile Thread thread;
+
+    /**
+     * {@link Inbox} node may not have proper context at creation time in case it
+     * creates on first message received from a remote source. This case the context
+     * sets in scope of {@link Inbox#init(ExecutionContext, RelDataType, Collection, Comparator)} method call.
+     */
+    private ExecutionContext<Row> ctx;
+
+    /** */
+    private RelDataType rowType;
+
+    /** */
+    private Downstream<Row> downstream;
+
+    /** */
+    private boolean closed;
+
+    /** */
+    private List<Node<Row>> sources;
+
+    /**
+     * @param ctx Execution context.
+     */
+    protected AbstractNode(ExecutionContext<Row> ctx, RelDataType rowType) {
+        this.ctx = ctx;
+        this.rowType = rowType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutionContext<Row> context() {
+        return ctx;
+    }
+
+    /** */
+    protected void context(ExecutionContext<Row> ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType rowType() {
+        return rowType;
+    }
+
+    /** */
+    protected void rowType(RelDataType rowType) {
+        this.rowType = rowType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(List<Node<Row>> sources) {
+        this.sources = sources;
+
+        for (int i = 0; i < sources.size(); i++)
+            sources.get(i).onRegister(requestDownstream(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<Node<Row>> sources() {
+        return sources;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        if (isClosed())
+            return;
+
+        closeInternal();
+
+        if (!nullOrEmpty(sources()))
+            sources().forEach(Commons::closeQuiet);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rewind() {
+        rewindInternal();
+
+        if (!nullOrEmpty(sources()))
+            sources().forEach(Node::rewind);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onRegister(Downstream<Row> downstream) {
+        this.downstream = downstream;
+    }
+
+    /**
+     * Processes given exception.
+     *
+     * @param e Exception.
+     */
+    public void onError(Throwable e) {
+        if (e instanceof ExecutionCancelledException)
+            log.warn("Execution is cancelled.", e);
+        else
+            onErrorInternal(e);
+    }
+
+    /** */
+    protected void closeInternal() {
+        closed = true;
+    }
+
+    /** */
+    protected abstract void rewindInternal();
+
+    /** */
+    protected void onErrorInternal(Throwable e) {
+        Downstream<Row> downstream = downstream();
+
+        assert downstream != null;
+
+        try {
+            downstream.onError(e);
+        }
+        finally {
+            Commons.closeQuiet(this);
+        }
+    }
+
+    /**
+     * @return {@code true} if the subtree is canceled.
+     */
+    protected boolean isClosed() {
+        return closed;
+    }
+
+    /** */
+    protected void checkState() throws Exception {
+        if (context().isCancelled())
+            throw new ExecutionCancelledException();
+        if (Thread.interrupted())
+            throw new IgniteInternalCheckedException("Thread was interrupted.");
+        if (!IgniteUtils.assertionsEnabled())
+            return;
+        if (thread == null)
+            thread = Thread.currentThread();
+        else
+            assert thread == Thread.currentThread();
+    }
+
+    /** */
+    protected abstract Downstream<Row> requestDownstream(int idx);
+
+    /** */
+    @Override public Downstream<Row> downstream() {
+        return downstream;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractSetOpNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractSetOpNode.java
new file mode 100644
index 0000000..397bf60
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractSetOpNode.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Abstract execution node for set operators (EXCEPT, INTERSECT).
+ */
+public abstract class AbstractSetOpNode<Row> extends AbstractNode<Row> {
+    /** */
+    private final AggregateType type;
+
+    /** */
+    private final Grouping<Row> grouping;
+
+    /** */
+    private int requested;
+
+    /** */
+    private int waiting;
+
+    /** Current source index. */
+    private int curSrcIdx;
+
+    /** */
+    private boolean inLoop;
+
+    /** */
+    protected AbstractSetOpNode(ExecutionContext<Row> ctx, RelDataType rowType, AggregateType type, boolean all,
+        RowFactory<Row> rowFactory, Grouping<Row> grouping) {
+        super(ctx, rowType);
+
+        this.type = type;
+        this.grouping = grouping;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) throws Exception {
+        assert !nullOrEmpty(sources());
+        assert rowsCnt > 0 && requested == 0;
+        assert waiting <= 0;
+
+        checkState();
+
+        requested = rowsCnt;
+
+        if (waiting == 0)
+            sources().get(curSrcIdx).request(waiting = inBufSize);
+        else if (!inLoop)
+            context().execute(this::flush, this::onError);
+    }
+
+    /** */
+    public void push(Row row, int idx) throws Exception {
+        assert downstream() != null;
+        assert waiting > 0;
+
+        checkState();
+
+        waiting--;
+
+        grouping.add(row, idx);
+
+        if (waiting == 0)
+            sources().get(curSrcIdx).request(waiting = inBufSize);
+    }
+
+    /** */
+    public void end(int idx) throws Exception {
+        assert downstream() != null;
+        assert waiting > 0;
+        assert curSrcIdx == idx;
+
+        checkState();
+
+        grouping.endOfSet(idx);
+
+        if (type == AggregateType.SINGLE && grouping.isEmpty())
+            curSrcIdx = sources().size(); // Skip subsequent sources.
+        else
+            curSrcIdx++;
+
+        if (curSrcIdx >= sources().size()) {
+            waiting = -1;
+
+            flush();
+        }
+        else
+            sources().get(curSrcIdx).request(waiting);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void rewindInternal() {
+        requested = 0;
+        waiting = 0;
+        grouping.groups.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        return new Downstream<Row>() {
+            @Override public void push(Row row) throws Exception {
+                AbstractSetOpNode.this.push(row, idx);
+            }
+
+            @Override public void end() throws Exception {
+                AbstractSetOpNode.this.end(idx);
+            }
+
+            @Override public void onError(Throwable e) {
+                AbstractSetOpNode.this.onError(e);
+            }
+        };
+    }
+
+    /** */
+    private void flush() throws Exception {
+        if (isClosed())
+            return;
+
+        checkState();
+
+        assert waiting == -1;
+
+        int processed = 0;
+
+        inLoop = true;
+
+        try {
+            if (requested > 0 && !grouping.isEmpty()) {
+                int toSnd = Math.min(requested, inBufSize - processed);
+
+                for (Row row : grouping.getRows(toSnd)) {
+                    requested--;
+
+                    downstream().push(row);
+
+                    processed++;
+                }
+
+                if (processed >= inBufSize && requested > 0) {
+                    // Allow others to do their job.
+                    context().execute(this::flush, this::onError);
+
+                    return;
+                }
+            }
+        }
+        finally {
+            inLoop = false;
+        }
+
+        if (requested > 0) {
+            requested = 0;
+
+            downstream().end();
+        }
+    }
+
+    /** */
+    protected abstract static class Grouping<Row> {
+        /** */
+        protected final Map<GroupKey, int[]> groups = new HashMap<>();
+
+        /** */
+        protected final RowHandler<Row> hnd;
+
+        /** */
+        protected final AggregateType type;
+
+        /** */
+        protected final boolean all;
+
+        /** */
+        protected final RowFactory<Row> rowFactory;
+
+        /** Processed rows count in current set. */
+        protected int rowsCnt = 0;
+
+        /** */
+        protected Grouping(ExecutionContext<Row> ctx, RowFactory<Row> rowFactory, AggregateType type, boolean all) {
+            hnd = ctx.rowHandler();
+            this.type = type;
+            this.all = all;
+            this.rowFactory = rowFactory;
+        }
+
+        /** */
+        private void add(Row row, int setIdx) {
+            if (type == AggregateType.REDUCE) {
+                assert setIdx == 0 : "Unexpected set index: " + setIdx;
+
+                addOnReducer(row);
+            }
+            else if (type == AggregateType.MAP)
+                addOnMapper(row, setIdx);
+            else
+                addOnSingle(row, setIdx);
+
+            rowsCnt++;
+        }
+
+        /**
+         * @param cnt Number of rows.
+         *
+         * @return Actually sent rows number.
+         */
+        private List<Row> getRows(int cnt) {
+            if (nullOrEmpty(groups))
+                return Collections.emptyList();
+            else if (type == AggregateType.MAP)
+                return getOnMapper(cnt);
+            else
+                return getOnSingleOrReducer(cnt);
+        }
+
+        /** */
+        protected GroupKey key(Row row) {
+            int size = hnd.columnCount(row);
+
+            Object[] fields = new Object[size];
+
+            for (int i = 0; i < size; i++)
+                fields[i] = hnd.get(i, row);
+
+            return new GroupKey(fields);
+        }
+
+        /** */
+        protected void endOfSet(int setIdx) {
+            rowsCnt = 0;
+        }
+
+        /** */
+        protected abstract void addOnSingle(Row row, int setIdx);
+
+        /** */
+        protected abstract void addOnMapper(Row row, int setIdx);
+
+        /** */
+        protected void addOnReducer(Row row) {
+            GroupKey grpKey = (GroupKey)hnd.get(0, row);
+            int[] cntrsMap = (int[])hnd.get(1, row);
+
+            int[] cntrs = groups.computeIfAbsent(grpKey, k -> new int[cntrsMap.length]);
+
+            assert cntrs.length == cntrsMap.length;
+
+            for (int i = 0; i < cntrsMap.length; i++)
+                cntrs[i] += cntrsMap[i];
+        }
+
+        /** */
+        protected List<Row> getOnMapper(int cnt) {
+            Iterator<Map.Entry<GroupKey, int[]>> it = groups.entrySet().iterator();
+
+            int amount = Math.min(cnt, groups.size());
+            List<Row> res = new ArrayList<>(amount);
+
+            while (amount > 0 && it.hasNext()) {
+                Map.Entry<GroupKey, int[]> entry = it.next();
+
+                // Skip row if it doesn't affect the final result.
+                if (affectResult(entry.getValue())) {
+                    res.add(rowFactory.create(entry.getKey(), entry.getValue()));
+
+                    amount--;
+                }
+
+                it.remove();
+            }
+
+            return res;
+        }
+
+        /** */
+        protected List<Row> getOnSingleOrReducer(int cnt) {
+            Iterator<Map.Entry<GroupKey, int[]>> it = groups.entrySet().iterator();
+
+            List<Row> res = new ArrayList<>(cnt);
+
+            while (it.hasNext() && cnt > 0) {
+                Map.Entry<GroupKey, int[]> entry = it.next();
+
+                GroupKey key = entry.getKey();
+
+                Object[] fields = new Object[key.fieldsCount()];
+
+                for (int i = 0; i < fields.length; i++)
+                    fields[i] = key.field(i);
+
+                Row row = rowFactory.create(fields);
+
+                int[] cntrs = entry.getValue();
+
+                int availableRows = availableRows(entry.getValue());
+
+                if (availableRows <= cnt) {
+                    it.remove();
+
+                    if (availableRows == 0)
+                        continue;
+
+                    cnt -= availableRows;
+                }
+                else {
+                    availableRows = cnt;
+
+                    decrementAvailableRows(cntrs, availableRows);
+
+                    cnt = 0;
+                }
+
+                for (int i = 0; i < availableRows; i++)
+                    res.add(row);
+            }
+
+            return res;
+        }
+
+        /**
+         * Return {@code true} if counters affects the final result, or {@code false} if row can be skipped.
+         */
+        protected abstract boolean affectResult(int[] cntrs);
+
+        /** */
+        protected abstract int availableRows(int[] cntrs);
+
+        /** */
+        protected abstract void decrementAvailableRows(int[] cntrs, int amount);
+
+        /** */
+        private boolean isEmpty() {
+            return groups.isEmpty();
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java
new file mode 100644
index 0000000..de71f0d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java
@@ -0,0 +1,479 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class CorrelatedNestedLoopJoinNode<Row> extends AbstractNode<Row> {
+    /** */
+    private final Predicate<Row> cond;
+
+    /** */
+    private final List<CorrelationId> correlationIds;
+
+    /** */
+    private final JoinRelType joinType;
+
+    /** */
+    private final RowHandler<Row> handler;
+
+    /** */
+    private final int leftInBufferSize;
+
+    /** */
+    private final int rightInBufferSize;
+
+    /** */
+    private final BitSet leftMatched = new BitSet();
+
+    /** */
+    private int requested;
+
+    /** */
+    private int waitingLeft;
+
+    /** */
+    private int waitingRight;
+
+    /** */
+    private List<Row> leftInBuf;
+
+    /** */
+    private List<Row> rightInBuf;
+
+    /** */
+    private int leftIdx;
+
+    /** */
+    private int rightIdx;
+
+    /** */
+    private Row rightEmptyRow;
+
+    /** */
+    private State state = State.INITIAL;
+
+    /** */
+    private enum State {
+        INITIAL, FILLING_LEFT, FILLING_RIGHT, IDLE, IN_LOOP, END
+    }
+
+    /**
+     * @param ctx Execution context.
+     * @param cond Join expression.
+     */
+    public CorrelatedNestedLoopJoinNode(ExecutionContext<Row> ctx, RelDataType rowType, Predicate<Row> cond,
+        Set<CorrelationId> correlationIds, JoinRelType joinType) {
+        super(ctx, rowType);
+
+        assert !nullOrEmpty(correlationIds);
+
+        this.cond = cond;
+        this.correlationIds = new ArrayList<>(correlationIds);
+        this.joinType = joinType;
+
+        leftInBufferSize = correlationIds.size();
+        rightInBufferSize = inBufSize;
+
+        handler = ctx.rowHandler();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) throws Exception {
+        assert !nullOrEmpty(sources()) && sources().size() == 2;
+        assert rowsCnt > 0 && requested == 0;
+
+        checkState();
+
+        requested = rowsCnt;
+
+        onRequest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void rewindInternal() {
+        leftInBuf = null;
+        rightInBuf = null;
+
+        leftIdx = 0;
+        rightIdx = 0;
+
+        requested = 0;
+        waitingLeft = 0;
+        waitingRight = 0;
+
+        state = State.INITIAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        if (idx == 0)
+            return new Downstream<Row>() {
+                /** {@inheritDoc} */
+                @Override public void push(Row row) throws Exception {
+                    pushLeft(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() throws Exception {
+                    endLeft();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    CorrelatedNestedLoopJoinNode.this.onError(e);
+                }
+            };
+        else if (idx == 1)
+            return new Downstream<Row>() {
+                /** {@inheritDoc} */
+                @Override public void push(Row row) throws Exception {
+                    pushRight(row);
+                }
+
+                /** {@inheritDoc} */
+                @Override public void end() throws Exception {
+                    endRight();
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onError(Throwable e) {
+                    CorrelatedNestedLoopJoinNode.this.onError(e);
+                }
+            };
+
+        throw new IndexOutOfBoundsException();
+    }
+
+    /** */
+    private void pushLeft(Row row) throws Exception {
+        assert downstream() != null;
+        assert waitingLeft > 0;
+
+        checkState();
+
+        waitingLeft--;
+
+        if (leftInBuf == null)
+            leftInBuf = new ArrayList<>(leftInBufferSize);
+
+        leftInBuf.add(row);
+
+        onPushLeft();
+    }
+
+    /** */
+    private void pushRight(Row row) throws Exception {
+        assert downstream() != null;
+        assert waitingRight > 0;
+
+        checkState();
+
+        waitingRight--;
+
+        if (rightInBuf == null)
+            rightInBuf = new ArrayList<>(rightInBufferSize);
+
+        rightInBuf.add(row);
+
+        onPushRight();
+    }
+
+    /** */
+    private void endLeft() throws Exception {
+        assert downstream() != null;
+        assert waitingLeft > 0;
+
+        checkState();
+
+        waitingLeft = -1;
+
+        if (leftInBuf == null)
+            leftInBuf = Collections.emptyList();
+
+        onEndLeft();
+    }
+
+    /** */
+    private void endRight() throws Exception {
+        assert downstream() != null;
+        assert waitingRight > 0;
+
+        checkState();
+
+        waitingRight = -1;
+
+        if (rightInBuf == null)
+            rightInBuf = Collections.emptyList();
+
+        onEndRight();
+    }
+
+    /** */
+    private void onRequest() throws Exception {
+        switch (state) {
+            case IN_LOOP:
+            case FILLING_RIGHT:
+            case FILLING_LEFT:
+                break;
+            case INITIAL:
+                assert waitingLeft == 0;
+                assert waitingRight == 0;
+                assert nullOrEmpty(leftInBuf);
+                assert nullOrEmpty(rightInBuf);
+
+                context().execute(() -> {
+                    checkState();
+
+                    state = State.FILLING_LEFT;
+                    leftSource().request(waitingLeft = leftInBufferSize);
+                }, this::onError);
+
+                break;
+            case IDLE:
+                assert rightInBuf != null;
+                assert leftInBuf != null;
+                assert waitingRight == -1 || waitingRight == 0 && rightInBuf.size() == rightInBufferSize;
+                assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+                context().execute(() -> {
+                    checkState();
+
+                    join();
+                }, this::onError);
+
+                break;
+
+            case END:
+                downstream().end();
+                break;
+
+            default:
+                throw new AssertionError("Unexpected state:" + state);
+        }
+    }
+
+    /** */
+    private void onPushLeft() throws Exception {
+        assert state == State.FILLING_LEFT : "Unexpected state:" + state;
+        assert waitingRight == 0 || waitingRight == -1;
+        assert nullOrEmpty(rightInBuf);
+
+        if (leftInBuf.size() == leftInBufferSize) {
+            assert waitingLeft == 0;
+
+            prepareCorrelations();
+
+            if (waitingRight == -1)
+                rightSource().rewind();
+
+            state = State.FILLING_RIGHT;
+            rightSource().request(waitingRight = rightInBufferSize);
+        }
+    }
+
+    /** */
+    private void onPushRight() throws Exception {
+        assert state == State.FILLING_RIGHT : "Unexpected state:" + state;
+        assert !nullOrEmpty(leftInBuf);
+        assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+        if (rightInBuf.size() == rightInBufferSize) {
+            assert waitingRight == 0;
+
+            state = State.IDLE;
+
+            join();
+        }
+    }
+
+    /** */
+    private void onEndLeft() throws Exception {
+        assert state == State.FILLING_LEFT : "Unexpected state:" + state;
+        assert waitingLeft == -1;
+        assert waitingRight == 0 || waitingRight == -1;
+        assert nullOrEmpty(rightInBuf);
+
+        if (nullOrEmpty(leftInBuf)) {
+            waitingRight = -1;
+
+            state = State.END;
+
+            if (requested > 0)
+                downstream().end();
+        }
+        else {
+            prepareCorrelations();
+
+            if (waitingRight == -1)
+                rightSource().rewind();
+
+            state = State.FILLING_RIGHT;
+
+            rightSource().request(waitingRight = rightInBufferSize);
+        }
+    }
+
+    /** */
+    private void onEndRight() throws Exception {
+        assert state == State.FILLING_RIGHT : "Unexpected state:" + state;
+        assert waitingRight == -1;
+        assert !nullOrEmpty(leftInBuf);
+        assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+        state = State.IDLE;
+
+        join();
+    }
+
+    /** */
+    private void join() throws Exception {
+        assert state == State.IDLE;
+
+        state = State.IN_LOOP;
+
+        try {
+            while (requested > 0 && rightIdx < rightInBuf.size()) {
+                if (leftIdx == leftInBuf.size())
+                    leftIdx = 0;
+
+                while (requested > 0 && leftIdx < leftInBuf.size()) {
+                    checkState();
+
+                    Row row = handler.concat(leftInBuf.get(leftIdx), rightInBuf.get(rightIdx));
+
+                    if (cond.test(row)) {
+                        leftMatched.set(leftIdx);
+
+                        requested--;
+
+                        downstream().push(row);
+                    }
+
+                    leftIdx++;
+                }
+
+                if (leftIdx == leftInBuf.size())
+                    rightInBuf.set(rightIdx++, null);
+            }
+        }
+        finally {
+            state = State.IDLE;
+        }
+
+        if (rightIdx == rightInBuf.size()) {
+            leftIdx = 0;
+            rightIdx = 0;
+
+            if (waitingRight == 0) {
+                rightInBuf = null;
+
+                state = State.FILLING_RIGHT;
+
+                rightSource().request(waitingRight = rightInBufferSize);
+
+                return;
+            }
+
+            if (joinType == JoinRelType.LEFT && !nullOrEmpty(leftInBuf)) {
+                if (rightEmptyRow == null)
+                    rightEmptyRow = handler.factory(context().getTypeFactory(), rightSource().rowType()).create();
+
+                int notMatchedIdx = leftMatched.nextClearBit(0);
+
+                while (requested > 0 && notMatchedIdx < leftInBuf.size()) {
+                    downstream().push(handler.concat(leftInBuf.get(notMatchedIdx), rightEmptyRow));
+
+                    requested--;
+
+                    leftMatched.set(notMatchedIdx);
+
+                    notMatchedIdx = leftMatched.nextClearBit(notMatchedIdx + 1);
+                }
+
+                if (requested == 0 && notMatchedIdx < leftInBuf.size())
+                    return; // Some rows required to be pushed, wait for request.
+            }
+
+            if (waitingLeft == 0) {
+                rightInBuf = null;
+                leftInBuf = null;
+                leftMatched.clear();
+
+                state = State.FILLING_LEFT;
+
+                leftSource().request(waitingLeft = leftInBufferSize);
+
+                return;
+            }
+
+            assert waitingLeft == -1 && waitingRight == -1;
+
+            if (requested > 0) {
+                leftInBuf = null;
+                rightInBuf = null;
+
+                state = State.END;
+
+                if (requested > 0)
+                    downstream().end();
+
+                return;
+            }
+
+            // let's free the rows for GC
+            leftInBuf = Collections.emptyList();
+            rightInBuf = Collections.emptyList();
+        }
+    }
+
+    /** */
+    private Node<Row> leftSource() {
+        return sources().get(0);
+    }
+
+    /** */
+    private Node<Row> rightSource() {
+        return sources().get(1);
+    }
+
+    /** */
+    private void prepareCorrelations() {
+        for (int i = 0; i < correlationIds.size(); i++) {
+            Row row = i < leftInBuf.size() ? leftInBuf.get(i) : first(leftInBuf);
+            context().setCorrelated(row, correlationIds.get(i).getId());
+        }
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
similarity index 53%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
index ce7d926..8a1bcc8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
@@ -14,27 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.calcite;
 
-/** Stubs */
-public class Stubs {
-    /** */
-    public static int intFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
 
-    /** */
-    public static long longFoo(Object... args) {
-        return args == null ? 0 : args.length;
-    }
+/**
+ * Represents an abstract data consumer.
+ *
+ * <b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request(int)},
+ * {@link Downstream#push(Object)} and {@link Downstream#end()} methods should be used from one single thread.
+ */
+public interface Downstream<Row> {
+    /**
+     * Pushes a row to consumer.
+     *
+     * @param row Data row.
+     */
+    void push(Row row) throws Exception;
 
-    /** */
-    public static boolean boolFoo(Object... args) {
-        return args == null;
-    }
+    /**
+     * Signals that data is over.
+     */
+    void end() throws Exception;
 
-    /** */
-    public static String stringFoo(Object... args) {
-        return args == null ? "null" : "not null";
-    }
+    /**
+     * Notifies consumer about error.
+     *
+     * @param e Error to notify consumer about.
+     */
+    void onError(Throwable e);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
new file mode 100644
index 0000000..f8afce8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.function.Predicate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class FilterNode<Row> extends AbstractNode<Row> implements SingleNode<Row>, Downstream<Row> {
+    /** */
+    private final Predicate<Row> pred;
+
+    /** */
+    private final Deque<Row> inBuf = new ArrayDeque<>(inBufSize);
+
+    /** */
+    private int requested;
+
+    /** */
+    private int waiting;
+
+    /** */
+    private boolean inLoop;
+
+    /**
+     * @param ctx Execution context.
+     * @param pred Predicate.
+     */
+    public FilterNode(ExecutionContext<Row> ctx, RelDataType rowType, Predicate<Row> pred) {
+        super(ctx, rowType);
+
+        this.pred = pred;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void request(int rowsCnt) throws Exception {
+        assert !nullOrEmpty(sources()) && sources().size() == 1;
+        assert rowsCnt > 0 && requested == 0;
+
+        checkState();
+
+        requested = rowsCnt;
+
+        if (!inLoop)
+            context().execute(this::doFilter, this::onError);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void push(Row row) throws Exception {
+        assert downstream() != null;
+        assert waiting > 0;
+
+        checkState();
+
+        waiting--;
+
+        if (pred.test(row))
+            inBuf.add(row);
+
+        filter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void end() throws Exception {
+        assert downstream() != null;
+        assert waiting > 0;
+
+        checkState();
+
+        waiting = -1;
+
+        filter();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Downstream<Row> requestDownstream(int idx) {
+        if (idx != 0)
+            throw new IndexOutOfBoundsException();
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void rewindInternal() {
+        requested = 0;
+        waiting = 0;
+        inBuf.clear();
+    }
+
+    /** */
+    private void doFilter() throws Exception {
+        checkState();
+
+        filter();
+    }
+
+    /** */
+    private void filter() throws Exception {
+        inLoop = true;
+        try {
+            while (requested > 0 && !inBuf.isEmpty()) {
+                checkState();
+
+                requested--;
+                downstream().push(inBuf.remove());
+            }
+        }
+        finally {
+            inLoop = false;
+        }
+
+        if (inBuf.isEmpty() && waiting == 0)
+            source().request(waiting = inBufSize);
+
+        if (waiting == -1 && requested > 0) {
+            assert inBuf.isEmpty();
... 9610 lines suppressed ...