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 23:49:53 UTC

[2/2] incubator-ignite git commit: ignite-gg9499 - minor refactoring

ignite-gg9499 - minor refactoring


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

Branch: refs/heads/ignite-gg9499
Commit: 4546628ec2f0d7c24877e1ba0c508315a7045253
Parents: 98df8e1
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Dec 24 01:49:44 2014 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Dec 24 01:49:44 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           | 476 -------------------
 .../processors/query/h2/sql/GridSelect.java     | 241 ----------
 .../query/h2/sql/GridSqlAggregateFunction.java  |  69 +++
 .../processors/query/h2/sql/GridSqlAlias.java   |  55 +++
 .../processors/query/h2/sql/GridSqlColumn.java  |   3 +-
 .../processors/query/h2/sql/GridSqlConst.java   |  39 ++
 .../processors/query/h2/sql/GridSqlElement.java |  14 +-
 .../query/h2/sql/GridSqlFunction.java           |  24 +-
 .../query/h2/sql/GridSqlFunctionType.java       |  62 +++
 .../processors/query/h2/sql/GridSqlJoin.java    |  68 +++
 .../query/h2/sql/GridSqlOperation.java          |  73 +++
 .../query/h2/sql/GridSqlOperationType.java      | 202 ++++++++
 .../query/h2/sql/GridSqlQueryParser.java        | 476 +++++++++++++++++++
 .../processors/query/h2/sql/GridSqlSelect.java  | 241 ++++++++++
 .../query/h2/sql/GridSqlSubquery.java           |  44 ++
 .../processors/query/h2/sql/GridSqlTable.java   |  55 +++
 .../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  |  10 +-
 26 files changed, 1409 insertions(+), 1413 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4546628e/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
deleted file mode 100644
index dffee20..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAggregateFunction.java
+++ /dev/null
@@ -1,70 +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.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/4546628e/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
deleted file mode 100644
index 7c9a080..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridAlias.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/* @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/4546628e/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
deleted file mode 100644
index ba01dd5..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridFunctionType.java
+++ /dev/null
@@ -1,62 +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 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/4546628e/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
deleted file mode 100644
index d703fe7..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridJoin.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/* @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/4546628e/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
deleted file mode 100644
index a52e33e..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperation.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/* @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(GridOperationType opType, GridSqlElement arg) {
-        this(opType);
-
-        addChild(arg);
-    }
-
-    /**
-     * @param opType Op type.
-     * @param left Left.
-     * @param right Right.
-     */
-    public GridOperation(GridOperationType 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 GridOperationType opType() {
-        return opType;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSQL() {
-        return opType.toSql(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4546628e/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
deleted file mode 100644
index 8c56731..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridOperationType.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/* @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/4546628e/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
deleted file mode 100644
index 5b5a434..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridQueryParser.java
+++ /dev/null
@@ -1,476 +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.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.*;
-
-/**
- * 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 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 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 = VIEW_QUERY.get((TableView)tbl);
-
-                assert0(qry instanceof Select, qry);
-
-                res = new GridSubquery(toGridSelect((Select)qry));
-            }
-            else
-                throw new IgniteException("Unsupported query: " + filter);
-
-            String alias = ALIAS.get(filter);
-
-            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 = GROUP_INDEXES.get(select);
-
-        if (grpIdx != null) {
-            for (int idx : grpIdx)
-                res.addGroupExpression(toGridExpression(expressions.get(idx)));
-        }
-
-        assert0(select.getHaving() == null, select);
-
-        int havingIdx = HAVING_INDEX.get(select);
-
-        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(LEFT_CI.get((ConditionIn)expression)));
-
-            List<Expression> vals = VALUE_LIST_CI.get((ConditionIn)expression);
-
-            for (Expression val : vals)
-                res.addChild(toGridExpression(val));
-
-            return res;
-        }
-
-        if (expression instanceof ConditionInConstantSet) {
-            GridOperation res = new GridOperation(IN);
-
-            res.addChild(toGridExpression(LEFT_CICS.get((ConditionInConstantSet)expression)));
-
-            List<Expression> vals = VALUE_LIST_CICS.get((ConditionInConstantSet)expression);
-
-            for (Expression val : vals)
-                res.addChild(toGridExpression(val));
-
-            return res;
-        }
-
-        if (expression instanceof ConditionInSelect) {
-            GridOperation res = new GridOperation(IN);
-
-            boolean all = ALL.get((ConditionInSelect)expression);
-            int compareType = COMPARE_TYPE.get((ConditionInSelect)expression);
-
-            assert0(!all, expression);
-            assert0(compareType == Comparison.EQUAL, expression);
-
-            res.addChild(toGridExpression(LEFT_CIS.get((ConditionInSelect)expression)));
-
-            Query qry = QUERY.get((ConditionInSelect)expression);
-
-            assert0(qry instanceof Select, qry);
-
-            res.addChild(new GridSubquery(toGridSelect((Select)qry)));
-
-            return res;
-        }
-
-        if (expression instanceof CompareLike) {
-            assert0(ESCAPE.get((CompareLike)expression) == null, expression);
-
-            boolean regexp = REGEXP_CL.get((CompareLike)expression);
-
-            return new GridOperation(regexp ? REGEXP : LIKE, toGridExpression(LEFT.get((CompareLike)expression)),
-                toGridExpression(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(toGridExpression(arg));
-
-            if (f.getFunctionType() == Function.CAST)
-                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) {
-            GridAggregateFunction res = new GridAggregateFunction(DISTINCT.get((Aggregate)expression),
-                TYPE.get((Aggregate)expression));
-
-            Expression on = ON.get((Aggregate)expression);
-
-            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);
-    }
-
-    /**
-     * @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);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4546628e/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
deleted file mode 100644
index 30a50e8..0000000
--- a/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSelect.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/* @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/4546628e/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
new file mode 100644
index 0000000..a0daab0
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAggregateFunction.java
@@ -0,0 +1,69 @@
+/* @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]);
+    }
+
+    /** {@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/4546628e/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
new file mode 100644
index 0000000..5b00fb1
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlAlias.java
@@ -0,0 +1,55 @@
+/* @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/4546628e/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
index 046d9ca..ef9b70c 100644
--- 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
@@ -10,10 +10,9 @@
 package org.gridgain.grid.kernal.processors.query.h2.sql;
 
 /**
- *
+ * Column.
  */
 public class GridSqlColumn extends GridSqlElement implements GridSqlValue {
-
     /** */
     private final GridSqlElement expressionInFrom;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4546628e/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
new file mode 100644
index 0000000..01eff8c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlConst.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 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/4546628e/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
index c658e24..3a3acbe 100644
--- 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
@@ -9,8 +9,6 @@
 
 package org.gridgain.grid.kernal.processors.query.h2.sql;
 
-import org.jetbrains.annotations.*;
-
 import java.util.*;
 
 /**
@@ -34,22 +32,24 @@ public abstract class GridSqlElement implements Cloneable {
      * @param expr Expr.
      */
     public void addChild(GridSqlElement expr) {
-        assert expr != null;
+        if (expr == null)
+            throw new NullPointerException();
 
         children.add(expr);
     }
 
     /**
-     *
+     * @return First child.
      */
-    @NotNull public GridSqlElement child() {
+    public GridSqlElement child() {
         return children.get(0);
     }
 
     /**
-     *
+     * @param idx Index.
+     * @return Child.
      */
-    @NotNull public GridSqlElement child(int idx) {
+    public GridSqlElement child(int idx) {
         return children.get(idx);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4546628e/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
index 6256380..0b6b73d 100644
--- 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
@@ -20,13 +20,13 @@ import java.util.*;
  */
 public class GridSqlFunction extends GridSqlElement {
     /** */
-    private static final Map<String, GridFunctionType> TYPE_MAP = new HashMap<>();
+    private static final Map<String, GridSqlFunctionType> TYPE_MAP = new HashMap<>();
 
     /**
      *
      */
     static {
-        for (GridFunctionType type : GridFunctionType.values())
+        for (GridSqlFunctionType type : GridSqlFunctionType.values())
             TYPE_MAP.put(type.name(), type);
     }
 
@@ -34,7 +34,7 @@ public class GridSqlFunction extends GridSqlElement {
     private final String name;
 
     /** */
-    protected final GridFunctionType type;
+    protected final GridSqlFunctionType type;
 
     /**  */
     private String castType;
@@ -42,12 +42,12 @@ public class GridSqlFunction extends GridSqlElement {
     /**
      * @param type Function type.
      */
-    public GridSqlFunction(GridFunctionType type) {
+    public GridSqlFunction(GridSqlFunctionType type) {
         name = type.functionName();
 
         this.type = type;
 
-        if (type == GridFunctionType.CONVERT)
+        if (type == GridSqlFunctionType.CONVERT)
             throw new UnsupportedOperationException();
     }
 
@@ -59,7 +59,7 @@ public class GridSqlFunction extends GridSqlElement {
     }
 
     /**
-     * @param castType Type for {@link GridFunctionType#CAST} function.
+     * @param castType Type for {@link GridSqlFunctionType#CAST} function.
      */
     public void setCastType(String castType) {
         this.castType = castType;
@@ -69,7 +69,7 @@ public class GridSqlFunction extends GridSqlElement {
     @Override public String getSQL() {
         StatementBuilder buff = new StatementBuilder(name);
 
-        if (type == GridFunctionType.CASE) {
+        if (type == GridSqlFunctionType.CASE) {
             if (!children.isEmpty())
                 buff.append(" ").append(child().getSQL());
 
@@ -85,19 +85,19 @@ public class GridSqlFunction extends GridSqlElement {
 
         buff.append('(');
 
-        if (type == GridFunctionType.CAST) {
+        if (type == GridSqlFunctionType.CAST) {
             assert !F.isEmpty(castType) : castType;
             assert children().size() == 1;
 
             buff.append(child().getSQL()).append(" AS ").append(castType);
         }
-        else if (type == GridFunctionType.CONVERT) {
+        else if (type == GridSqlFunctionType.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();
+        else if (type == GridSqlFunctionType.EXTRACT) {
+            ValueString v = (ValueString)((GridSqlConst)child(0)).value();
             buff.append(v.getString()).append(" FROM ").append(child(1).getSQL());
         }
         else {
@@ -120,7 +120,7 @@ public class GridSqlFunction extends GridSqlElement {
     /**
      * @return Type.
      */
-    public GridFunctionType type() {
+    public GridSqlFunctionType type() {
         return type;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4546628e/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
new file mode 100644
index 0000000..e9aa8c9
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlFunctionType.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 GridSqlFunctionType {
+    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.
+     */
+    GridSqlFunctionType(int argCnt) {
+        name = name();
+        this.argCnt = argCnt;
+    }
+
+    /**
+     * @param name Name.
+     * @param argCnt Count.
+     */
+    GridSqlFunctionType(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/4546628e/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
new file mode 100644
index 0000000..a05f973
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlJoin.java
@@ -0,0 +1,68 @@
+/* @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/4546628e/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
new file mode 100644
index 0000000..4477f0c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperation.java
@@ -0,0 +1,73 @@
+/* @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/4546628e/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
new file mode 100644
index 0000000..35fe568
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlOperationType.java
@@ -0,0 +1,202 @@
+/* @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/4546628e/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
new file mode 100644
index 0000000..3c8c77a
--- /dev/null
+++ b/modules/indexing/src/main/java/org/gridgain/grid/kernal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -0,0 +1,476 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.query.h2.sql;
+
+import org.apache.ignite.*;
+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.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 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 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(toGridSelect((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 toGridSelect(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(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 GridSqlJoin(from, gridFilter);
+
+            allFilers.remove(filter);
+
+            filter = filter.getJoin();
+        }
+        while (filter != null);
+
+        res.from(from);
+
+        assert allFilers.isEmpty();
+
+        ArrayList<Expression> expressions = select.getExpressions();
+
+        int[] grpIdx = GROUP_INDEXES.get(select);
+
+        if (grpIdx != null) {
+            for (int idx : grpIdx)
+                res.addGroupExpression(toGridExpression(expressions.get(idx)));
+        }
+
+        assert0(select.getHaving() == null, select);
+
+        int havingIdx = HAVING_INDEX.get(select);
+
+        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 GridSqlAlias(expression.getAlias(), toGridExpression(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, toGridExpression(OPERATION_LEFT.get(operation)));
+            }
+
+            return new GridSqlOperation(OPERATION_OP_TYPES[type],
+                toGridExpression(OPERATION_LEFT.get(operation)),
+                toGridExpression(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 = toGridExpression(COMPARISON_LEFT.get(cmp));
+
+            if (opType.childrenCount() == 1)
+                return new GridSqlOperation(opType, left);
+
+            GridSqlElement right = toGridExpression(COMPARISON_RIGHT.get(cmp));
+
+            return new GridSqlOperation(opType, left, right);
+        }
+
+        if (expression instanceof ConditionNot)
+            return new GridSqlOperation(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 GridSqlOperation(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 GridSqlSubquery(toGridSelect((Select)qry));
+        }
+
+        if (expression instanceof ConditionIn) {
+            GridSqlOperation res = new GridSqlOperation(IN);
+
+            res.addChild(toGridExpression(LEFT_CI.get((ConditionIn)expression)));
+
+            List<Expression> vals = VALUE_LIST_CI.get((ConditionIn)expression);
+
+            for (Expression val : vals)
+                res.addChild(toGridExpression(val));
+
+            return res;
+        }
+
+        if (expression instanceof ConditionInConstantSet) {
+            GridSqlOperation res = new GridSqlOperation(IN);
+
+            res.addChild(toGridExpression(LEFT_CICS.get((ConditionInConstantSet)expression)));
+
+            List<Expression> vals = VALUE_LIST_CICS.get((ConditionInConstantSet)expression);
+
+            for (Expression val : vals)
+                res.addChild(toGridExpression(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(toGridExpression(LEFT_CIS.get((ConditionInSelect)expression)));
+
+            Query qry = QUERY.get((ConditionInSelect)expression);
+
+            assert0(qry instanceof Select, qry);
+
+            res.addChild(new GridSqlSubquery(toGridSelect((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, toGridExpression(LEFT.get((CompareLike)expression)),
+                toGridExpression(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(toGridExpression(arg));
+
+            if (f.getFunctionType() == Function.CAST)
+                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(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);
+    }
+
+    /**
+     * @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);
+            }
+        }
+    }
+}