http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
new file mode 100644
index 0000000..00ff500
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -0,0 +1,290 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep;
+
+import org.apache.ignite.*;
+import org.h2.engine.*;
+import org.h2.index.*;
+import org.h2.message.*;
+import org.h2.result.*;
+import org.h2.table.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Merge index.
+ */
+public abstract class GridMergeIndex extends BaseIndex {
+    /** */
+    protected final GridResultPage<?> END = new GridResultPage<Object>(null, 
null);
+
+    /** */
+    private static final int MAX_FETCH_SIZE = 100000;
+
+    /** */
+    private final AtomicInteger cnt = new AtomicInteger(0);
+
+    /** Result sources. */
+    private final AtomicInteger srcs = new AtomicInteger(0);
+
+    /**
+     * Will be r/w from query execution thread only, does not need to be 
threadsafe.
+     */
+    private ArrayList<Row> fetched = new ArrayList<>();
+
+    /**
+     * @param tbl Table.
+     * @param name Index name.
+     * @param type Type.
+     * @param cols Columns.
+     */
+    public GridMergeIndex(GridMergeTable tbl, String name, IndexType type, 
IndexColumn[] cols) {
+        initBaseIndex(tbl, 0, name, cols, type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCount(Session session) {
+        return cnt.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCountApproximation() {
+        return getRowCount(null);
+    }
+
+    /**
+     * @param srcs Number of sources.
+     */
+    public void setNumberOfSources(int srcs) {
+        this.srcs.set(srcs);
+    }
+
+    /**
+     * @param cnt Count.
+     */
+    public void addCount(int cnt) {
+        this.cnt.addAndGet(cnt);
+    }
+
+    /**
+     * @param page Page.
+     */
+    public final void addPage(GridResultPage<?> page) {
+        if (!page.response().rows().isEmpty())
+            addPage0(page);
+        else
+            assert page.response().isLast();
+
+        if (page.response().isLast()) {
+            int srcs0 = srcs.decrementAndGet();
+
+            assert srcs0 >= 0;
+
+            if (srcs0 == 0)
+                addPage0(END); // We've fetched all.
+        }
+    }
+
+    /**
+     * @param page Page.
+     */
+    protected abstract void addPage0(GridResultPage<?> page);
+
+    /** {@inheritDoc} */
+    @Override public Cursor find(Session session, SearchRow first, SearchRow 
last) {
+        if (fetched == null)
+            throw new IgniteException("Fetched result set was too large.");
+
+        if (fetched.size() == cnt.get())  // We've fetched all the rows.
+            return findAllFetched(fetched, first, last);
+
+        return findInStream(first, last);
+    }
+
+    /**
+     * @param first First row.
+     * @param last Last row.
+     * @return Cursor. Usually it must be {@link FetchingCursor} instance.
+     */
+    protected abstract Cursor findInStream(@Nullable SearchRow first, 
@Nullable SearchRow last);
+
+    /**
+     * @param fetched Fetched rows.
+     * @param first First row.
+     * @param last Last row.
+     * @return Cursor.
+     */
+    protected Cursor findAllFetched(List<Row> fetched, @Nullable SearchRow 
first, @Nullable SearchRow last) {
+        return new IteratorCursor(fetched.iterator());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkRename() {
+        throw DbException.getUnsupportedException("rename");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(Session session) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void add(Session session, Row row) {
+        throw DbException.getUnsupportedException("add");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(Session session, Row row) {
+        throw DbException.getUnsupportedException("remove row");
+    }
+
+    /** {@inheritDoc} */
+    @Override public double getCost(Session session, int[] masks, TableFilter 
filter, SortOrder sortOrder) {
+        return getRowCountApproximation() + Constants.COST_ROW_OFFSET;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(Session session) {
+        throw DbException.getUnsupportedException("remove index");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void truncate(Session session) {
+        throw DbException.getUnsupportedException("truncate");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canGetFirstOrLast() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cursor findFirstOrLast(Session session, boolean first) {
+        throw DbException.getUnsupportedException("findFirstOrLast");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean needRebuild() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getDiskSpaceUsed() {
+        return 0;
+    }
+
+    /**
+     * Cursor over iterator.
+     */
+    protected class IteratorCursor implements Cursor {
+        /** */
+        protected Iterator<Row> iter;
+
+        /** */
+        protected Row cur;
+
+        /**
+         * @param iter Iterator.
+         */
+        public IteratorCursor(Iterator<Row> iter) {
+            assert iter != null;
+
+            this.iter = iter;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Row get() {
+            return cur;
+        }
+
+        /** {@inheritDoc} */
+        @Override public SearchRow getSearchRow() {
+            return get();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            cur = iter.hasNext() ? iter.next() : null;
+
+            return cur != null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean previous() {
+            throw DbException.getUnsupportedException("previous");
+        }
+    }
+
+    /**
+     * Fetching cursor.
+     */
+    protected class FetchingCursor extends IteratorCursor {
+        /** */
+        private Iterator<Row> stream;
+
+        /**
+         */
+        public FetchingCursor(Iterator<Row> stream) {
+            super(new FetchedIterator());
+
+            assert stream != null;
+
+            this.stream = stream;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            if (super.next()) {
+                assert cur != null;
+
+                if (iter == stream && fetched != null) { // Cache fetched rows 
for reuse.
+                    if (fetched.size() == MAX_FETCH_SIZE)
+                        fetched = null; // Throw away fetched result if it is 
too large.
+                    else
+                        fetched.add(cur);
+                }
+
+                return true;
+            }
+
+            if (iter == stream) // We've fetched the stream.
+                return false;
+
+            iter = stream; // Switch from cached to stream.
+
+            return next();
+        }
+    }
+
+    /**
+     * List iterator without {@link ConcurrentModificationException}.
+     */
+    private class FetchedIterator implements Iterator<Row> {
+        /** */
+        private int idx;
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return fetched != null && idx < fetched.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Row next() {
+            return fetched.get(idx++);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove() {
+            throw new UnsupportedOperationException();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
new file mode 100644
index 0000000..d5cbf9e
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
@@ -0,0 +1,85 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep;
+
+import org.apache.ignite.*;
+import org.h2.index.*;
+import org.h2.result.*;
+import org.h2.table.*;
+import org.h2.value.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Unsorted merge index.
+ */
+public class GridMergeIndexUnsorted extends GridMergeIndex {
+    /** */
+    private final BlockingQueue<GridResultPage<?>> queue = new 
LinkedBlockingQueue<>();
+
+    /**
+     * @param tbl  Table.
+     * @param name Index name.
+     */
+    public GridMergeIndexUnsorted(GridMergeTable tbl, String name) {
+        super(tbl, name, IndexType.createScan(false), 
IndexColumn.wrap(tbl.getColumns()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage0(GridResultPage<?> page) {
+        queue.add(page);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Cursor findInStream(@Nullable SearchRow first, 
@Nullable SearchRow last) {
+        return new FetchingCursor(new Iterator<Row>() {
+            /** */
+            Iterator<Value[]> iter = Collections.emptyIterator();
+
+            @Override public boolean hasNext() {
+                if (iter.hasNext())
+                    return true;
+
+                GridResultPage<?> page;
+
+                try {
+                    page = queue.take();
+                }
+                catch (InterruptedException e) {
+                    throw new IgniteException("Query execution was 
interrupted.", e);
+                }
+
+                if (page == END) {
+                    assert queue.isEmpty() : "It must be the last page: " + 
queue;
+
+                    return false; // We are done.
+                }
+
+                page.fetchNextPage();
+
+                iter = page.response().rows().iterator();
+
+                assert iter.hasNext();
+
+                return true;
+            }
+
+            @Override public Row next() {
+                return new Row(iter.next(), 0);
+            }
+
+            @Override public void remove() {
+                throw new UnsupportedOperationException();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java
new file mode 100644
index 0000000..f10aec0
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeTable.java
@@ -0,0 +1,178 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep;
+
+import org.h2.api.*;
+import org.h2.command.ddl.*;
+import org.h2.engine.*;
+import org.h2.index.*;
+import org.h2.message.*;
+import org.h2.result.*;
+import org.h2.table.*;
+
+import java.util.*;
+
+/**
+ * Merge table for distributed queries.
+ */
+public class GridMergeTable extends TableBase {
+    /** */
+    private final ArrayList<Index> idxs = new ArrayList<>(1);
+
+    /** */
+    private final GridMergeIndex idx;
+
+    /**
+     * @param data Data.
+     */
+    public GridMergeTable(CreateTableData data) {
+        super(data);
+
+        idx = new GridMergeIndexUnsorted(this, "merge_scan");
+
+        idxs.add(idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void lock(Session session, boolean exclusive, boolean 
force) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(Session ses) {
+        idx.close(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unlock(Session s) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index addIndex(Session session, String indexName, int 
indexId, IndexColumn[] cols,
+        IndexType indexType, boolean create, String indexComment) {
+        throw DbException.getUnsupportedException("addIndex");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeRow(Session session, Row row) {
+        throw DbException.getUnsupportedException("removeRow");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void truncate(Session session) {
+        throw DbException.getUnsupportedException("truncate");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addRow(Session session, Row row) {
+        throw DbException.getUnsupportedException("addRow");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkSupportAlter() {
+        throw DbException.getUnsupportedException("alter");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTableType() {
+        return EXTERNAL_TABLE_ENGINE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridMergeIndex getScanIndex(Session session) {
+        return idx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index getUniqueIndex() {
+        return null; // We don't have a PK.
+    }
+
+    /** {@inheritDoc} */
+    @Override public ArrayList<Index> getIndexes() {
+        return idxs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isLockedExclusively() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getMaxDataModificationId() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDeterministic() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canGetRowCount() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canDrop() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCount(Session ses) {
+        return idx.getRowCount(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCountApproximation() {
+        return idx.getRowCountApproximation();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getDiskSpaceUsed() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkRename() {
+        throw DbException.getUnsupportedException("rename");
+    }
+
+    /**
+     * Engine.
+     */
+    public static class Engine implements TableEngine {
+        /** */
+        private static ThreadLocal<GridMergeTable> createdTbl = new 
ThreadLocal<>();
+
+        /**
+         * @return Created table.
+         */
+        public static GridMergeTable getCreated() {
+            GridMergeTable tbl = createdTbl.get();
+
+            assert tbl != null;
+
+            createdTbl.remove();
+
+            return tbl;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Table createTable(CreateTableData data) {
+            GridMergeTable tbl = new GridMergeTable(data);
+
+            createdTbl.set(tbl);
+
+            return tbl;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
new file mode 100644
index 0000000..67c464c
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -0,0 +1,242 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.query.h2.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.kernal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.query.h2.twostep.messages.*;
+import org.jdk8.backport.*;
+
+import java.sql.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Reduce query executor.
+ */
+public class GridReduceQueryExecutor {
+    /** */
+    private GridKernalContext ctx;
+
+    /** */
+    private IgniteH2Indexing h2;
+
+    /** */
+    private IgniteLogger log;
+
+    /** */
+    private final AtomicLong reqIdGen = new AtomicLong();
+
+    /** */
+    private final ConcurrentMap<Long, QueryRun> runs = new 
ConcurrentHashMap8<>();
+
+    /**
+     * @param ctx Context.
+     * @param h2 H2 Indexing.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void start(final GridKernalContext ctx, IgniteH2Indexing h2) throws 
IgniteCheckedException {
+        this.ctx = ctx;
+        this.h2 = h2;
+
+        log = ctx.log(GridReduceQueryExecutor.class);
+
+        // TODO handle node failure.
+
+        ctx.io().addUserMessageListener(GridTopic.TOPIC_QUERY, new 
IgniteBiPredicate<UUID, Object>() {
+            @Override public boolean apply(UUID nodeId, Object msg) {
+                assert msg != null;
+
+                ClusterNode node = ctx.discovery().node(nodeId);
+
+                if (msg instanceof GridNextPageResponse)
+                    onNextPage(node, (GridNextPageResponse)msg);
+                else if (msg instanceof GridQueryFailResponse)
+                    onFail(node, (GridQueryFailResponse)msg);
+
+                return true;
+            }
+        });
+    }
+
+    private void onFail(ClusterNode node, GridQueryFailResponse msg) {
+        U.error(log, "Failed to execute query.", msg.error());
+    }
+
+    private void onNextPage(final ClusterNode node, GridNextPageResponse msg) {
+        final long qryReqId = msg.queryRequestId();
+        final int qry = msg.query();
+        final int pageSize = msg.rows().size();
+
+        QueryRun r = runs.get(qryReqId);
+
+        GridMergeIndex idx = r.tbls.get(msg.query()).getScanIndex(null);
+
+        if (msg.allRows() != -1) { // Only the first page contains row count.
+            idx.addCount(msg.allRows());
+
+            r.latch.countDown();
+        }
+
+        idx.addPage(new GridResultPage<UUID>(node.id(), msg) {
+            @Override public void fetchNextPage() {
+                try {
+                    ctx.io().sendUserMessage(F.asList(node), new 
GridNextPageRequest(qryReqId, qry, pageSize));
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param space Space name.
+     * @param qry Query.
+     * @return Future.
+     */
+    public IgniteFuture<GridCacheSqlResult> query(String space, 
GridCacheTwoStepQuery qry) {
+        long qryReqId = reqIdGen.incrementAndGet();
+
+        QueryRun r = new QueryRun();
+
+        r.tbls = new ArrayList<>(qry.mapQueries().size());
+
+        try {
+            r.conn = h2.connectionForSpace(space);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFutureEx<>(e);
+        }
+
+        Collection<ClusterNode> nodes = ctx.grid().cluster().nodes(); // TODO 
filter nodes somehow?
+
+        for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
+            GridMergeTable tbl;
+
+            try {
+                tbl = createTable(r.conn, mapQry);
+            }
+            catch (IgniteCheckedException e) {
+                return new GridFinishedFutureEx<>(e);
+            }
+
+            tbl.getScanIndex(null).setNumberOfSources(nodes.size());
+
+            r.tbls.add(tbl);
+        }
+
+        r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
+
+        this.runs.put(qryReqId, r);
+
+        try {
+            ctx.io().sendUserMessage(nodes, new GridQueryRequest(qryReqId, 
1000, qry.mapQueries()), // TODO conf page size
+                GridTopic.TOPIC_QUERY, false, 0);
+
+            r.latch.await();
+
+            GridCacheSqlQuery rdc = qry.reduceQuery();
+
+            final ResultSet res = h2.executeSqlQueryWithTimer(r.conn, 
rdc.query(), F.asList(rdc.parameters()));
+
+            for (GridMergeTable tbl : r.tbls)
+                dropTable(r.conn, tbl.getName());
+
+            return new GridFinishedFuture(ctx, new Iter(res));
+        }
+        catch (IgniteCheckedException | InterruptedException | SQLException e) 
{
+            U.closeQuiet(r.conn);
+
+            return new GridFinishedFuture<>(ctx, e);
+        }
+    }
+
+    /**
+     * @param conn Connection.
+     * @param tblName Table name.
+     * @throws SQLException If failed.
+     */
+    private void dropTable(Connection conn, String tblName) throws 
SQLException {
+        try (Statement s = conn.createStatement()) {
+            s.execute("DROP TABLE " + tblName);
+        }
+    }
+
+    /**
+     * @param conn Connection.
+     * @param qry Query.
+     * @return Table.
+     * @throws IgniteCheckedException If failed.
+     */
+    private GridMergeTable createTable(Connection conn, GridCacheSqlQuery qry) 
throws IgniteCheckedException {
+        try {
+            try (PreparedStatement s = conn.prepareStatement(
+                "CREATE LOCAL TEMPORARY TABLE " + qry.alias() +
+                " ENGINE \"" + GridMergeTable.Engine.class.getName() + "\" " +
+                " AS SELECT * FROM (" + qry.query() + ") WHERE FALSE")) {
+                h2.bindParameters(s, F.asList(qry.parameters()));
+
+                s.execute();
+            }
+
+            return GridMergeTable.Engine.getCreated();
+        }
+        catch (SQLException e) {
+            U.closeQuiet(conn);
+
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class QueryRun {
+        /** */
+        private List<GridMergeTable> tbls;
+
+        /** */
+        private CountDownLatch latch;
+
+        /** */
+        private Connection conn;
+    }
+
+    /**
+     *
+     */
+    private static class Iter extends GridH2ResultSetIterator<List<?>> 
implements GridCacheSqlResult {
+        /**
+         * @param data Data array.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected Iter(ResultSet data) throws IgniteCheckedException {
+            super(data);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected List<?> createRow() {
+            ArrayList<Object> res = new ArrayList<>(row.length);
+
+            Collections.addAll(res, row);
+
+            return res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
new file mode 100644
index 0000000..22a5dd9
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
@@ -0,0 +1,59 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.processors.query.h2.twostep.messages.*;
+
+/**
+ * Page result.
+ */
+public class GridResultPage<Z> {
+    /** */
+    private final Z src;
+
+    /** */
+    private final GridNextPageResponse res;
+
+    /**
+     * @param src Source.
+     * @param res Response.
+     */
+    protected GridResultPage(Z src, GridNextPageResponse res) {
+        this.src = src;
+        this.res = res;
+    }
+
+    /**
+     * @return Result source.
+     */
+    public Z source() {
+        return src;
+    }
+
+    /**
+     * @return Response.
+     */
+    public GridNextPageResponse response() {
+        return res;
+    }
+
+    /**
+     * Request next page.
+     */
+    public void fetchNextPage() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridResultPage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java
new file mode 100644
index 0000000..49592ca
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageRequest.java
@@ -0,0 +1,59 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep.messages;
+
+
+import java.io.*;
+
+/**
+ * Request to fetch next page.
+ */
+public class GridNextPageRequest implements Serializable {
+    /** */
+    private long qryReqId;
+
+    /** */
+    private int qry;
+
+    /** */
+    private int pageSize;
+
+    /**
+     * @param qryReqId Query request ID.
+     * @param qry Query.
+     * @param pageSize Page size.
+     */
+    public GridNextPageRequest(long qryReqId, int qry, int pageSize) {
+        this.qryReqId = qryReqId;
+        this.qry = qry;
+        this.pageSize = pageSize;
+    }
+
+    /**
+     * @return Query request ID.
+     */
+    public long queryRequestId() {
+        return qryReqId;
+    }
+
+    /**
+     * @return Query.
+     */
+    public int query() {
+        return qry;
+    }
+
+    /**
+     * @return Page size.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java
new file mode 100644
index 0000000..fce08e1
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridNextPageResponse.java
@@ -0,0 +1,180 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep.messages;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.h2.store.*;
+import org.h2.value.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Next page response.
+ */
+public class GridNextPageResponse implements Externalizable {
+    /** */
+    private long qryReqId;
+
+    /** */
+    private int qry;
+
+    /** */
+    private int page;
+
+    /** */
+    private int allRows;
+
+    /** */
+    private Collection<Value[]> rows;
+
+    /** */
+    private boolean last;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public GridNextPageResponse() {
+        // No-op.
+    }
+
+    /**
+     * @param qryReqId Query request ID.
+     * @param qry Query.
+     * @param page Page.
+     * @param allRows All rows count.
+     * @param last Last row.
+     * @param rows Rows.
+     */
+    public GridNextPageResponse(long qryReqId, int qry, int page, int allRows, 
boolean last, Collection<Value[]> rows) {
+        assert rows != null;
+
+        this.qryReqId = qryReqId;
+        this.qry = qry;
+        this.page = page;
+        this.allRows = allRows;
+        this.last = last;
+        this.rows = rows;
+    }
+
+    /**
+     * @return Query request ID.
+     */
+    public long queryRequestId() {
+        return qryReqId;
+    }
+
+    /**
+     * @return Query.
+     */
+    public int query() {
+        return qry;
+    }
+
+    /**
+     * @return Page.
+     */
+    public int page() {
+        return page;
+    }
+
+    /**
+     * @return All rows.
+     */
+    public int allRows() {
+        return allRows;
+    }
+
+    /**
+     * @return {@code true} If this is the last page.
+     */
+    public boolean isLast() {
+        return last;
+    }
+
+    /**
+     * @return Rows.
+     */
+    public Collection<Value[]> rows() {
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(qryReqId);
+        out.writeInt(qry);
+        out.writeInt(page);
+        out.writeBoolean(last);
+        out.writeInt(allRows);
+
+        out.writeInt(rows.size());
+
+        if (rows.isEmpty())
+            return;
+
+        Data data = Data.create(null, 512);
+
+        boolean first = true;
+
+        for (Value[] row : rows) {
+            if (first) {
+                out.writeInt(row.length);
+
+                first = false;
+            }
+
+            for (Value val : row)
+                data.writeValue(val);
+        }
+
+        out.writeInt(data.length());
+        out.write(data.getBytes(), 0, data.length());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, 
ClassNotFoundException {
+        qryReqId = in.readLong();
+        qry = in.readInt();
+        page = in.readInt();
+        last = in.readBoolean();
+        allRows = in.readInt();
+
+        int rowCnt = in.readInt();
+
+        if (rowCnt == 0)
+            rows = Collections.emptyList();
+        else {
+            rows = new ArrayList<>(rowCnt);
+
+            int cols = in.readInt();
+            int dataSize = in.readInt();
+
+            byte[] dataBytes = new byte[dataSize];
+
+            in.readFully(dataBytes);
+
+            Data data = Data.create(null, dataBytes);
+
+            for (int r = 0; r < rowCnt; r++) {
+                Value[] row = new Value[cols];
+
+                for (int c = 0; c < cols; c++)
+                    row[c] = data.readValue();
+
+                rows.add(row);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridNextPageResponse.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java
new file mode 100644
index 0000000..8d9fc89
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryAck.java
@@ -0,0 +1,34 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep.messages;
+
+import java.io.*;
+
+/**
+ * TODO write doc
+ */
+public class GridQueryAck implements Serializable {
+    /** */
+    private long reqId;
+
+    /**
+     * @param reqId Request ID.
+     */
+    public GridQueryAck(long reqId) {
+        this.reqId = reqId;
+    }
+
+    /**
+     * @return Request ID.
+     */
+    public long requestId() {
+        return reqId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
new file mode 100644
index 0000000..a0eb9cd
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
@@ -0,0 +1,46 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep.messages;
+
+import java.io.*;
+
+/**
+ * Error message.
+ */
+public class GridQueryFailResponse implements Serializable {
+    /** */
+    private long qryReqId;
+
+    /** */
+    private Throwable err;
+
+    /**
+     * @param qryReqId Query request ID.
+     * @param err Error.
+     */
+    public GridQueryFailResponse(long qryReqId, Throwable err) {
+        this.qryReqId = qryReqId;
+        this.err = err;
+    }
+
+    /**
+     * @return Query request ID.
+     */
+    public long queryRequestId() {
+        return qryReqId;
+    }
+
+    /**
+     * @return Error.
+     */
+    public Throwable error() {
+        return err;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
new file mode 100644
index 0000000..c37158a
--- /dev/null
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
@@ -0,0 +1,61 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.query.h2.twostep.messages;
+
+import org.gridgain.grid.kernal.processors.cache.query.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Query request.
+ */
+public class GridQueryRequest implements Serializable {
+    /** */
+    private long reqId;
+
+    /** */
+    private int pageSize;
+
+    /** */
+    private Collection<GridCacheSqlQuery> qrys;
+
+    /**
+     * @param reqId Request ID.
+     * @param pageSize Page size.
+     * @param qrys Queries.
+     */
+    public GridQueryRequest(long reqId, int pageSize, 
Collection<GridCacheSqlQuery> qrys) {
+        this.reqId = reqId;
+        this.pageSize = pageSize;
+        this.qrys = qrys;
+    }
+
+    /**
+     * @return Request ID.
+     */
+    public long requestId() {
+        return reqId;
+    }
+
+    /**
+     * @return Page size.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     * @return Queries.
+     */
+    public Collection<GridCacheSqlQuery> queries() {
+        return qrys;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java
deleted file mode 100644
index c9aeee9..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.h2.util.*;
-
-import static 
org.gridgain.grid.kernal.processors.query.h2.sql.GridSqlFunctionType.*;
-
-/**
- * Aggregate function.
- */
-public class GridSqlAggregateFunction extends GridSqlFunction {
-    /** */
-    private static final GridSqlFunctionType[] TYPE_INDEX = new 
GridSqlFunctionType[]{
-        COUNT_ALL, COUNT, GROUP_CONCAT, SUM, MIN, MAX, AVG,
-//        STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, BOOL_OR, BOOL_AND, 
SELECTIVITY, HISTOGRAM,
-    };
-
-    /** */
-    private final boolean distinct;
-
-    /**
-     * @param distinct Distinct.
-     * @param type Type.
-     */
-    public GridSqlAggregateFunction(boolean distinct, GridSqlFunctionType 
type) {
-        super(type);
-
-        this.distinct = distinct;
-    }
-
-    /**
-     * @param distinct Distinct.
-     * @param typeId Type.
-     */
-    public GridSqlAggregateFunction(boolean distinct, int typeId) {
-        this(distinct, TYPE_INDEX[typeId]);
-    }
-
-    /**
-     * @return Distinct.
-     */
-    public boolean distinct() {
-        return distinct;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        String text;
-
-        switch (type) {
-            case GROUP_CONCAT:
-                throw new UnsupportedOperationException();
-
-            case COUNT_ALL:
-                return "COUNT(*)";
-
-            default:
-                text = type.name();
-
-                break;
-        }
-
-        if (distinct)
-            return text + "(DISTINCT " + child().getSQL() + ")";
-
-        return text + StringUtils.enclose(child().getSQL());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java
deleted file mode 100644
index 5b00fb1..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.h2.command.*;
-
-/**
- * Alias for column or table.
- */
-public class GridSqlAlias extends GridSqlElement {
-    /** */
-    private final String alias;
-
-    /** */
-    private final boolean useAs;
-
-    /**
-     * @param alias Alias.
-     * @param expr Expr.
-     */
-    public GridSqlAlias(String alias, GridSqlElement expr) {
-        this(alias, expr, false);
-    }
-
-    /**
-     * @param alias Alias.
-     * @param expr Expr.
-     * @param useAs Use 'AS' keyword.
-     */
-    public GridSqlAlias(String alias, GridSqlElement expr, boolean useAs) {
-        addChild(expr);
-
-        this.useAs = useAs;
-        this.alias = alias;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        return child().getSQL() + (useAs ? " AS " : " ") + 
Parser.quoteIdentifier(alias);
-    }
-
-    /**
-     * @return Alias.
-     */
-    public String alias() {
-        return alias;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java
deleted file mode 100644
index 460ce1c..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-/**
- * Column.
- */
-public class GridSqlColumn extends GridSqlElement implements GridSqlValue {
-    /** */
-    private final GridSqlElement expressionInFrom;
-
-    /** */
-    private final String colName;
-
-    /** SQL from original query. May be qualified or unqualified column name. 
*/
-    private final String sqlText;
-
-    /**
-     * @param from From.
-     * @param name Name.
-     * @param sqlText Text.
-     */
-    public GridSqlColumn(GridSqlElement from, String name, String sqlText) {
-        assert sqlText != null;
-
-        expressionInFrom = from;
-        colName = name;
-        this.sqlText = sqlText;
-    }
-
-
-    /**
-     * @return Column name.
-     */
-    public String columnName() {
-        return colName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        return sqlText;
-    }
-
-    /**
-     * @return Expression in from.
-     */
-    public GridSqlElement expressionInFrom() {
-        return expressionInFrom;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java
deleted file mode 100644
index 01eff8c..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.h2.value.*;
-
-/**
- * Constant value.
- */
-public class GridSqlConst extends GridSqlElement implements GridSqlValue {
-    /** */
-    private final Value val;
-
-    /**
-     * @param val Value.
-     */
-    public GridSqlConst(Value val) {
-        this.val = val;
-    }
-
-    /**
-     * @return Value.
-     */
-    public Value value() {
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        return val.getSQL();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java
deleted file mode 100644
index d3463fd..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import java.util.*;
-
-/**
- * Abstract SQL element.
- */
-public abstract class GridSqlElement implements Cloneable {
-    /** */
-    protected List<GridSqlElement> children = new ArrayList<>();
-
-    /** {@inheritDoc} */
-    public abstract String getSQL();
-
-    /**
-     * @return Children.
-     */
-    public List<GridSqlElement> children() {
-        return children;
-    }
-
-    /**
-     * @param expr Expr.
-     * @return {@code this}.
-     */
-    public GridSqlElement addChild(GridSqlElement expr) {
-        if (expr == null)
-            throw new NullPointerException();
-
-        children.add(expr);
-
-        return this;
-    }
-
-    /**
-     * @return First child.
-     */
-    public GridSqlElement child() {
-        return children.get(0);
-    }
-
-    /**
-     * @param idx Index.
-     * @return Child.
-     */
-    public GridSqlElement child(int idx) {
-        return children.get(idx);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"CloneCallsConstructors", 
"CloneDoesntDeclareCloneNotSupportedException"})
-    @Override protected GridSqlElement clone() {
-        try {
-            GridSqlElement res = (GridSqlElement)super.clone();
-
-            res.children = new ArrayList<>(children);
-
-            return res;
-        }
-        catch (CloneNotSupportedException e) {
-            throw new IllegalStateException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java
deleted file mode 100644
index fff0090..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.apache.ignite.internal.util.typedef.*;
-import org.h2.util.*;
-import org.h2.value.*;
-
-import java.util.*;
-
-import static 
org.gridgain.grid.kernal.processors.query.h2.sql.GridSqlFunctionType.*;
-
-/**
- * Function.
- */
-public class GridSqlFunction extends GridSqlElement {
-    /** */
-    private static final Map<String, GridSqlFunctionType> TYPE_MAP = new 
HashMap<>();
-
-    /**
-     *
-     */
-    static {
-        for (GridSqlFunctionType type : GridSqlFunctionType.values())
-            TYPE_MAP.put(type.name(), type);
-    }
-
-    /** */
-    private final String name;
-
-    /** */
-    protected final GridSqlFunctionType type;
-
-    /**  */
-    private String castType;
-
-    /**
-     * @param type Function type.
-     */
-    public GridSqlFunction(GridSqlFunctionType type) {
-        this(type, type.functionName());
-    }
-
-    /**
-     * @param type Type.
-     * @param name Name.
-     */
-    private GridSqlFunction(GridSqlFunctionType type, String name) {
-        if (name == null)
-            throw new NullPointerException();
-
-        if (type == null)
-            type = UNKNOWN_FUNCTION;
-
-        this.name = name;
-        this.type = type;
-    }
-
-    /**
-     * @param name Name.
-     */
-    public GridSqlFunction(String name) {
-        this(TYPE_MAP.get(name), name);
-    }
-
-    /**
-     * @param castType Type for {@link GridSqlFunctionType#CAST} function.
-     * @return {@code this}.
-     */
-    public GridSqlFunction setCastType(String castType) {
-        this.castType = castType;
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        StatementBuilder buff = new StatementBuilder(name);
-
-        if (type == CASE) {
-            if (!children.isEmpty())
-                buff.append(" ").append(child().getSQL());
-
-            for (int i = 1, len = children.size() - 1; i < len; i += 2) {
-                buff.append(" WHEN ").append(child(i).getSQL());
-                buff.append(" THEN ").append(child(i + 1).getSQL());
-            }
-            if (children.size() % 2 == 0)
-                buff.append(" ELSE ").append(child(children.size() - 
1).getSQL());
-
-            return buff.append(" END").toString();
-        }
-
-        buff.append('(');
-
-        if (type == CAST) {
-            assert !F.isEmpty(castType) : castType;
-            assert children().size() == 1;
-
-            buff.append(child().getSQL()).append(" AS ").append(castType);
-        }
-        else if (type == CONVERT) {
-            assert !F.isEmpty(castType) : castType;
-            assert children().size() == 1;
-
-            buff.append(child().getSQL()).append(',').append(castType);
-        }
-        else if (type == GridSqlFunctionType.EXTRACT) {
-            ValueString v = (ValueString)((GridSqlConst)child(0)).value();
-            buff.append(v.getString()).append(" FROM 
").append(child(1).getSQL());
-        }
-        else {
-            for (GridSqlElement e : children) {
-                buff.appendExceptFirst(", ");
-                buff.append(e.getSQL());
-            }
-        }
-
-        return buff.append(')').toString();
-    }
-
-    /**
-     * @return Name.
-     */
-    public String name() {
-        return name;
-    }
-
-    /**
-     * @return Type.
-     */
-    public GridSqlFunctionType type() {
-        return type;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java
deleted file mode 100644
index be4d90d..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.h2.expression.*;
-
-/**
- * Full list of available functions see at {@link Function}
- */
-public enum GridSqlFunctionType {
-    // Aggregate functions.
-    /** */
-    COUNT_ALL("COUNT(*)"),
-
-    /** */
-    COUNT,
-
-    /** */
-    SUM,
-
-    /** */
-    MIN,
-
-    /** */
-    MAX,
-
-    /** */
-    AVG,
-
-    /** */
-    GROUP_CONCAT,
-
-    // Functions with special handling.
-    /** */
-    CASE,
-
-    /** */
-    CAST,
-
-    /** */
-    CONVERT,
-
-    /** */
-    EXTRACT,
-
-    /** Constant for all other functions. */
-    UNKNOWN_FUNCTION;
-
-    /** */
-    private final String name;
-
-    /**
-     */
-    GridSqlFunctionType() {
-        name = name();
-    }
-
-    /**
-     * @param name Name.
-     */
-    GridSqlFunctionType(String name) {
-        this.name = name;
-    }
-
-    /**
-     * @return Function name.
-     */
-    public String functionName() {
-        return name;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java
deleted file mode 100644
index a05f973..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.h2.util.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Join of two tables or subqueries.
- */
-public class GridSqlJoin extends GridSqlElement {
-    /**
-     * @param leftTbl Left table.
-     * @param rightTbl Right table.
-     */
-    public GridSqlJoin(GridSqlElement leftTbl, GridSqlElement rightTbl) {
-        addChild(leftTbl);
-        addChild(rightTbl);
-    }
-
-    /**
-     * @return Table 1.
-     */
-    public GridSqlElement leftTable() {
-        return child(0);
-    }
-
-    /**
-     * @return Table 2.
-     */
-    public GridSqlElement rightTable() {
-        return child(1);
-    }
-
-    /**
-     * @return {@code ON} Condition.
-     */
-    @Nullable public GridSqlElement on() {
-        return child(2);
-    }
-
-    /**
-     * @return {@code true} If it is a {@code LEFT JOIN}.
-     */
-    public boolean leftJoin() {
-        return false; // TODO
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        StatementBuilder buff = new StatementBuilder();
-
-        buff.append(leftTable().getSQL());
-
-        buff.append(leftJoin() ? " \n LEFT JOIN " : " \n INNER JOIN ");
-
-        buff.append(rightTable().getSQL());
-
-        return buff.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java
deleted file mode 100644
index 4477f0c..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-/**
- * Unary or binary operation.
- */
-public class GridSqlOperation extends GridSqlElement {
-    /** */
-    private final GridSqlOperationType opType;
-
-    /**
-     * @param opType Operation type.
-     */
-    public GridSqlOperation(GridSqlOperationType opType) {
-        this.opType = opType;
-    }
-
-    /**
-     * @param opType Op type.
-     * @param arg argument.
-     */
-    public GridSqlOperation(GridSqlOperationType opType, GridSqlElement arg) {
-        this(opType);
-
-        addChild(arg);
-    }
-
-    /**
-     * @param opType Op type.
-     * @param left Left.
-     * @param right Right.
-     */
-    public GridSqlOperation(GridSqlOperationType opType, GridSqlElement left, 
GridSqlElement right) {
-        this(opType);
-
-        addChild(left);
-        addChild(right);
-    }
-
-    /**
-     * @return Left.
-     */
-    public GridSqlElement left() {
-        return child(0);
-    }
-
-    /**
-     * @return Right.
-     */
-    public GridSqlElement right() {
-        return child(1);
-    }
-
-    /**
-     * @return Operation type.
-     */
-    public GridSqlOperationType opType() {
-        return opType;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        return opType.toSql(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java
deleted file mode 100644
index 35fe568..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.h2.util.*;
-
-/**
- * Operation type.
- */
-public enum GridSqlOperationType {
-    // from org.h2.expression.Operation
-    CONCAT(2, new BiExpressionSqlGenerator("||")),
-    PLUS(2, new BiExpressionSqlGenerator("+")),
-    MINUS(2, new BiExpressionSqlGenerator("-")),
-    MULTIPLY(2, new BiExpressionSqlGenerator("*")),
-    DIVIDE(2, new BiExpressionSqlGenerator("/")),
-    MODULUS(2, new BiExpressionSqlGenerator("%")),
-    NEGATE(1, new PrefixSqlGenerator("-")),
-
-    // from org.h2.expression.Comparison
-    EQUAL(2, new BiExpressionSqlGenerator("=")),
-    EQUAL_NULL_SAFE(2, new BiExpressionSqlGenerator("IS")),
-    BIGGER_EQUAL(2, new BiExpressionSqlGenerator(">=")),
-    BIGGER(2, new BiExpressionSqlGenerator(">")),
-    SMALLER_EQUAL(2, new BiExpressionSqlGenerator("<=")),
-    SMALLER(2, new BiExpressionSqlGenerator("<")),
-    NOT_EQUAL(2, new BiExpressionSqlGenerator("<>")),
-    NOT_EQUAL_NULL_SAFE(2, new BiExpressionSqlGenerator("IS NOT")),
-
-    SPATIAL_INTERSECTS(2, new IntersectsSqlGenerator()),
-    IS_NULL(1, new SuffixSqlGenerator("IS NULL")),
-    IS_NOT_NULL(1, new SuffixSqlGenerator("IS NOT NULL")),
-
-    NOT(1, new PrefixSqlGenerator("NOT")),
-
-    // from org.h2.expression.ConditionAndOr
-    AND(2, new BiExpressionSqlGenerator("AND")),
-    OR(2, new BiExpressionSqlGenerator("OR")),
-
-    // from
-    REGEXP(2, new BiExpressionSqlGenerator("REGEXP")),
-    LIKE(2, new BiExpressionSqlGenerator("LIKE")),
-
-    IN(-1, new ConditionInSqlGenerator()),
-
-    ;
-    /** */
-    private final SqlGenerator sqlGenerator;
-
-    /** */
-    private final int childrenCnt;
-
-    /**
-     * @param childrenCnt Children count.
-     * @param sqlGenerator sqlGenerator.
-     */
-    GridSqlOperationType(int childrenCnt, SqlGenerator sqlGenerator) {
-        this.childrenCnt = childrenCnt;
-        this.sqlGenerator = sqlGenerator;
-    }
-
-    /**
-     * @param operation Operation.
-     */
-    public String toSql(GridSqlOperation operation) {
-        return sqlGenerator.getSql(operation);
-    }
-
-    /**
-     * @return Children count.
-     */
-    public int childrenCount() {
-        return childrenCnt;
-    }
-
-    /**
-     *
-     */
-    private static interface SqlGenerator {
-
-        /**
-         * @param operation Operation expression.
-         */
-        public String getSql(GridSqlOperation operation);
-    }
-
-    /**
-     *
-     */
-    private static class BiExpressionSqlGenerator implements SqlGenerator {
-
-        /** */
-        private final String delim;
-
-        /**
-         * @param delim Delimiter.
-         */
-        private BiExpressionSqlGenerator(String delim) {
-            this.delim = delim;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getSql(GridSqlOperation operation) {
-            assert operation.opType().childrenCnt == 2;
-
-            return '(' + operation.child(0).getSQL() + " " + delim + " " + 
operation.child(1).getSQL() + ')';
-        }
-    }
-
-    /**
-     *
-     */
-    private static class IntersectsSqlGenerator implements SqlGenerator {
-
-        /** {@inheritDoc} */
-        @Override public String getSql(GridSqlOperation operation) {
-            assert operation.opType().childrenCnt == 2;
-
-            return "(INTERSECTS(" + operation.child(0) + ", " + 
operation.child(1) + "))";
-        }
-    }
-
-    /**
-     *
-     */
-    private static class PrefixSqlGenerator implements SqlGenerator {
-        /** */
-        private final String text;
-
-        /**
-         * @param text Text.
-         */
-        private PrefixSqlGenerator(String text) {
-            this.text = text;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getSql(GridSqlOperation operation) {
-            assert operation.opType().childrenCnt == 1;
-
-            return '(' + text + ' ' + operation.child().getSQL() + ')';
-        }
-    }
-
-    /**
-     *
-     */
-    private static class SuffixSqlGenerator implements SqlGenerator {
-        /** */
-        private final String text;
-
-        /**
-         * @param text Text.
-         */
-        private SuffixSqlGenerator(String text) {
-            this.text = text;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getSql(GridSqlOperation operation) {
-            assert operation.opType().childrenCnt == 1;
-
-            return '(' + operation.child().getSQL() + ' ' + text + ')';
-        }
-    }
-
-    /**
-     *
-     */
-    private static class ConditionInSqlGenerator implements SqlGenerator {
-
-        /** {@inheritDoc} */
-        @Override public String getSql(GridSqlOperation operation) {
-            StatementBuilder buff = new StatementBuilder("(");
-
-            buff.append(operation.child(0).getSQL()).append(" IN(");
-
-            assert operation.children().size() > 1;
-
-            if (operation.children().size() == 2) {
-                String child = operation.child(1).getSQL();
-
-                buff.append(' ').append(StringUtils.unEnclose(child)).append(' 
');
-            }
-            else {
-                for (int i = 1; i < operation.children().size(); i++) {
-                    buff.appendExceptFirst(", ");
-                    buff.append(operation.child(i).getSQL());
-                }
-            }
-
-            return buff.append("))").toString();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java
deleted file mode 100644
index 0b7da9e..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-/**
- * Query parameter.
- */
-public class GridSqlParameter extends GridSqlElement implements GridSqlValue {
-    /** Index. */
-    private int idx;
-
-    /**
-     * @param idx Index.
-     */
-    public GridSqlParameter(int idx) {
-        this.idx = idx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        return "?" + (idx + 1);
-    }
-
-    /**
-     * @return Index.
-     */
-    public int index() {
-        return idx;
-    }
-
-    /**
-     * @param idx New index.
-     */
-    public void index(int idx) {
-        this.idx = idx;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2851b52b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java
 
b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java
deleted file mode 100644
index 5c7c762..0000000
--- 
a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java
+++ /dev/null
@@ -1,495 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.query.h2.sql;
-
-import org.apache.ignite.*;
-import org.h2.command.dml.*;
-import org.h2.engine.*;
-import org.h2.expression.*;
-import org.h2.jdbc.*;
-import org.h2.result.*;
-import org.h2.table.*;
-import org.jetbrains.annotations.*;
-
-import java.lang.reflect.*;
-import java.sql.*;
-import java.util.*;
-import java.util.Set;
-
-import static 
org.gridgain.grid.kernal.processors.query.h2.sql.GridSqlOperationType.*;
-
-/**
- * H2 Query parser.
- */
-@SuppressWarnings("TypeMayBeWeakened")
-public class GridSqlQueryParser {
-    /** */
-    private static final GridSqlOperationType[] OPERATION_OP_TYPES = new 
GridSqlOperationType[]{CONCAT, PLUS, MINUS, MULTIPLY, DIVIDE, null, MODULUS};
-
-    /** */
-    private static final GridSqlOperationType[] COMPARISON_TYPES = new 
GridSqlOperationType[]{EQUAL, BIGGER_EQUAL, BIGGER, SMALLER_EQUAL,
-        SMALLER, NOT_EQUAL, IS_NULL, IS_NOT_NULL,
-        null, null, null, SPATIAL_INTERSECTS /* 11 */, null, null, null, null, 
EQUAL_NULL_SAFE /* 16 */, null, null, null, null,
-        NOT_EQUAL_NULL_SAFE /* 21 */};
-
-    /** */
-    private static final Getter<Select, Expression> CONDITION = 
getter(Select.class, "condition");
-
-    /** */
-    private static final Getter<Select, int[]> GROUP_INDEXES = 
getter(Select.class, "groupIndex");
-
-    /** */
-    private static final Getter<Operation, Integer> OPERATION_TYPE = 
getter(Operation.class, "opType");
-
-    /** */
-    private static final Getter<Operation, Expression> OPERATION_LEFT = 
getter(Operation.class, "left");
-
-    /** */
-    private static final Getter<Operation, Expression> OPERATION_RIGHT = 
getter(Operation.class, "right");
-
-    /** */
-    private static final Getter<Comparison, Integer> COMPARISON_TYPE = 
getter(Comparison.class, "compareType");
-
-    /** */
-    private static final Getter<Comparison, Expression> COMPARISON_LEFT = 
getter(Comparison.class, "left");
-
-    /** */
-    private static final Getter<Comparison, Expression> COMPARISON_RIGHT = 
getter(Comparison.class, "right");
-
-    /** */
-    private static final Getter<ConditionAndOr, Integer> ANDOR_TYPE = 
getter(ConditionAndOr.class, "andOrType");
-
-    /** */
-    private static final Getter<ConditionAndOr, Expression> ANDOR_LEFT = 
getter(ConditionAndOr.class, "left");
-
-    /** */
-    private static final Getter<ConditionAndOr, Expression> ANDOR_RIGHT = 
getter(ConditionAndOr.class, "right");
-
-    /** */
-    private static final Getter<TableView, Query> VIEW_QUERY = 
getter(TableView.class, "viewQuery");
-
-    /** */
-    private static final Getter<TableFilter, String> ALIAS = 
getter(TableFilter.class, "alias");
-
-    /** */
-    private static final Getter<Select, Integer> HAVING_INDEX = 
getter(Select.class, "havingIndex");
-
-    /** */
-    private static final Getter<ConditionIn, Expression> LEFT_CI = 
getter(ConditionIn.class, "left");
-
-    /** */
-    private static final Getter<ConditionIn, List<Expression>> VALUE_LIST_CI = 
getter(ConditionIn.class, "valueList");
-
-    /** */
-    private static final Getter<ConditionInConstantSet, Expression> LEFT_CICS =
-        getter(ConditionInConstantSet.class, "left");
-
-    /** */
-    private static final Getter<ConditionInConstantSet, List<Expression>> 
VALUE_LIST_CICS =
-        getter(ConditionInConstantSet.class, "valueList");
-
-    /** */
-    private static final Getter<ConditionInSelect, Expression> LEFT_CIS = 
getter(ConditionInSelect.class, "left");
-
-    /** */
-    private static final Getter<ConditionInSelect, Boolean> ALL = 
getter(ConditionInSelect.class, "all");
-
-    /** */
-    private static final Getter<ConditionInSelect, Integer> COMPARE_TYPE = 
getter(ConditionInSelect.class,
-        "compareType");
-
-    /** */
-    private static final Getter<ConditionInSelect, Query> QUERY = 
getter(ConditionInSelect.class, "query");
-
-    /** */
-    private static final Getter<CompareLike, Expression> LEFT = 
getter(CompareLike.class, "left");
-
-    /** */
-    private static final Getter<CompareLike, Expression> RIGHT = 
getter(CompareLike.class, "right");
-
-    /** */
-    private static final Getter<CompareLike, Expression> ESCAPE = 
getter(CompareLike.class, "escape");
-
-    /** */
-    private static final Getter<CompareLike, Boolean> REGEXP_CL = 
getter(CompareLike.class, "regexp");
-
-    /** */
-    private static final Getter<Aggregate, Boolean> DISTINCT = 
getter(Aggregate.class, "distinct");
-
-    /** */
-    private static final Getter<Aggregate, Integer> TYPE = 
getter(Aggregate.class, "type");
-
-    /** */
-    private static final Getter<Aggregate, Expression> ON = 
getter(Aggregate.class, "on");
-
-    /** */
-    private final IdentityHashMap<Object, Object> h2ObjToGridObj = new 
IdentityHashMap<>();
-
-    /**
-     * @param conn Connection.
-     * @param select Select query.
-     * @return Parsed select query.
-     */
-    public static GridSqlSelect parse(Connection conn, String select) {
-        Session ses = (Session)((JdbcConnection)conn).getSession();
-
-        return new GridSqlQueryParser().parse((Select)ses.prepare(select));
-    }
-
-    /**
-     * @param filter Filter.
-     */
-    private GridSqlElement parse(TableFilter filter) {
-        GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(filter);
-
-        if (res == null) {
-            Table tbl = filter.getTable();
-
-            if (tbl instanceof TableBase)
-                res = new GridSqlTable(tbl.getSchema().getName(), 
tbl.getName());
-            else if (tbl instanceof TableView) {
-                Query qry = VIEW_QUERY.get((TableView)tbl);
-
-                assert0(qry instanceof Select, qry);
-
-                res = new GridSqlSubquery(parse((Select)qry));
-            }
-            else
-                throw new IgniteException("Unsupported query: " + filter);
-
-            String alias = ALIAS.get(filter);
-
-            if (alias != null)
-                res = new GridSqlAlias(alias, res, false);
-
-            h2ObjToGridObj.put(filter, res);
-        }
-
-        return res;
-    }
-
-    /**
-     * @param select Select.
-     */
-    public GridSqlSelect parse(Select select) {
-        GridSqlSelect res = (GridSqlSelect)h2ObjToGridObj.get(select);
-
-        if (res != null)
-            return res;
-
-        res = new GridSqlSelect();
-
-        h2ObjToGridObj.put(select, res);
-
-        res.distinct(select.isDistinct());
-
-        Expression where = CONDITION.get(select);
-        res.where(parseExpression(where));
-
-        Set<TableFilter> allFilers = new HashSet<>(select.getTopFilters());
-
-        GridSqlElement from = null;
-
-        TableFilter filter = select.getTopTableFilter();
-        do {
-            assert0(filter != null, select);
-            assert0(!filter.isJoinOuter(), select);
-            assert0(filter.getNestedJoin() == null, select);
-            assert0(filter.getJoinCondition() == null, select);
-            assert0(filter.getFilterCondition() == null, select);
-
-            GridSqlElement gridFilter = parse(filter);
-
-            from = from == null ? gridFilter : new GridSqlJoin(from, 
gridFilter);
-
-            allFilers.remove(filter);
-
-            filter = filter.getJoin();
-        }
-        while (filter != null);
-
-        res.from(from);
-
-        assert allFilers.isEmpty();
-
-        ArrayList<Expression> expressions = select.getExpressions();
-
-        for (Expression exp : expressions)
-            res.addExpression(parseExpression(exp));
-
-        int[] grpIdx = GROUP_INDEXES.get(select);
-
-        if (grpIdx != null) {
-            res.groupColumns(grpIdx);
-
-            for (int idx : grpIdx)
-                res.addGroupExpression(parseExpression(expressions.get(idx)));
-        }
-
-        assert0(select.getHaving() == null, select);
-
-        int havingIdx = HAVING_INDEX.get(select);
-
-        if (havingIdx >= 0)
-            res.having(parseExpression(expressions.get(havingIdx)));
-
-        for (int i = 0; i < select.getColumnCount(); i++)
-            res.addSelectExpression(parseExpression(expressions.get(i)));
-
-        SortOrder sortOrder = select.getSortOrder();
-
-        if (sortOrder != null) {
-            int[] indexes = sortOrder.getQueryColumnIndexes();
-            int[] sortTypes = sortOrder.getSortTypes();
-
-            for (int i = 0; i < indexes.length; i++)
-                res.addSort(parseExpression(expressions.get(indexes[i])), 
sortTypes[i]);
-        }
-
-        return res;
-    }
-
-    /**
-     * @param expression Expression.
-     */
-    private GridSqlElement parseExpression(@Nullable Expression expression) {
-        if (expression == null)
-            return null;
-
-        GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(expression);
-
-        if (res == null) {
-            res = parseExpression0(expression);
-
-            h2ObjToGridObj.put(expression, res);
-        }
-
-        return res;
-    }
-
-    /**
-     * @param expression Expression.
-     */
-    private GridSqlElement parseExpression0(Expression expression) {
-        if (expression instanceof ExpressionColumn) {
-            TableFilter tblFilter = 
((ExpressionColumn)expression).getTableFilter();
-
-            GridSqlElement gridTblFilter = parse(tblFilter);
-
-            return new GridSqlColumn(gridTblFilter, 
expression.getColumnName(), expression.getSQL());
-        }
-
-        if (expression instanceof Alias)
-            return new GridSqlAlias(expression.getAlias(), 
parseExpression(expression.getNonAliasExpression()), true);
-
-        if (expression instanceof ValueExpression)
-            return new GridSqlConst(expression.getValue(null));
-
-        if (expression instanceof Operation) {
-            Operation operation = (Operation)expression;
-
-            Integer type = OPERATION_TYPE.get(operation);
-
-            if (type == Operation.NEGATE) {
-                assert OPERATION_RIGHT.get(operation) == null;
-
-                return new GridSqlOperation(GridSqlOperationType.NEGATE, 
parseExpression(OPERATION_LEFT.get(operation)));
-            }
-
-            return new GridSqlOperation(OPERATION_OP_TYPES[type],
-                parseExpression(OPERATION_LEFT.get(operation)),
-                parseExpression(OPERATION_RIGHT.get(operation)));
-        }
-
-        if (expression instanceof Comparison) {
-            Comparison cmp = (Comparison)expression;
-
-            GridSqlOperationType opType = 
COMPARISON_TYPES[COMPARISON_TYPE.get(cmp)];
-
-            assert opType != null : COMPARISON_TYPE.get(cmp);
-
-            GridSqlElement left = parseExpression(COMPARISON_LEFT.get(cmp));
-
-            if (opType.childrenCount() == 1)
-                return new GridSqlOperation(opType, left);
-
-            GridSqlElement right = parseExpression(COMPARISON_RIGHT.get(cmp));
-
-            return new GridSqlOperation(opType, left, right);
-        }
-
-        if (expression instanceof ConditionNot)
-            return new GridSqlOperation(NOT, 
parseExpression(expression.getNotIfPossible(null)));
-
-        if (expression instanceof ConditionAndOr) {
-            ConditionAndOr andOr = (ConditionAndOr)expression;
-
-            int type = ANDOR_TYPE.get(andOr);
-
-            assert type == ConditionAndOr.AND || type == ConditionAndOr.OR;
-
-            return new GridSqlOperation(type == ConditionAndOr.AND ? AND : OR,
-                parseExpression(ANDOR_LEFT.get(andOr)), 
parseExpression(ANDOR_RIGHT.get(andOr)));
-        }
-
-        if (expression instanceof Subquery) {
-            Query qry = ((Subquery)expression).getQuery();
-
-            assert0(qry instanceof Select, expression);
-
-            return new GridSqlSubquery(parse((Select) qry));
-        }
-
-        if (expression instanceof ConditionIn) {
-            GridSqlOperation res = new GridSqlOperation(IN);
-
-            res.addChild(parseExpression(LEFT_CI.get((ConditionIn) 
expression)));
-
-            List<Expression> vals = VALUE_LIST_CI.get((ConditionIn)expression);
-
-            for (Expression val : vals)
-                res.addChild(parseExpression(val));
-
-            return res;
-        }
-
-        if (expression instanceof ConditionInConstantSet) {
-            GridSqlOperation res = new GridSqlOperation(IN);
-
-            
res.addChild(parseExpression(LEFT_CICS.get((ConditionInConstantSet) 
expression)));
-
-            List<Expression> vals = 
VALUE_LIST_CICS.get((ConditionInConstantSet)expression);
-
-            for (Expression val : vals)
-                res.addChild(parseExpression(val));
-
-            return res;
-        }
-
-        if (expression instanceof ConditionInSelect) {
-            GridSqlOperation res = new GridSqlOperation(IN);
-
-            boolean all = ALL.get((ConditionInSelect)expression);
-            int compareType = COMPARE_TYPE.get((ConditionInSelect)expression);
-
-            assert0(!all, expression);
-            assert0(compareType == Comparison.EQUAL, expression);
-
-            res.addChild(parseExpression(LEFT_CIS.get((ConditionInSelect) 
expression)));
-
-            Query qry = QUERY.get((ConditionInSelect)expression);
-
-            assert0(qry instanceof Select, qry);
-
-            res.addChild(new GridSqlSubquery(parse((Select) qry)));
-
-            return res;
-        }
-
-        if (expression instanceof CompareLike) {
-            assert0(ESCAPE.get((CompareLike)expression) == null, expression);
-
-            boolean regexp = REGEXP_CL.get((CompareLike)expression);
-
-            return new GridSqlOperation(regexp ? REGEXP : LIKE, 
parseExpression(LEFT.get((CompareLike) expression)),
-                parseExpression(RIGHT.get((CompareLike) expression)));
-        }
-
-        if (expression instanceof Function) {
-            Function f = (Function)expression;
-
-            GridSqlFunction res = new GridSqlFunction(f.getName());
-
-            for (Expression arg : f.getArgs())
-                res.addChild(parseExpression(arg));
-
-            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == 
Function.CONVERT)
-                res.setCastType(new Column(null, f.getType(), 
f.getPrecision(), f.getScale(), f.getDisplaySize())
-                    .getCreateSQL());
-
-            return res;
-        }
-
-        if (expression instanceof Parameter)
-            return new GridSqlParameter(((Parameter)expression).getIndex());
-
-        if (expression instanceof Aggregate) {
-            GridSqlAggregateFunction res = new 
GridSqlAggregateFunction(DISTINCT.get((Aggregate)expression),
-                TYPE.get((Aggregate)expression));
-
-            Expression on = ON.get((Aggregate)expression);
-
-            if (on != null)
-                res.addChild(parseExpression(on));
-
-            return res;
-        }
-
-        throw new IgniteException("Unsupported expression: " + expression + " 
[type=" +
-            expression.getClass().getSimpleName() + ']');
-    }
-
-    /**
-     * @param cond Condition.
-     * @param o Object.
-     */
-    private static void assert0(boolean cond, Object o) {
-        if (!cond)
-            throw new IgniteException("Unsupported query: " + o);
-    }
-
-    /**
-     * @param cls Class.
-     * @param fldName Fld name.
-     */
-    private static <T, R> Getter<T, R> getter(Class<T> cls, String fldName) {
-        Field field;
-
-        try {
-            field = cls.getDeclaredField(fldName);
-        }
-        catch (NoSuchFieldException e) {
-            throw new RuntimeException(e);
-        }
-
-        field.setAccessible(true);
-
-        return new Getter<>(field);
-    }
-
-    /**
-     * Field getter.
-     */
-    @SuppressWarnings("unchecked")
-    private static class Getter<T, R> {
-        /** */
-        private final Field fld;
-
-        /**
-         * @param fld Fld.
-         */
-        private Getter(Field fld) {
-            this.fld = fld;
-        }
-
-        /**
-         * @param obj Object.
-         * @return Result.
-         */
-        public R get(T obj) {
-            try {
-                return (R)fld.get(obj);
-            }
-            catch (IllegalAccessException e) {
-                throw new IgniteException(e);
-            }
-        }
-    }
-}

Reply via email to