You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2014/12/23 21:36:34 UTC

[51/53] [abbrv] incubator-ignite git commit: ignite-gg9499 - sql parser

ignite-gg9499 - sql parser


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/0497d7f0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/0497d7f0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/0497d7f0

Branch: refs/heads/ignite-gg9499
Commit: 0497d7f0b8bb28eee878bc6a883c0b6a58dda797
Parents: b16f120
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 23 23:09:09 2014 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 23 23:09:09 2014 +0300

----------------------------------------------------------------------
 .../query/h2/sql/GridAggregateFunction.java     |  70 ++++
 .../processors/query/h2/sql/GridAlias.java      |  63 ++++
 .../query/h2/sql/GridFunctionType.java          |  62 ++++
 .../processors/query/h2/sql/GridJoin.java       |  60 +++
 .../processors/query/h2/sql/GridOperation.java  |  75 ++++
 .../query/h2/sql/GridOperationType.java         | 202 ++++++++++
 .../query/h2/sql/GridQueryParser.java           | 370 +++++++++++++++++++
 .../processors/query/h2/sql/GridQueryUtils.java | 148 ++++++++
 .../processors/query/h2/sql/GridSelect.java     | 241 ++++++++++++
 .../processors/query/h2/sql/GridSqlColumn.java  |  56 +++
 .../processors/query/h2/sql/GridSqlElement.java |  68 ++++
 .../query/h2/sql/GridSqlFunction.java           | 117 ++++++
 .../query/h2/sql/GridSqlParameter.java          |  44 +++
 .../processors/query/h2/sql/GridSqlValue.java   |  17 +
 .../processors/query/h2/sql/GridSubquery.java   |  44 +++
 .../processors/query/h2/sql/GridTable.java      |  55 +++
 .../query/h2/sql/GridValueExpression.java       |  39 ++
 .../processors/query/h2/sql/GridQueryTest.java  | 248 +++++++++++++
 18 files changed, 1979 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAggregateFunction.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAggregateFunction.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAggregateFunction.java
new file mode 100644
index 0000000..dffee20
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAggregateFunction.java
@@ -0,0 +1,70 @@
+/* @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.GridFunctionType.*;
+
+/**
+ * Aggregate function.
+ */
+public class GridAggregateFunction extends GridSqlFunction {
+
+    /** */
+    private static final GridFunctionType[] TYPE_INDEX = new GridFunctionType[]{
+        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 GridAggregateFunction(boolean distinct, GridFunctionType type) {
+        super(type);
+
+        this.distinct = distinct;
+    }
+
+    /**
+     * @param distinct Distinct.
+     * @param typeId Type.
+     */
+    public GridAggregateFunction(boolean distinct, int typeId) {
+        this(distinct, TYPE_INDEX[typeId]);
+    }
+
+    /** {@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/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAlias.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAlias.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAlias.java
new file mode 100644
index 0000000..7c9a080
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAlias.java
@@ -0,0 +1,63 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.command.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Alias for column or table.
+ */
+public class GridAlias extends GridSqlElement {
+
+    /** */
+    private final String alias;
+
+    /** */
+    private final boolean useAs;
+
+    /**
+     * @param alias Alias.
+     * @param expr Expr.
+     */
+    public GridAlias(@NotNull String alias, @NotNull GridSqlElement expr) {
+        this(alias, expr, false);
+    }
+
+    /**
+     * @param alias Alias.
+     * @param expr Expr.
+     * @param useAs Use 'AS' keyword.
+     */
+    public GridAlias(@NotNull String alias, @NotNull 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.
+     */
+    @NotNull
+    public String alias() {
+        return alias;
+    }
+
+    /** {@inheritDoc} */
+    public GridSqlElement getNonAliasExpression() {
+        return child();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridFunctionType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridFunctionType.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridFunctionType.java
new file mode 100644
index 0000000..ba01dd5
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridFunctionType.java
@@ -0,0 +1,62 @@
+/* @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 GridFunctionType {
+    ABS(1), CEIL(1), FLOOR(1), COS(1), PI(0), POWER(2), RAND(-1), ROUND(1),
+    CASE(-1), CAST(1), CONVERT(1), EXTRACT(2),
+    DAY_OF_MONTH(1), DAY_OF_WEEK(1), DAY_OF_YEAR(1),
+
+    // Aggregate functions.
+    COUNT_ALL("COUNT(*)", 0), COUNT(1), GROUP_CONCAT(1), SUM(1), MIN(1), MAX(1), AVG(1), STDDEV_POP(1), STDDEV_SAMP(1),
+    VAR_POP(1), VAR_SAMP(1), BOOL_OR(1), BOOL_AND(1), SELECTIVITY(1), HISTOGRAM(1);
+
+    /** */
+    private final String name;
+
+    /** */
+    private final int argCnt;
+
+    /**
+     * @param argCnt Count.
+     */
+    GridFunctionType(int argCnt) {
+        name = name();
+        this.argCnt = argCnt;
+    }
+
+    /**
+     * @param name Name.
+     * @param argCnt Count.
+     */
+    GridFunctionType(String name, int argCnt) {
+        this.name = name;
+        this.argCnt = argCnt;
+    }
+
+    /**
+     * @return Argument count.
+     */
+    public int argumentCount() {
+        return argCnt;
+    }
+
+    /**
+     * @return Function name.
+     */
+    public String functionName() {
+        return name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridJoin.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridJoin.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridJoin.java
new file mode 100644
index 0000000..d703fe7
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridJoin.java
@@ -0,0 +1,60 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.util.*;
+
+/**
+ *
+ */
+public class GridJoin extends GridSqlElement {
+
+    /** */
+    private final GridSqlElement tbl1;
+
+    /** */
+    private final GridSqlElement tbl2;
+
+    /**
+     * @param tbl1 Table 1.
+     * @param tbl2 Table 2.
+     */
+    public GridJoin(GridSqlElement tbl1, GridSqlElement tbl2) {
+        this.tbl1 = tbl1;
+        this.tbl2 = tbl2;
+    }
+
+    /**
+     * @return Table 1.
+     */
+    public GridSqlElement table1() {
+        return tbl1;
+    }
+
+    /**
+     * @return Table 2.
+     */
+    public GridSqlElement table2() {
+        return tbl2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder();
+
+        buff.append(tbl1.getSQL());
+
+        buff.append(" \n INNER JOIN ");
+
+        buff.append(tbl2.getSQL());
+
+        return buff.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperation.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperation.java
new file mode 100644
index 0000000..c38ae85
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperation.java
@@ -0,0 +1,75 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.jetbrains.annotations.*;
+
+/**
+ *
+ */
+public class GridOperation extends GridSqlElement {
+    /** */
+    private final GridOperationType opType;
+
+    /**
+     * @param opType Operation type.
+     */
+    public GridOperation(@NotNull GridOperationType opType) {
+        this.opType = opType;
+    }
+
+    /**
+     * @param opType Op type.
+     * @param arg argument.
+     */
+    public GridOperation(@NotNull GridOperationType opType, @NotNull GridSqlElement arg) {
+        this(opType);
+
+        addChild(arg);
+    }
+
+    /**
+     * @param opType Op type.
+     * @param left Left.
+     * @param right Right.
+     */
+    public GridOperation(@NotNull GridOperationType opType, @NotNull GridSqlElement left, @NotNull 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 GridOperationType opType() {
+        return opType;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return opType.toSql(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperationType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperationType.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperationType.java
new file mode 100644
index 0000000..8c56731
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperationType.java
@@ -0,0 +1,202 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.util.*;
+
+/**
+ *
+ */
+public enum GridOperationType {
+    // 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.
+     */
+    GridOperationType(int childrenCnt, SqlGenerator sqlGenerator) {
+        this.childrenCnt = childrenCnt;
+        this.sqlGenerator = sqlGenerator;
+    }
+
+    /**
+     * @param operation Operation.
+     */
+    public String toSql(GridOperation operation) {
+        return sqlGenerator.getSql(operation);
+    }
+
+    /**
+     * @return Children count.
+     */
+    public int childrenCount() {
+        return childrenCnt;
+    }
+
+    /**
+     *
+     */
+    private static interface SqlGenerator {
+
+        /**
+         * @param operation Operation expression.
+         */
+        public String getSql(GridOperation 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(GridOperation 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(GridOperation 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(GridOperation 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(GridOperation operation) {
+            assert operation.opType().childrenCnt == 1;
+
+            return '(' + operation.child().getSQL() + ' ' + text + ')';
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ConditionInSqlGenerator implements SqlGenerator {
+
+        /** {@inheritDoc} */
+        @Override public String getSql(GridOperation 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/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryParser.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryParser.java
new file mode 100644
index 0000000..d31bee3
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryParser.java
@@ -0,0 +1,370 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.apache.ignite.*;
+import org.gridgain.grid.kernal.processors.query.h2.sql.GridQueryUtils.*;
+import org.h2.command.dml.*;
+import org.h2.expression.*;
+import org.h2.result.*;
+import org.h2.table.*;
+import org.jetbrains.annotations.*;
+
+import java.lang.reflect.*;
+import java.util.*;
+import java.util.Set;
+
+import static org.gridgain.grid.kernal.processors.query.h2.sql.GridOperationType.*;
+import static org.gridgain.grid.kernal.processors.query.h2.sql.GridQueryUtils.*;
+
+/**
+ * H2 Query parser.
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class GridQueryParser {
+    /** */
+    private static final GridOperationType[] OPERATION_OP_TYPES = new GridOperationType[]{CONCAT, PLUS, MINUS, MULTIPLY, DIVIDE, null, MODULUS};
+
+    /** */
+    private static final GridOperationType[] COMPARISON_TYPES = new GridOperationType[]{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 Field GROUP_INDEXES = GridQueryUtils.getField(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 final IdentityHashMap<Object, Object> h2ObjToGridObj = new IdentityHashMap<>();
+
+    /**
+     * @param filter Filter.
+     */
+    private GridSqlElement toGridTableFilter(TableFilter filter) {
+        GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(filter);
+
+        if (res == null) {
+            Table tbl = filter.getTable();
+
+            if (tbl instanceof TableBase)
+                res = new GridTable(tbl.getSchema().getName(), tbl.getName());
+            else if (tbl instanceof TableView) {
+                Query qry = GridQueryUtils.getFieldValue(tbl, "viewQuery");
+
+                assert0(qry instanceof Select, qry);
+
+                res = new GridSubquery(toGridSelect((Select)qry));
+            }
+            else
+                throw new IgniteException("Unsupported query: " + filter);
+
+            String alias = GridQueryUtils.getFieldValue(filter, "alias");
+
+            if (alias != null)
+                res = new GridAlias(alias, res, false);
+
+            h2ObjToGridObj.put(filter, res);
+        }
+
+        return res;
+    }
+
+    /**
+     * @param select Select.
+     */
+    public GridSelect toGridSelect(Select select) {
+        GridSelect res = (GridSelect)h2ObjToGridObj.get(select);
+
+        if (res != null)
+            return res;
+
+        res = new GridSelect();
+
+        h2ObjToGridObj.put(select, res);
+
+        res.distinct(select.isDistinct());
+
+        Expression where = CONDITION.get(select);
+        res.where(toGridExpression(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 = toGridTableFilter(filter);
+
+            from = from == null ? gridFilter : new GridJoin(from, gridFilter);
+
+            allFilers.remove(filter);
+
+            filter = filter.getJoin();
+        }
+        while (filter != null);
+
+        res.from(from);
+
+        assert allFilers.isEmpty();
+
+        ArrayList<Expression> expressions = select.getExpressions();
+
+        int[] grpIdx = GridQueryUtils.getFieldValue(GROUP_INDEXES, select);
+
+        if (grpIdx != null) {
+            for (int idx : grpIdx)
+                res.addGroupExpression(toGridExpression(expressions.get(idx)));
+        }
+
+        assert0(select.getHaving() == null, select);
+
+        int havingIdx = GridQueryUtils.getFieldValue(select, "havingIndex");
+
+        if (havingIdx >= 0)
+            res.having(toGridExpression(expressions.get(havingIdx)));
+
+        for (int i = 0; i < select.getColumnCount(); i++)
+            res.addSelectExpression(toGridExpression(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(toGridExpression(expressions.get(indexes[i])), sortTypes[i]);
+        }
+
+        return res;
+    }
+
+    /**
+     * @param expression Expression.
+     */
+    private GridSqlElement toGridExpression(@Nullable Expression expression) {
+        if (expression == null)
+            return null;
+
+        GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(expression);
+
+        if (res == null) {
+            res = toGridExpression0(expression);
+
+            h2ObjToGridObj.put(expression, res);
+        }
+
+        return res;
+    }
+
+    /**
+     * @param expression Expression.
+     */
+    @NotNull private GridSqlElement toGridExpression0(@NotNull Expression expression) {
+        if (expression instanceof ExpressionColumn) {
+            TableFilter tblFilter = ((ExpressionColumn)expression).getTableFilter();
+
+            GridSqlElement gridTblFilter = toGridTableFilter(tblFilter);
+
+            return new GridSqlColumn(gridTblFilter, expression.getColumnName(), expression.getSQL());
+        }
+
+        if (expression instanceof Alias)
+            return new GridAlias(expression.getAlias(), toGridExpression(expression.getNonAliasExpression()), true);
+
+        if (expression instanceof ValueExpression)
+            return new GridValueExpression(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 GridOperation(GridOperationType.NEGATE, toGridExpression(OPERATION_LEFT.get(operation)));
+            }
+
+            return new GridOperation(OPERATION_OP_TYPES[type],
+                toGridExpression(OPERATION_LEFT.get(operation)),
+                toGridExpression(OPERATION_RIGHT.get(operation)));
+        }
+
+        if (expression instanceof Comparison) {
+            Comparison cmp = (Comparison)expression;
+
+            GridOperationType opType = COMPARISON_TYPES[COMPARISON_TYPE.get(cmp)];
+
+            assert opType != null : COMPARISON_TYPE.get(cmp);
+
+            GridSqlElement left = toGridExpression(COMPARISON_LEFT.get(cmp));
+
+            if (opType.childrenCount() == 1)
+                return new GridOperation(opType, left);
+
+            GridSqlElement right = toGridExpression(COMPARISON_RIGHT.get(cmp));
+
+            return new GridOperation(opType, left, right);
+        }
+
+        if (expression instanceof ConditionNot)
+            return new GridOperation(NOT, toGridExpression(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 GridOperation(type == ConditionAndOr.AND ? AND : OR,
+                toGridExpression(ANDOR_LEFT.get(andOr)), toGridExpression(ANDOR_RIGHT.get(andOr)));
+        }
+
+        if (expression instanceof Subquery) {
+            Query qry = ((Subquery)expression).getQuery();
+
+            assert0(qry instanceof Select, expression);
+
+            return new GridSubquery(toGridSelect((Select)qry));
+        }
+
+        if (expression instanceof ConditionIn) {
+            GridOperation res = new GridOperation(IN);
+
+            res.addChild(toGridExpression((Expression)GridQueryUtils.getFieldValue(expression, "left")));
+
+            List<Expression> vals = GridQueryUtils.getFieldValue(expression, "valueList");
+
+            for (Expression val : vals)
+                res.addChild(toGridExpression(val));
+
+            return res;
+        }
+
+        if (expression instanceof ConditionInConstantSet) {
+            GridOperation res = new GridOperation(IN);
+
+            res.addChild(toGridExpression((Expression)GridQueryUtils.getFieldValue(expression, "left")));
+
+            List<Expression> vals = GridQueryUtils.getFieldValue(expression, "valueList");
+
+            for (Expression val : vals)
+                res.addChild(toGridExpression(val));
+
+            return res;
+        }
+
+        if (expression instanceof ConditionInSelect) {
+            GridOperation res = new GridOperation(IN);
+
+            boolean all = GridQueryUtils.getFieldValue(expression, "all");
+            int compareType = GridQueryUtils.getFieldValue(expression, "compareType");
+
+            assert0(!all, expression);
+            assert0(compareType == Comparison.EQUAL, expression);
+
+            res.addChild(toGridExpression((Expression)GridQueryUtils.getFieldValue(expression, "left")));
+
+            Query qry = GridQueryUtils.getFieldValue(expression, "query");
+
+            assert0(qry instanceof Select, qry);
+
+            res.addChild(new GridSubquery(toGridSelect((Select)qry)));
+
+            return res;
+        }
+
+        if (expression instanceof CompareLike) {
+            assert0(GridQueryUtils.getFieldValue(expression, "escape") == null, expression);
+
+            boolean regexp = GridQueryUtils.getFieldValue(expression, "regexp");
+
+            return new GridOperation(regexp ? REGEXP : LIKE,
+                toGridExpression((Expression)GridQueryUtils.getFieldValue(expression, "left")),
+                toGridExpression((Expression)GridQueryUtils.getFieldValue(expression, "right")));
+        }
+
+        if (expression instanceof Function) {
+            Function function = (Function)expression;
+
+            GridSqlFunction res = new GridSqlFunction(function.getName());
+
+            for (Expression arg : function.getArgs())
+                res.addChild(toGridExpression(arg));
+
+            return res;
+        }
+
+        if (expression instanceof Parameter)
+            return new GridSqlParameter(((Parameter)expression).getIndex());
+
+        if (expression instanceof Aggregate) {
+            GridAggregateFunction res = new GridAggregateFunction(GridQueryUtils.<Boolean>getFieldValue(expression,
+                "distinct"), GridQueryUtils.<Integer>getFieldValue(expression, "type"));
+
+            Expression on = GridQueryUtils.getFieldValue(expression, "on");
+
+            if (on != null)
+                res.addChild(toGridExpression(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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryUtils.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryUtils.java
new file mode 100644
index 0000000..0ffac88
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryUtils.java
@@ -0,0 +1,148 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.table.*;
+
+import java.lang.reflect.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class GridQueryUtils {
+
+    /**
+     * @param res
+     * @param tbl
+     */
+    public static void extractTables(Set<TableBase> res, Table tbl) {
+        if (tbl instanceof TableBase) {
+            res.add((TableBase)tbl);
+
+            return;
+        }
+
+        if (tbl instanceof TableView) {
+            List<Table> tableList = getTables((TableView)tbl);
+
+            for (Table table : tableList) {
+                extractTables(res, table);
+            }
+        }
+        else
+            throw new IllegalArgumentException("Unknown table type: " + tbl);
+    }
+
+    /**
+     * @param view Table view.
+     * @return List of tables.
+     */
+    public static List<Table> getTables(TableView view) {
+        try {
+            Field field = TableView.class.getDeclaredField("tables");
+
+            field.setAccessible(true);
+
+            return (List<Table>)field.get(view);
+        }
+        catch (NoSuchFieldException | IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     *
+     * @param fld
+     * @param obj
+     * @param <T>
+     * @return
+     */
+    public static <T> T getFieldValue(Field fld, Object obj) {
+        try {
+            return (T)fld.get(obj);
+        }
+        catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * @param cls
+     * @param fieldName
+     * @return
+     */
+    public static Field getField(Class<?> cls, String fieldName) {
+        Field field;
+        try {
+            field = cls.getDeclaredField(fieldName);
+        }
+        catch (NoSuchFieldException e) {
+            throw new RuntimeException(e);
+        }
+
+        field.setAccessible(true);
+
+        return field;
+    }
+
+    /**
+     * @param cls Class.
+     * @param obj Object.
+     * @param fldName Fld name.
+     */
+    public static <C> Object getFieldValue(Class<? super C> cls, C obj, String fldName) {
+        Field field = getField(cls, fldName);
+
+        return getFieldValue(field, obj);
+    }
+
+    /**
+     * @param obj Object.
+     * @param fldName Fld name.
+     */
+    public static <T> T getFieldValue(Object obj, String fldName) {
+        return (T)getFieldValue((Class<? super Object>)obj.getClass(), obj, fldName);
+    }
+
+    /**
+     * @param cls Class.
+     * @param fldName Fld name.
+     */
+    public static <T, R> Getter<T, R> getter(Class<T> cls, String fldName) {
+        Field field = getField(cls, fldName);
+
+        return new Getter<>(field);
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public 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) {
+            return getFieldValue(fld, obj);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSelect.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSelect.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSelect.java
new file mode 100644
index 0000000..30a50e8
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSelect.java
@@ -0,0 +1,241 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.result.*;
+import org.h2.util.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class GridSelect implements Cloneable {
+    /** */
+    private boolean distinct;
+
+    /** */
+    private List<GridSqlElement> select = new ArrayList<>();
+
+    /** */
+    private List<GridSqlElement> groups = new ArrayList<>();
+
+    /** */
+    private GridSqlElement from;
+
+    /** */
+    private GridSqlElement where;
+
+    /** */
+    private GridSqlElement having;
+
+    /** */
+    private Map<GridSqlElement, Integer> sort = new LinkedHashMap<>();
+
+    /**
+     * @return Distinct.
+     */
+    public boolean distinct() {
+        return distinct;
+    }
+
+    /**
+     * @param distinct New distinct.
+     */
+    public void distinct(boolean distinct) {
+        this.distinct = distinct;
+    }
+
+    /**
+     * @return Generate sql.
+     */
+    public String getSQL() {
+        StatementBuilder buff = new StatementBuilder("SELECT");
+
+        if (distinct)
+            buff.append(" DISTINCT");
+
+        for (GridSqlElement expression : select) {
+            buff.appendExceptFirst(",");
+            buff.append('\n');
+            buff.append(StringUtils.indent(expression.getSQL(), 4, false));
+        }
+
+        buff.append("\nFROM ").append(from.getSQL());
+
+        if (where != null)
+            buff.append("\nWHERE ").append(StringUtils.unEnclose(where.getSQL()));
+
+        if (!groups.isEmpty()) {
+            buff.append("\nGROUP BY ");
+
+            buff.resetCount();
+
+            for (GridSqlElement expression : groups) {
+                buff.appendExceptFirst(", ");
+
+                if (expression instanceof GridAlias)
+                    buff.append(StringUtils.unEnclose(((GridAlias)expression).getNonAliasExpression().getSQL()));
+                else
+                    buff.append(StringUtils.unEnclose(expression.getSQL()));
+            }
+        }
+
+        if (having != null)
+            buff.append("\nHAVING ").append(StringUtils.unEnclose(having.getSQL()));
+
+        if (!sort.isEmpty()) {
+            buff.append("\nORDER BY ");
+
+            buff.resetCount();
+
+            for (Map.Entry<GridSqlElement, Integer> entry : sort.entrySet()) {
+                buff.appendExceptFirst(", ");
+
+                GridSqlElement expression = entry.getKey();
+
+                int idx = select.indexOf(expression);
+
+                if (idx >= 0)
+                    buff.append(idx + 1);
+                else
+                    buff.append('=').append(StringUtils.unEnclose(expression.getSQL()));
+
+                int type = entry.getValue();
+
+                if ((type & SortOrder.DESCENDING) != 0)
+                    buff.append(" DESC");
+
+                if ((type & SortOrder.NULLS_FIRST) != 0)
+                    buff.append(" NULLS FIRST");
+                else if ((type & SortOrder.NULLS_LAST) != 0)
+                    buff.append(" NULLS LAST");
+            }
+        }
+
+        return buff.toString();
+    }
+
+    /**
+     * @return Expressions.
+     */
+    public List<GridSqlElement> select() {
+        return select;
+    }
+
+    /**
+     * @param expression Expression.
+     */
+    public void addSelectExpression(GridSqlElement expression) {
+        select.add(expression);
+    }
+
+    /**
+     * @return Expressions.
+     */
+    public List<GridSqlElement> groups() {
+        return groups;
+    }
+
+    /**
+     *
+     */
+    public void clearGroups() {
+        groups.clear();
+    }
+
+    /**
+     * @param expression Expression.
+     */
+    public void addGroupExpression(GridSqlElement expression) {
+        groups.add(expression);
+    }
+
+    /**
+     * @return Tables.
+     */
+    public GridSqlElement from() {
+        return from;
+    }
+
+    /**
+     * @param from From element.
+     */
+    public void from(GridSqlElement from) {
+        this.from = from;
+    }
+
+    /**
+     * @return Where.
+     */
+    public GridSqlElement where() {
+        return where;
+    }
+
+    /**
+     * @param where New where.
+     */
+    public void where(GridSqlElement where) {
+        this.where = where;
+    }
+
+    /**
+     * @return Having.
+     */
+    public GridSqlElement having() {
+        return having;
+    }
+
+    /**
+     * @param having New having.
+     */
+    public void having(GridSqlElement having) {
+        this.having = having;
+    }
+
+    /**
+     * @return Sort.
+     */
+    public Map<GridSqlElement, Integer> sort() {
+        return sort;
+    }
+
+    /**
+     *
+     */
+    public void clearSort() {
+        sort.clear();
+    }
+
+    /**
+     * @param expression Expression.
+     * @param sortType The sort type bit mask (SortOrder.DESCENDING, SortOrder.NULLS_FIRST, SortOrder.NULLS_LAST).
+     */
+    public void addSort(GridSqlElement expression, int sortType) {
+        sort.put(expression, sortType);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"CloneCallsConstructors", "CloneDoesntDeclareCloneNotSupportedException"})
+    @Override public GridSelect clone() {
+        try {
+            GridSelect res = (GridSelect)super.clone();
+
+            res.select = new ArrayList<>(select);
+            res.groups = new ArrayList<>(groups);
+            res.sort = new LinkedHashMap<>(sort);
+
+            return res;
+        }
+        catch (CloneNotSupportedException e) {
+            throw new RuntimeException(e); // Never thrown.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/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
new file mode 100644
index 0000000..046d9ca
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlColumn.java
@@ -0,0 +1,56 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+/**
+ *
+ */
+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) {
+        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/0497d7f0/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
new file mode 100644
index 0000000..f569261
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlElement.java
@@ -0,0 +1,68 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.jetbrains.annotations.*;
+
+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.
+     */
+    public void addChild(@NotNull GridSqlElement expr) {
+        children.add(expr);
+    }
+
+    /**
+     *
+     */
+    @NotNull public GridSqlElement child() {
+        return children.get(0);
+    }
+
+    /**
+     *
+     */
+    @NotNull public GridSqlElement child(int idx) {
+        return children.get(idx);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("CloneCallsConstructors")
+    @Override protected GridSqlElement clone() {
+        try {
+            GridSqlElement res = (GridSqlElement)super.clone();
+
+            res.children = new ArrayList<>(children);
+
+            return res;
+        }
+        catch (CloneNotSupportedException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/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
new file mode 100644
index 0000000..5e9e112
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunction.java
@@ -0,0 +1,117 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.util.*;
+import org.h2.value.*;
+
+import java.util.*;
+
+/**
+ * Function.
+ */
+public class GridSqlFunction extends GridSqlElement {
+    /** */
+    private static final Map<String, GridFunctionType> TYPE_MAP = new HashMap<>();
+
+    /**
+     *
+     */
+    static {
+        for (GridFunctionType type : GridFunctionType.values())
+            TYPE_MAP.put(type.name(), type);
+    }
+
+    /** */
+    private final String name;
+
+    /** */
+    protected final GridFunctionType type;
+
+    /**
+     * @param type Function type.
+     */
+    public GridSqlFunction(GridFunctionType type) {
+        name = type.functionName();
+
+        this.type = type;
+
+        if (type == GridFunctionType.CAST || type == GridFunctionType.CONVERT)
+            throw new UnsupportedOperationException();
+    }
+
+    /**
+     * @param name Name.
+     */
+    public GridSqlFunction(String name) {
+        this(TYPE_MAP.get(name));
+
+        if (type == GridFunctionType.CAST || type == GridFunctionType.CONVERT)
+            throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        StatementBuilder buff = new StatementBuilder(name);
+
+        if (type == GridFunctionType.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 == GridFunctionType.CAST) {
+            throw new UnsupportedOperationException("CAST");
+//            buff.append(child().getSQL()).append(" AS ").
+//                append(new Column(null, dataType, precision, scale, displaySize).getCreateSQL());
+        }
+        else if (type == GridFunctionType.CONVERT) {
+            throw new UnsupportedOperationException("CONVERT");
+//            buff.append(args[0].getSQL()).append(',').
+//                append(new Column(null, dataType, precision, scale, displaySize).getCreateSQL());
+        }
+        else if (type == GridFunctionType.EXTRACT) {
+            ValueString v = (ValueString) ((GridValueExpression)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 GridFunctionType type() {
+        return type;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/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
new file mode 100644
index 0000000..0b7da9e
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlParameter.java
@@ -0,0 +1,44 @@
+/* @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/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlValue.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlValue.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlValue.java
new file mode 100644
index 0000000..62765e7
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlValue.java
@@ -0,0 +1,17 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+/**
+ * Marker interface for a simple value.
+ */
+public interface GridSqlValue {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSubquery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSubquery.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSubquery.java
new file mode 100644
index 0000000..79dc7c0
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSubquery.java
@@ -0,0 +1,44 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+/**
+ * Subquery.
+ */
+public class GridSubquery extends GridSqlElement {
+    /** */
+    private GridSelect select;
+
+    /**
+     * @param select Select.
+     */
+    public GridSubquery(GridSelect select) {
+        this.select = select;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        return "(" + select.getSQL() + ")";
+    }
+
+    /**
+     * @return Select.
+     */
+    public GridSelect select() {
+        return select;
+    }
+
+    /**
+     * @param select New select.
+     */
+    public void select(GridSelect select) {
+        this.select = select;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridTable.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridTable.java
new file mode 100644
index 0000000..53b3b3f
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridTable.java
@@ -0,0 +1,55 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.command.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Table with optional schema.
+ */
+public class GridTable extends GridSqlElement {
+    /** */
+    private final String schema;
+
+    /** */
+    private final String tblName;
+
+    /**
+     * @param schema Schema.
+     * @param tblName Table name.
+     */
+    public GridTable(@Nullable String schema, String tblName) {
+        this.schema = schema;
+        this.tblName = tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        if (schema == null)
+            return Parser.quoteIdentifier(tblName);
+
+        return Parser.quoteIdentifier(schema) + '.' + Parser.quoteIdentifier(tblName);
+    }
+
+    /**
+     * @return Schema.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0497d7f0/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridValueExpression.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridValueExpression.java b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridValueExpression.java
new file mode 100644
index 0000000..05c5c6c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridValueExpression.java
@@ -0,0 +1,39 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.h2.value.*;
+
+/**
+ * Constant value.
+ */
+public class GridValueExpression extends GridSqlElement implements GridSqlValue {
+    /** */
+    private final Value val;
+
+    /**
+     * @param val Value.
+     */
+    public GridValueExpression(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/0497d7f0/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryTest.java
new file mode 100644
index 0000000..4feaba6
--- /dev/null
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryTest.java
@@ -0,0 +1,248 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.query.*;
+import org.gridgain.grid.kernal.*;
+import org.gridgain.grid.kernal.processors.cache.*;
+import org.gridgain.grid.kernal.processors.query.*;
+import org.gridgain.grid.kernal.processors.query.h2.*;
+import org.h2.command.*;
+import org.h2.command.dml.*;
+import org.h2.engine.*;
+import org.h2.jdbc.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class GridQueryTest extends GridCacheAbstractQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheMode cacheMode() {
+        return GridCacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        GridCache cache = ignite.cache(null);
+
+        cache.putx("testAddr", new Address());
+        cache.putx("testPerson", new Person());
+    }
+
+    /**
+     *
+     */
+    public void testAllExampless() throws Exception {
+        checkQuery("select name, date from Person");
+        checkQuery("select distinct name, date from Person");
+        checkQuery("select * from Person p");
+        checkQuery("select * from Person");
+        checkQuery("select distinct * from Person");
+        checkQuery("select p.name, date from Person p");
+
+        checkQuery("select * from Person p, Address a");
+        checkQuery("select * from Person, Address");
+        checkQuery("select p.* from Person p, Address a");
+        checkQuery("select person.* from Person, Address a");
+        checkQuery("select p.*, street from Person p, Address a");
+        checkQuery("select p.name, a.street from Person p, Address a");
+        checkQuery("select distinct p.name, a.street from Person p, Address a");
+        checkQuery("select distinct name, street from Person, Address");
+        checkQuery("select p1.name, a2.street from Person p1, Address a1, Person p2, Address a2");
+
+        checkQuery("select p.name n, a.street s from Person p, Address a");
+        checkQuery("select p.name, 1 as i, 'aaa' s from Person p");
+
+        checkQuery("select p.name + 'a', 1 * 3 as i, 'aaa' s, -p.old, -p.old as old from Person p");
+        checkQuery("select p.name || 'a' + p.name, (p.old * 3) % p.old - p.old / p.old, p.name = 'aaa', " +
+            " p.name is p.name, p.old > 0, p.old >= 0, p.old < 0, p.old <= 0, p.old <> 0, p.old is not p.old, " +
+            " p.old is null, p.old is not null " +
+            " from Person p");
+
+        checkQuery("select p.name from Person p where name <> 'ivan'");
+        checkQuery("select p.name from Person p where name like 'i%'");
+        checkQuery("select p.name from Person p where name regexp 'i%'");
+        checkQuery("select p.name from Person p, Address a where p.name <> 'ivan' and a.id > 10 or not (a.id = 100)");
+
+        checkQuery("select case p.name when 'a' then 1 when 'a' then 2 end as a from Person p");
+        checkQuery("select case p.name when 'a' then 1 when 'a' then 2 else -1 end as a from Person p");
+
+        checkQuery("select abs(p.old)  from Person p");
+        checkQuery("select * from Person p where p.name in ('a', 'b', '_' + RAND())"); // test ConditionIn
+        checkQuery("select * from Person p where p.name in ('a', 'b', 'c')"); // test ConditionInConstantSet
+        checkQuery("select * from Person p where p.name in (select a.street from Address a)"); // test ConditionInConstantSet
+
+        checkQuery("select (select a.street from Address a where a.id = p.addrId) from Person p"); // test ConditionInConstantSet
+
+        checkQuery("select p.name, ? from Person p where name regexp ? and p.old < ?");
+
+        checkQuery("select count(*) as a from Person");
+        checkQuery("select count(*) as a, count(p.*), count(p.name) from Person p");
+        checkQuery("select count(distinct p.name) from Person p");
+
+        checkQuery("select p.name, avg(p.old), max(p.old) from Person p group by p.name");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by n");
+
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.addrId, p.name");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name, p.addrId");
+        checkQuery("select p.name n, max(p.old) + min(p.old) / count(distinct p.old) from Person p group by p.name");
+        checkQuery("select p.name n, max(p.old) maxOld, min(p.old) minOld from Person p group by p.name having maxOld > 10 and min(p.old) < 1");
+
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name order by n");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name order by p.name");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name order by p.name, m");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name order by p.name, max(p.old) desc");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name order by p.name nulls first");
+        checkQuery("select p.name n, avg(p.old) a, max(p.old) m from Person p group by p.name order by p.name nulls last");
+        checkQuery("select p.name n from Person p order by p.old + 10");
+        checkQuery("select p.name n from Person p order by p.old + 10, p.name");
+        checkQuery("select p.name n from Person p order by p.old + 10, p.name desc");
+
+        checkQuery("select p.name n from Person p, (select a.street from Address a where a.street is not null) ");
+        checkQuery("select street from Person p, (select a.street from Address a where a.street is not null) ");
+        checkQuery("select addr.street from Person p, (select a.street from Address a where a.street is not null) addr");
+    }
+
+    /**
+     *
+     */
+    public void testExample1() throws Exception {
+        Select select = parse("select p.name n, max(p.old) maxOld, min(p.old) minOld from Person p group by p.name having maxOld > 10 and min(p.old) < 1");
+
+        GridQueryParser ses = new GridQueryParser();
+
+        GridSelect gridSelect = ses.toGridSelect(select);
+
+        //System.out.println(select.getPlanSQL());
+        System.out.println(gridSelect.getSQL());
+    }
+
+    /**
+     *
+     */
+    private JdbcConnection connection() throws Exception {
+        GridKernalContext ctx = ((GridKernal)ignite).context();
+
+        GridQueryProcessor qryProcessor = ctx.query();
+
+        GridH2Indexing idx = (GridH2Indexing)GridQueryUtils.getFieldValue(GridQueryProcessor.class, qryProcessor,
+            "idx");
+
+        return (JdbcConnection)idx.connectionForSpace(null);
+    }
+
+    /**
+     * @param sql Sql.
+     */
+    private GridSelect toGridSelect(String sql) throws Exception {
+        Session ses = (Session)connection().getSession();
+
+        Select select = (Select)ses.prepare(sql);
+
+        return new GridQueryParser().toGridSelect(select);
+    }
+
+    /**
+     * @param sql Sql.
+     */
+    private <T extends Prepared> T parse(String sql) throws Exception {
+        Session ses = (Session)connection().getSession();
+
+        return (T)ses.prepare(sql);
+    }
+
+    /**
+     * @param sql1 Sql 1.
+     * @param sql2 Sql 2.
+     */
+    private void assertSqlEquals(String sql1, String sql2) {
+        assertEquals(normalizeSql(sql1), normalizeSql(sql2));
+    }
+
+    /**
+     * @param sql Sql.
+     */
+    private static String normalizeSql(String sql) {
+        return sql.toLowerCase()
+            .replaceAll("/\\*(?:.|\r|\n)*?\\*/", " ")
+            .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " ")
+            .replaceAll("\\s+", " ")
+            .replaceAll("\\( +", "(")
+            .replaceAll(" +\\)", ")")
+            .trim();
+    }
+
+    /**
+     * @param qry Query.
+     */
+    private void checkQuery(String qry) throws Exception {
+        Prepared prepared = parse(qry);
+
+        GridQueryParser ses = new GridQueryParser();
+
+        String res;
+
+        if (prepared instanceof Select)
+            res = ses.toGridSelect((Select)prepared).getSQL();
+        else
+            throw new UnsupportedOperationException();
+
+        assertSqlEquals(prepared.getPlanSQL(), res);
+
+        System.out.println(normalizeSql(res));
+    }
+
+    /**
+     *
+     */
+    public static class Person implements Serializable {
+        @GridCacheQuerySqlField(index = true)
+        public Date date = new Date();
+
+        @GridCacheQuerySqlField(index = true)
+        public String name = "Ivan";
+
+        @GridCacheQuerySqlField(index = true)
+        public String parentName;
+
+        @GridCacheQuerySqlField(index = true)
+        public int addrId;
+
+        @GridCacheQuerySqlField(index = true)
+        public int old;
+    }
+
+    /**
+     *
+     */
+    public static class Address implements Serializable {
+        @GridCacheQuerySqlField(index = true)
+        public int id;
+
+        @GridCacheQuerySqlField(index = true)
+        public int streetNumber;
+
+        @GridCacheQuerySqlField(index = true)
+        public String street = "Nevskiy";
+    }
+
+}