You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by av...@apache.org on 2017/03/01 14:32:54 UTC

[11/50] [abbrv] ignite git commit: ignite-3860 - merge to 1.9 # Conflicts: # modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java # modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/

ignite-3860 - merge to 1.9
# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java


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

Branch: refs/heads/master
Commit: 3737407b28294e08e3e5d9a54b7efa350a7377c1
Parents: 5b94a7d
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Feb 20 22:06:50 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Feb 20 22:06:50 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |    6 +
 .../cache/query/GridCacheSqlQuery.java          |   45 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  104 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |    2 +-
 .../query/h2/opt/GridH2CollocationModel.java    |   78 +-
 .../query/h2/opt/GridH2IndexBase.java           |   47 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |   46 +-
 .../processors/query/h2/sql/GridSqlAlias.java   |   13 +-
 .../processors/query/h2/sql/GridSqlArray.java   |    8 +-
 .../processors/query/h2/sql/GridSqlAst.java     |   61 +
 .../processors/query/h2/sql/GridSqlColumn.java  |   85 +-
 .../processors/query/h2/sql/GridSqlConst.java   |    6 +-
 .../processors/query/h2/sql/GridSqlElement.java |   43 +-
 .../query/h2/sql/GridSqlFunction.java           |   16 +-
 .../processors/query/h2/sql/GridSqlJoin.java    |   35 +-
 .../processors/query/h2/sql/GridSqlKeyword.java |    3 +-
 .../query/h2/sql/GridSqlOperation.java          |    6 +-
 .../query/h2/sql/GridSqlOperationType.java      |    4 +-
 .../query/h2/sql/GridSqlParameter.java          |    4 +-
 .../query/h2/sql/GridSqlPlaceholder.java        |    2 +-
 .../processors/query/h2/sql/GridSqlQuery.java   |   80 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  228 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      | 1633 +++++++++++++++---
 .../processors/query/h2/sql/GridSqlSelect.java  |  121 +-
 .../query/h2/sql/GridSqlStatement.java          |    6 +-
 .../query/h2/sql/GridSqlSubquery.java           |   31 +-
 .../processors/query/h2/sql/GridSqlTable.java   |   19 +-
 .../processors/query/h2/sql/GridSqlType.java    |    6 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |   66 +-
 .../processors/query/h2/sql/GridSqlValue.java   |   25 -
 .../query/h2/twostep/GridMapQueryExecutor.java  |    7 +-
 .../query/h2/twostep/GridMergeIndex.java        |  418 ++++-
 .../query/h2/twostep/GridMergeIndexSorted.java  |  284 +++
 .../h2/twostep/GridMergeIndexUnsorted.java      |   40 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   26 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |    7 +
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |    2 +-
 .../query/IgniteSqlSplitterSelfTest.java        |  101 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   41 +-
 39 files changed, 2974 insertions(+), 781 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 914ba2f..47a66fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -403,6 +403,12 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_SQL_MERGE_TABLE_MAX_SIZE = "IGNITE_SQL_MERGE_TABLE_MAX_SIZE";
 
+    /**
+     * Property controlling number of SQL result rows that will be fetched into a merge table at once before
+     * applying binary search for the bounds.
+     */
+    public static final String IGNITE_SQL_MERGE_TABLE_PREFETCH_SIZE = "IGNITE_SQL_MERGE_TABLE_PREFETCH_SIZE";
+
     /** Maximum size for affinity assignment history. */
     public static final String IGNITE_AFFINITY_HISTORY_SIZE = "IGNITE_AFFINITY_HISTORY_SIZE";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index ff1ac7f..18688b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@ -83,22 +83,11 @@ public class GridCacheSqlQuery implements Message, GridCacheQueryMarshallable {
 
     /**
      * @param qry Query.
-     * @param params Query parameters.
      */
-    public GridCacheSqlQuery(String qry, Object[] params) {
+    public GridCacheSqlQuery(String qry) {
         A.ensure(!F.isEmpty(qry), "qry must not be empty");
 
         this.qry = qry;
-
-        this.params = F.isEmpty(params) ? EMPTY_PARAMS : params;
-        paramsSize = this.params.length;
-    }
-
-    /**
-     * @param paramIdxs Parameter indexes.
-     */
-    public void parameterIndexes(int[] paramIdxs) {
-        this.paramIdxs = paramIdxs;
     }
 
     /**
@@ -126,12 +115,44 @@ public class GridCacheSqlQuery implements Message, GridCacheQueryMarshallable {
     }
 
     /**
+     * @param qry Query.
+     * @return {@code this}.
+     */
+    public GridCacheSqlQuery query(String qry) {
+        this.qry = qry;
+
+        return this;
+    }
+
+    /**
      * @return Parameters.
      */
     public Object[] parameters() {
         return params;
     }
 
+    /**
+     * @return Parameter indexes.
+     */
+    public int[] parameterIndexes() {
+        return paramIdxs;
+    }
+
+    /**
+     * @param params Parameters.
+     * @param paramIdxs Parameter indexes.
+     * @return {@code this} For chaining.
+     */
+    public GridCacheSqlQuery parameters(Object[] params, int[] paramIdxs) {
+        this.params = F.isEmpty(params) ? EMPTY_PARAMS : params;
+
+        paramsSize = this.params.length;
+
+        this.paramIdxs = paramIdxs;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public void marshall(Marshaller m) {
         if (paramsBytes != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 8088f80..15e7fc6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -214,6 +214,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         ";ROW_FACTORY=\"" + GridH2RowFactory.class.getName() + "\"" +
         ";DEFAULT_TABLE_ENGINE=" + GridH2DefaultTableEngine.class.getName();
 
+        // Uncomment this setting to get debug output from H2 to sysout.
+//        ";TRACE_LEVEL_SYSTEM_OUT=3";
+
     /** */
     private static final int PREPARED_STMT_CACHE_SIZE = 256;
 
@@ -1008,9 +1011,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         int timeoutMillis, @Nullable GridQueryCancel cancel)
         throws IgniteCheckedException {
 
-        if (timeoutMillis > 0)
-            ((Session)((JdbcConnection)conn).getSession()).setQueryTimeout(timeoutMillis);
-
         if (cancel != null) {
             cancel.set(new Runnable() {
                 @Override public void run() {
@@ -1024,6 +1024,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             });
         }
 
+        if (timeoutMillis > 0)
+            session(conn).setQueryTimeout(timeoutMillis);
+
         try {
             return stmt.executeQuery();
         }
@@ -1036,7 +1039,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         finally {
             if (timeoutMillis > 0)
-                ((Session)((JdbcConnection)conn).getSession()).setQueryTimeout(0);
+                session(conn).setQueryTimeout(0);
         }
     }
 
@@ -1135,7 +1138,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param distributedJoins If distributed joins are enabled.
      * @param enforceJoinOrder Enforce join order of tables.
      */
-    public void setupConnection(Connection conn, boolean distributedJoins, boolean enforceJoinOrder) {
+    public static void setupConnection(Connection conn, boolean distributedJoins, boolean enforceJoinOrder) {
         Session s = session(conn);
 
         s.setForceJoinOrder(enforceJoinOrder);
@@ -1278,7 +1281,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         final boolean distributedJoins = qry.isDistributedJoins() && cctx.isPartitioned();
         final boolean grpByCollocated = qry.isCollocated();
 
-        GridCacheTwoStepQuery twoStepQry;
+        GridCacheTwoStepQuery twoStepQry = null;
         List<GridQueryFieldMetadata> meta;
 
         final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(space, sqlQry, grpByCollocated,
@@ -1292,65 +1295,72 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         else {
             final UUID locNodeId = ctx.localNodeId();
 
-            setupConnection(c, distributedJoins, enforceJoinOrder);
+            // Here we will just parse the statement, no need to optimize it at all.
+            setupConnection(c, /*distributedJoins*/false, /*enforceJoinOrder*/true);
 
             GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, 0, PREPARE)
                 .distributedJoins(distributedJoins));
 
-            PreparedStatement stmt;
+            PreparedStatement stmt = null;
 
             boolean cachesCreated = false;
 
             try {
-                while (true) {
-                    try {
-                        // Do not cache this statement because the whole two step query object will be cached later on.
-                        stmt = prepareStatement(c, sqlQry, false);
+                try {
+                    while (true) {
+                        try {
+                            // Do not cache this statement because the whole two step query object will be cached later on.
+                            stmt = prepareStatement(c, sqlQry, false);
 
-                        break;
-                    }
-                    catch (SQLException e) {
-                        if (!cachesCreated && e.getErrorCode() == ErrorCode.SCHEMA_NOT_FOUND_1) {
-                            try {
-                                ctx.cache().createMissingCaches();
-                            }
-                            catch (IgniteCheckedException ignored) {
-                                throw new CacheException("Failed to create missing caches.", e);
-                            }
+                            break;
+                        }
+                        catch (SQLException e) {
+                            if (!cachesCreated && e.getErrorCode() == ErrorCode.SCHEMA_NOT_FOUND_1) {
+                                try {
+                                    ctx.cache().createMissingCaches();
+                                }
+                                catch (IgniteCheckedException ignored) {
+                                    throw new CacheException("Failed to create missing caches.", e);
+                                }
 
-                            cachesCreated = true;
+                                cachesCreated = true;
+                            }
+                            else
+                                throw new IgniteSQLException("Failed to parse query: " + sqlQry,
+                                    IgniteQueryErrorCode.PARSING, e);
                         }
-                        else
-                            throw new IgniteSQLException("Failed to parse query: " + sqlQry,
-                                IgniteQueryErrorCode.PARSING, e);
                     }
-                }
-            }
-            finally {
-                GridH2QueryContext.clearThreadLocal();
-            }
 
-            Prepared prepared = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
 
-            if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery) qry).isQuery() != prepared.isQuery())
-                throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
-                    IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+                    Prepared prepared = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
 
-            if (!prepared.isQuery()) {
-                try {
-                    return dmlProc.updateSqlFieldsTwoStep(cctx.namexx(), stmt, qry, cancel);
+                    if (qry instanceof JdbcSqlFieldsQuery && ((JdbcSqlFieldsQuery) qry).isQuery() != prepared.isQuery())
+                        throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
+                            IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+
+                    if (prepared.isQuery()) {
+                        bindParameters(stmt, F.asList(qry.getArgs()));
+
+                        twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), grpByCollocated,
+                            distributedJoins, enforceJoinOrder, this);
+
+                        assert twoStepQry != null;
+                    }
                 }
-                catch (IgniteCheckedException e) {
-                    throw new IgniteSQLException("Failed to execute DML statement [qry=" + sqlQry + ", params=" +
-                        Arrays.deepToString(qry.getArgs()) + "]", e);
+                finally {
+                    GridH2QueryContext.clearThreadLocal();
                 }
-            }
-
-            try {
-                bindParameters(stmt, F.asList(qry.getArgs()));
 
-                twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), grpByCollocated,
-                    distributedJoins);
+                // It is a DML statement if we did not create a twoStepQuery.
+                if (twoStepQry == null) {
+                    try {
+                        return dmlProc.updateSqlFieldsTwoStep(cctx.namexx(), stmt, qry, cancel);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IgniteSQLException("Failed to execute DML statement [qry=" + sqlQry + ", params=" +
+                            Arrays.deepToString(qry.getArgs()) + "]", e);
+                    }
+                }
 
                 List<Integer> caches;
                 List<Integer> extraCaches = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index fdcd164..fc82161 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -76,7 +76,7 @@ public final class UpdatePlanBuilder {
         @Nullable Integer errKeysPos) throws IgniteCheckedException {
         assert !prepared.isQuery();
 
-        GridSqlStatement stmt = new GridSqlQueryParser().parse(prepared);
+        GridSqlStatement stmt = new GridSqlQueryParser(false).parse(prepared);
 
         if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert)
             return planForInsert(stmt);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
index 0f76316..ce11fd5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
@@ -23,6 +23,7 @@ import java.util.List;
 import javax.cache.CacheException;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.h2.command.dml.Query;
 import org.h2.command.dml.Select;
 import org.h2.command.dml.SelectUnion;
@@ -38,6 +39,8 @@ import org.h2.table.Table;
 import org.h2.table.TableFilter;
 import org.h2.table.TableView;
 
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
+
 /**
  * Collocation model for a query.
  */
@@ -98,6 +101,63 @@ public final class GridH2CollocationModel {
     }
 
     /**
+     * @return Table filter for this collocation model.
+     */
+    private TableFilter filter() {
+        return upper == null ? null : upper.childFilters[filter];
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        calculate();
+
+        SB b = new SB();
+
+        for (int lvl = 0; lvl < 20; lvl++) {
+            if (!toString(b, lvl))
+                break;
+
+            b.a('\n');
+        }
+
+        return b.toString();
+    }
+
+    /**
+     * @param b String builder.
+     * @param lvl Depth level.
+     */
+    private boolean toString(SB b, int lvl) {
+        boolean res = false;
+
+        if (lvl == 0) {
+            TableFilter f = filter();
+            String tblAlias = f == null ? "^" : f.getTableAlias();
+
+            b.a("[tbl=").a(tblAlias).a(", type=").a(type).a(", mul=").a(multiplier).a("]");
+
+            res = true;
+        }
+        else if (childFilters != null) {
+            assert lvl > 0;
+
+            lvl--;
+
+            for (int i = 0; i < childFilters.length; i++) {
+                if (lvl == 0)
+                    b.a(" | ");
+
+                res |= child(i, true).toString(b, lvl);
+            }
+
+            if (lvl == 0)
+                b.a(" | ");
+        }
+
+        return res;
+    }
+
+    /**
      * @param upper Upper.
      * @param filter Filter.
      * @param unions Unions.
@@ -240,7 +300,7 @@ public final class GridH2CollocationModel {
             assert childFilters == null;
 
             // We are at table instance.
-            GridH2Table tbl = (GridH2Table)upper.childFilters[filter].getTable();
+            GridH2Table tbl = (GridH2Table)filter().getTable();
 
             // Only partitioned tables will do distributed joins.
             if (!tbl.isPartitioned()) {
@@ -261,7 +321,7 @@ public final class GridH2CollocationModel {
                 // It is enough to make sure that our previous join by affinity key is collocated, then we are
                 // collocated. If we at least have affinity key condition, then we do unicast which is cheaper.
                 switch (upper.joinedWithCollocated(filter)) {
-                    case JOINED_WITH_COLLOCATED:
+                    case COLLOCATED_JOIN:
                         type = Type.PARTITIONED_COLLOCATED;
                         multiplier = MULTIPLIER_COLLOCATED;
 
@@ -349,15 +409,14 @@ public final class GridH2CollocationModel {
 
             for (int i = 0; i < idxConditions.size(); i++) {
                 IndexCondition c = idxConditions.get(i);
-
+                int colId = c.getColumn().getColumnId();
                 int cmpType = c.getCompareType();
 
                 if ((cmpType == Comparison.EQUAL || cmpType == Comparison.EQUAL_NULL_SAFE) &&
-                    c.getColumn().getColumnId() == affColId && c.isEvaluatable()) {
+                    (colId == affColId || colId == KEY_COL) && c.isEvaluatable()) {
                     affKeyCondFound = true;
 
                     Expression exp = c.getExpression();
-
                     exp = exp.getNonAliasExpression();
 
                     if (exp instanceof ExpressionColumn) {
@@ -369,11 +428,14 @@ public final class GridH2CollocationModel {
                         if (prevJoin != null) {
                             GridH2CollocationModel cm = child(indexOf(prevJoin), true);
 
-                            if (cm != null) {
+                            // If the previous joined model is a subquery (view), we can not be sure that
+                            // the found affinity column is the needed one, since we can select multiple
+                            // different affinity columns from different tables.
+                            if (cm != null && !cm.view) {
                                 Type t = cm.type(true);
 
                                 if (t.isPartitioned() && t.isCollocated() && isAffinityColumn(prevJoin, expCol, validate))
-                                    return Affinity.JOINED_WITH_COLLOCATED;
+                                    return Affinity.COLLOCATED_JOIN;
                             }
                         }
                     }
@@ -778,6 +840,6 @@ public final class GridH2CollocationModel {
         HAS_AFFINITY_CONDITION,
 
         /** */
-        JOINED_WITH_COLLOCATED
+        COLLOCATED_JOIN
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index bab219c..bdfddd5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -245,14 +245,15 @@ public abstract class GridH2IndexBase extends BaseIndex {
      * @param filter Current filter.
      * @return Multiplier.
      */
-    public int getDistributedMultiplier(Session ses, TableFilter[] filters, int filter) {
+    public final int getDistributedMultiplier(Session ses, TableFilter[] filters, int filter) {
         GridH2QueryContext qctx = GridH2QueryContext.get();
 
-        // We do complex optimizations with respect to distributed joins only on prepare stage
-        // because on run stage reordering of joined tables by Optimizer is explicitly disabled
-        // and thus multiplier will be always the same, so it will not affect choice of index.
+        // We do optimizations with respect to distributed joins only on PREPARE stage only.
+        // Notice that we check for isJoinBatchEnabled, because we can do multiple different
+        // optimization passes on PREPARE stage.
         // Query expressions can not be distributed as well.
-        if (qctx == null || qctx.type() != PREPARE || !qctx.distributedJoins() || ses.isPreparingQueryExpression())
+        if (qctx == null || qctx.type() != PREPARE || !qctx.distributedJoins() ||
+            !ses.isJoinBatchEnabled() || ses.isPreparingQueryExpression())
             return GridH2CollocationModel.MULTIPLIER_COLLOCATED;
 
         // We have to clear this cache because normally sub-query plan cost does not depend on anything
@@ -368,17 +369,17 @@ public abstract class GridH2IndexBase extends BaseIndex {
 
         IndexColumn affCol = getTable().getAffinityKeyColumn();
 
-        int affColId;
-        boolean ucast;
+        int affColId = -1;
+        boolean ucast = false;
 
         if (affCol != null) {
             affColId = affCol.column.getColumnId();
             int[] masks = filter.getMasks();
-            ucast = masks != null && masks[affColId] == IndexCondition.EQUALITY;
-        }
-        else {
-            affColId = -1;
-            ucast = false;
+
+            if (masks != null) {
+                ucast = (masks[affColId] & IndexCondition.EQUALITY) != 0 ||
+                    (masks[KEY_COL] & IndexCondition.EQUALITY) != 0;
+            }
         }
 
         GridCacheContext<?,?> cctx = getTable().rowDescriptor().context();
@@ -713,6 +714,21 @@ public abstract class GridH2IndexBase extends BaseIndex {
     }
 
     /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    public static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {
+        // TODO Optimize: use binary search if the range in array is big.
+        for (int i = off, last = arr.length - 1; i < last; i++) {
+            if (cmp.compare(arr[i], arr[i + 1]) <= 0)
+                break;
+
+            U.swap(arr, i, i + 1);
+        }
+    }
+
+    /**
      * @param msg Message.
      * @return Row.
      */
@@ -877,12 +893,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
             }
 
             // Bubble up current min stream with respect to fetched row to achieve correct sort order of streams.
-            for (int i = off, last = streams.length - 1; i < last; i++) {
-                if (compareRows(streams[i].get(rangeId), streams[i + 1].get(rangeId)) <= 0)
-                    break;
-
-                U.swap(streams, i, i + 1);
-            }
+            bubbleUp(streams, off, this);
 
             return true;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
index 9752c5f..35f0983 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/DmlAstUtils.java
@@ -95,7 +95,7 @@ public final class DmlAstUtils {
 
                 args[i] = arr;
 
-                GridSqlColumn newCol = new GridSqlColumn(null, from, colName, "TABLE." + colName);
+                GridSqlColumn newCol = new GridSqlColumn(null, from, null,"TABLE", colName);
 
                 newCol.resultType(cols[i].resultType());
 
@@ -146,10 +146,10 @@ public final class DmlAstUtils {
 
         Column h2ValCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.VAL_COL);
 
-        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName(), h2KeyCol.getSQL());
+        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName());
         keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
 
-        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName(), h2ValCol.getSQL());
+        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName());
         valCol.resultType(GridSqlType.fromColumn(h2ValCol));
 
         mapQry.addColumn(keyCol, true);
@@ -234,7 +234,7 @@ public final class DmlAstUtils {
 
         // Does this WHERE limit only by _key?
         if (isKeyEqualityCondition(whereOp))
-            return new IgnitePair<>(whereOp.child(1), null);
+            return new IgnitePair<>((GridSqlElement)whereOp.child(1), null);
 
         // Or maybe it limits both by _key and _val?
         if (whereOp.operationType() != GridSqlOperationType.AND)
@@ -255,13 +255,13 @@ public final class DmlAstUtils {
             if (!isValueEqualityCondition(rightOp))
                 return null;
 
-            return new IgnitePair<>(leftOp.child(1), rightOp.child(1));
+            return new IgnitePair<>((GridSqlElement)leftOp.child(1), (GridSqlElement)rightOp.child(1));
         }
         else if (isKeyEqualityCondition(rightOp)) { // _val = ? and _key = ?
             if (!isValueEqualityCondition(leftOp))
                 return null;
 
-            return new IgnitePair<>(rightOp.child(1), leftOp.child(1));
+            return new IgnitePair<>((GridSqlElement)rightOp.child(1), (GridSqlElement)leftOp.child(1));
         }
         else // Neither
             return null;
@@ -329,10 +329,10 @@ public final class DmlAstUtils {
 
         Column h2ValCol = gridTbl.getColumn(GridH2AbstractKeyValueRow.VAL_COL);
 
-        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName(), h2KeyCol.getSQL());
+        GridSqlColumn keyCol = new GridSqlColumn(h2KeyCol, tbl, h2KeyCol.getName());
         keyCol.resultType(GridSqlType.fromColumn(h2KeyCol));
 
-        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName(), h2ValCol.getSQL());
+        GridSqlColumn valCol = new GridSqlColumn(h2ValCol, tbl, h2ValCol.getName());
         valCol.resultType(GridSqlType.fromColumn(h2ValCol));
 
         mapQry.addColumn(keyCol, true);
@@ -424,7 +424,7 @@ public final class DmlAstUtils {
 
         sel.from(from);
 
-        GridSqlColumn col = new GridSqlColumn(null, from, "_IGNITE_ERR_KEYS", "TABLE._IGNITE_ERR_KEYS");
+        GridSqlColumn col = new GridSqlColumn(null, from, null, "TABLE", "_IGNITE_ERR_KEYS");
 
         sel.addColumn(col, true);
 
@@ -459,8 +459,8 @@ public final class DmlAstUtils {
         findParams(union.left(), params, target, paramIdxs);
         findParams(union.right(), params, target, paramIdxs);
 
-        findParams(qry.limit(), params, target, paramIdxs);
-        findParams(qry.offset(), params, target, paramIdxs);
+        findParams((GridSqlElement)qry.limit(), params, target, paramIdxs);
+        findParams((GridSqlElement)qry.offset(), params, target, paramIdxs);
 
         return target;
     }
@@ -477,16 +477,16 @@ public final class DmlAstUtils {
         if (params.length == 0)
             return target;
 
-        for (GridSqlElement el : qry.columns(false))
-            findParams(el, params, target, paramIdxs);
+        for (GridSqlAst el : qry.columns(false))
+            findParams((GridSqlElement)el, params, target, paramIdxs);
 
-        findParams(qry.from(), params, target, paramIdxs);
-        findParams(qry.where(), params, target, paramIdxs);
+        findParams((GridSqlElement)qry.from(), params, target, paramIdxs);
+        findParams((GridSqlElement)qry.where(), params, target, paramIdxs);
 
         // Don't search in GROUP BY and HAVING since they expected to be in select list.
 
-        findParams(qry.limit(), params, target, paramIdxs);
-        findParams(qry.offset(), params, target, paramIdxs);
+        findParams((GridSqlElement)qry.limit(), params, target, paramIdxs);
+        findParams((GridSqlElement)qry.offset(), params, target, paramIdxs);
 
         return target;
     }
@@ -524,10 +524,10 @@ public final class DmlAstUtils {
             paramIdxs.add(idx);
         }
         else if (el instanceof GridSqlSubquery)
-            findParams(((GridSqlSubquery)el).select(), params, target, paramIdxs);
+            findParams(((GridSqlSubquery)el).subquery(), params, target, paramIdxs);
         else
-            for (GridSqlElement child : el)
-                findParams(child, params, target, paramIdxs);
+            for (int i = 0; i < el.size(); i++)
+                findParams((GridSqlElement)el.child(i), params, target, paramIdxs);
     }
 
     /**
@@ -546,17 +546,17 @@ public final class DmlAstUtils {
 
         if (from instanceof GridSqlJoin) {
             // Left and right.
-            if (findTablesInFrom(from.child(0), c))
+            if (findTablesInFrom((GridSqlElement)from.child(0), c))
                 return true;
 
-            if (findTablesInFrom(from.child(1), c))
+            if (findTablesInFrom((GridSqlElement)from.child(1), c))
                 return true;
 
             // We don't process ON condition because it is not a joining part of from here.
             return false;
         }
         else if (from instanceof GridSqlAlias)
-            return findTablesInFrom(from.child(), c);
+            return findTablesInFrom((GridSqlElement)from.child(), c);
         else if (from instanceof GridSqlFunction)
             return false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlias.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlias.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlias.java
index 3fb6f3c..eb84c90 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlias.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlias.java
@@ -34,7 +34,7 @@ public class GridSqlAlias extends GridSqlElement {
      * @param alias Alias.
      * @param expr Expr.
      */
-    public GridSqlAlias(String alias, GridSqlElement expr) {
+    public GridSqlAlias(String alias, GridSqlAst expr) {
         this(alias, expr, false);
     }
 
@@ -43,8 +43,8 @@ public class GridSqlAlias extends GridSqlElement {
      * @param expr Expr.
      * @param useAs Use 'AS' keyword.
      */
-    public GridSqlAlias(String alias, GridSqlElement expr, boolean useAs) {
-        super(new ArrayList<GridSqlElement>(1));
+    public GridSqlAlias(String alias, GridSqlAst expr, boolean useAs) {
+        super(new ArrayList<GridSqlAst>(1));
 
         addChild(expr);
 
@@ -56,17 +56,18 @@ public class GridSqlAlias extends GridSqlElement {
      * @param el Element.
      * @return Unwrapped from alias element.
      */
-    public static GridSqlElement unwrap(GridSqlElement el) {
+    @SuppressWarnings("unchecked")
+    public static <X extends GridSqlAst> X unwrap(GridSqlAst el) {
         el = el instanceof GridSqlAlias ? el.child() : el;
 
         assert el != null;
 
-        return el;
+        return (X)el;
     }
 
     /** {@inheritDoc} */
     @Override public String getSQL() {
-        return child().getSQL() + (useAs ? " AS " : " ") + Parser.quoteIdentifier(alias);
+        return child(0).getSQL() + (useAs ? " AS " : " ") + Parser.quoteIdentifier(alias);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
index caa2000..2f7e2f0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
@@ -30,13 +30,13 @@ public class GridSqlArray extends GridSqlElement {
      * @param size Array size.
      */
     public GridSqlArray(int size) {
-        super(size == 0 ? Collections.<GridSqlElement>emptyList() : new ArrayList<GridSqlElement>(size));
+        super(size == 0 ? Collections.<GridSqlAst>emptyList() : new ArrayList<GridSqlAst>(size));
     }
 
     /**
      * @param children Initial child list.
      */
-    public GridSqlArray(List<GridSqlElement> children) {
+    public GridSqlArray(List<GridSqlAst> children) {
         super(children);
     }
 
@@ -47,9 +47,9 @@ public class GridSqlArray extends GridSqlElement {
 
         StatementBuilder buff = new StatementBuilder("(");
 
-        for (GridSqlElement e : children) {
+        for (int i = 0; i < size(); i++) {
             buff.appendExceptFirst(", ");
-            buff.append(e.getSQL());
+            buff.append(child(i).getSQL());
         }
 
         if (size() == 1)

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAst.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAst.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAst.java
new file mode 100644
index 0000000..b087e74
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAst.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2.sql;
+
+/**
+ * AST for SQL.
+ */
+public interface GridSqlAst {
+    /**
+     * @return Generate sql from this AST.
+     */
+    public String getSQL();
+
+    /**
+     * @return Number of child nodes.
+     */
+    public int size();
+
+    /**
+     * Get child by index.
+     *
+     * @param childIdx Index of the requested child.
+     * @return Child element.
+     */
+    public <E extends GridSqlAst> E child(int childIdx);
+
+    /**
+     * Get the first child.
+     *
+     * @return Child element.
+     */
+    public <E extends GridSqlAst> E child();
+
+    /**
+     * Set child.
+     *
+     * @param childIdx Index of the requested child.
+     * @param child Child element.
+     */
+    public <E extends GridSqlAst> void child(int childIdx, E child);
+
+    /**
+     * @return Optional expression result type (if this is an expression and result type is known).
+     */
+    public GridSqlType resultType();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
index 6ef4446..6f4141e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
@@ -18,20 +18,25 @@
 package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.Collections;
+import org.apache.ignite.internal.util.typedef.F;
+import org.h2.command.Parser;
 import org.h2.table.Column;
 
 /**
  * Column.
  */
-public class GridSqlColumn extends GridSqlElement implements GridSqlValue {
+public class GridSqlColumn extends GridSqlElement {
     /** */
-    private final GridSqlElement expressionInFrom;
+    private GridSqlAst from;
 
     /** */
-    private final String colName;
+    private String schema;
 
-    /** SQL from original query. May be qualified or unqualified column name. */
-    private final String sqlText;
+    /** */
+    private String tblAlias;
+
+    /** */
+    private String colName;
 
     /** */
     private Column col;
@@ -39,18 +44,30 @@ public class GridSqlColumn extends GridSqlElement implements GridSqlValue {
     /**
      * @param col Column.
      * @param from From.
-     * @param name Name.
-     * @param sqlText Text.
+     * @param colName Column name.
      */
-    public GridSqlColumn(Column col, GridSqlElement from, String name, String sqlText) {
-        super(Collections.<GridSqlElement>emptyList());
+    public GridSqlColumn(Column col, GridSqlAst from, String colName) {
+        this(col, from, null, null, colName);
+    }
 
-        assert sqlText != null;
+    /**
+     * @param col Column.
+     * @param from From.
+     * @param schema Schema name.
+     * @param tblAlias Table alias.
+     * @param colName Column name.
+     */
+    public GridSqlColumn(Column col, GridSqlAst from, String schema, String tblAlias, String colName) {
+        super(Collections.<GridSqlAst>emptyList());
+
+        assert !F.isEmpty(colName): colName;
 
-        expressionInFrom = from;
-        colName = name;
-        this.sqlText = sqlText;
         this.col = col;
+        this.from = from;
+
+        this.colName = colName;
+        this.schema = schema;
+        this.tblAlias = tblAlias;
     }
 
     /**
@@ -60,16 +77,52 @@ public class GridSqlColumn extends GridSqlElement implements GridSqlValue {
         return colName;
     }
 
+    /**
+     * @return Schema name.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @return Table alias.
+     */
+    public String tableAlias() {
+        return tblAlias;
+    }
+
+    /**
+     * @param tblAlias Table alias.
+     */
+    public void tableAlias(String tblAlias) {
+        this.tblAlias = tblAlias;
+    }
+
     /** {@inheritDoc} */
     @Override public String getSQL() {
-        return sqlText;
+        String sql = Parser.quoteIdentifier(colName);
+
+        if (tblAlias != null)
+            sql = Parser.quoteIdentifier(tblAlias) + "." + sql;
+
+        if (schema != null)
+            sql = Parser.quoteIdentifier(schema) + "." + sql;
+
+        return sql;
     }
 
     /**
      * @return Expression in from.
      */
-    public GridSqlElement expressionInFrom() {
-        return expressionInFrom;
+    public GridSqlAst expressionInFrom() {
+        return from;
+    }
+
+    /**
+     * @param from Expression in from.
+     */
+    public void expressionInFrom(GridSqlAlias from) {
+        this.from = from;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
index bb7d338..0d879a8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
@@ -26,13 +26,13 @@ import org.h2.value.ValueNull;
 /**
  * Constant value.
  */
-public class GridSqlConst extends GridSqlElement implements GridSqlValue {
+public class GridSqlConst extends GridSqlElement {
     /** */
     public static final GridSqlElement NULL = new GridSqlConst(ValueNull.INSTANCE)
         .resultType(GridSqlType.fromExpression(ValueExpression.getNull()));
 
     /** */
-    public static final GridSqlElement TRUE = new GridSqlConst(ValueBoolean.get(true))
+    public static final GridSqlConst TRUE = (GridSqlConst)new GridSqlConst(ValueBoolean.get(true))
         .resultType(GridSqlType.BOOLEAN);
 
     /** */
@@ -42,7 +42,7 @@ public class GridSqlConst extends GridSqlElement implements GridSqlValue {
      * @param val Value.
      */
     public GridSqlConst(Value val) {
-        super(Collections.<GridSqlElement>emptyList());
+        super(Collections.<GridSqlAst>emptyList());
 
         this.val = val;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
index d95c14a..00a8f22 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
@@ -17,15 +17,14 @@
 
 package org.apache.ignite.internal.processors.query.h2.sql;
 
-import java.util.Iterator;
 import java.util.List;
 
 /**
- * Abstract SQL element.
+ * Base class for all SQL AST nodes.
  */
-public abstract class GridSqlElement implements Iterable<GridSqlElement> {
+public abstract class GridSqlElement implements GridSqlAst {
     /** */
-    protected final List<GridSqlElement> children;
+    private final List<GridSqlAst> children;
 
     /** */
     private GridSqlType resultType;
@@ -33,7 +32,7 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     /**
      * @param children Initial child list.
      */
-    protected GridSqlElement(List<GridSqlElement> children) {
+    protected GridSqlElement(List<GridSqlAst> children) {
         assert children != null;
 
         this.children = children;
@@ -57,17 +56,10 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     }
 
     /**
-     * Get the SQL expression.
-     *
-     * @return the SQL expression.
-     */
-    public abstract String getSQL();
-
-    /**
      * @param expr Expr.
      * @return {@code this}.
      */
-    public GridSqlElement addChild(GridSqlElement expr) {
+    public GridSqlElement addChild(GridSqlAst expr) {
         if (expr == null)
             throw new NullPointerException();
 
@@ -76,27 +68,19 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
         return this;
     }
 
-    /**
-     * @return First child.
-     */
-    public <E extends GridSqlElement> E child() {
+    /** {@inheritDoc} */
+    @Override public <E extends GridSqlAst> E child() {
         return child(0);
     }
 
-    /**
-     * @param idx Index.
-     * @return Child.
-     */
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    public <E extends GridSqlElement> E child(int idx) {
+    @Override public <E extends GridSqlAst> E child(int idx) {
         return (E)children.get(idx);
     }
 
-    /**
-     * @param idx Index.
-     * @param child New child.
-     */
-    public void child(int idx, GridSqlElement child) {
+    /** {@inheritDoc} */
+    @Override public <E extends GridSqlAst> void child(int idx, E child) {
         if (child == null)
             throw new NullPointerException();
 
@@ -111,11 +95,6 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     }
 
     /** {@inheritDoc} */
-    @Override public Iterator<GridSqlElement> iterator() {
-        return children.iterator();
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return getSQL();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
index 4e93196..f194b07 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
@@ -66,7 +66,7 @@ public class GridSqlFunction extends GridSqlElement {
      * @param name Name.
      */
     private GridSqlFunction(String schema, GridSqlFunctionType type, String name) {
-        super(new ArrayList<GridSqlElement>());
+        super(new ArrayList<GridSqlAst>());
 
         if (name == null)
             throw new NullPointerException("name");
@@ -102,13 +102,13 @@ public class GridSqlFunction extends GridSqlElement {
         if (type == CASE) {
             buff.append(' ').append(child().getSQL());
 
-            for (int i = 1, len = children.size() - 1; i < len; i += 2) {
+            for (int i = 1, len = size() - 1; i < len; i += 2) {
                 buff.append(" WHEN ").append(child(i).getSQL());
                 buff.append(" THEN ").append(child(i + 1).getSQL());
             }
 
-            if ((children.size() & 1) == 0)
-                buff.append(" ELSE ").append(child(children.size() - 1).getSQL());
+            if ((size() & 1) == 0)
+                buff.append(" ELSE ").append(child(size() - 1).getSQL());
 
             return buff.append(" END").toString();
         }
@@ -137,9 +137,11 @@ public class GridSqlFunction extends GridSqlElement {
                 break;
 
             case TABLE:
-                for (GridSqlElement e : children) {
+                for (int i = 0; i < size(); i++) {
                     buff.appendExceptFirst(", ");
 
+                    GridSqlElement e = child(i);
+
                     // id int = ?, name varchar = ('aaa', 'bbb')
                     buff.append(((GridSqlAlias)e).alias())
                         .append(' ')
@@ -151,9 +153,9 @@ public class GridSqlFunction extends GridSqlElement {
                 break;
 
             default:
-                for (GridSqlElement e : children) {
+                for (int i = 0; i < size(); i++) {
                     buff.appendExceptFirst(", ");
-                    buff.append(e.getSQL());
+                    buff.append(child(i).getSQL());
                 }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
index f1ad2e5..afe4a53 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
@@ -27,6 +27,15 @@ import org.jetbrains.annotations.Nullable;
  */
 public class GridSqlJoin extends GridSqlElement {
     /** */
+    public static final int LEFT_TABLE_CHILD = 0;
+
+    /** */
+    public static final int RIGHT_TABLE_CHILD = 1;
+
+    /** */
+    public static final int ON_CHILD = 2;
+
+    /** */
     private boolean leftOuter;
 
     /**
@@ -36,7 +45,7 @@ public class GridSqlJoin extends GridSqlElement {
      * @param on Join condition.
      */
     public GridSqlJoin(GridSqlElement leftTbl, GridSqlElement rightTbl, boolean leftOuter, @Nullable GridSqlElement on) {
-        super(new ArrayList<GridSqlElement>(3));
+        super(new ArrayList<GridSqlAst>(3));
 
         addChild(leftTbl);
         addChild(rightTbl);
@@ -50,24 +59,38 @@ public class GridSqlJoin extends GridSqlElement {
     }
 
     /**
-     * @return Table 1.
+     * @return Left table.
      */
     public GridSqlElement leftTable() {
-        return child(0);
+        return child(LEFT_TABLE_CHILD);
+    }
+
+    /**
+     * @param tbl Right table to set.
+     */
+    public void leftTable(GridSqlElement tbl) {
+        child(LEFT_TABLE_CHILD, tbl);
     }
 
     /**
-     * @return Table 2.
+     * @return Right table.
      */
     public GridSqlElement rightTable() {
-        return child(1);
+        return child(RIGHT_TABLE_CHILD);
+    }
+
+    /**
+     * @param tbl Right table to set.
+     */
+    public void rightTable(GridSqlElement tbl) {
+        child(RIGHT_TABLE_CHILD, tbl);
     }
 
     /**
      * @return {@code JOIN ON} condition.
      */
     public GridSqlElement on() {
-        return child(2);
+        return child(ON_CHILD);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
index 8eacf02..48fc8a9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
@@ -35,7 +35,8 @@ public final class GridSqlKeyword extends GridSqlElement {
 
     /** */
     private GridSqlKeyword(String keyword) {
-        super(Collections.<GridSqlElement>emptyList());
+        super(Collections.<GridSqlAst>emptyList());
+
         this.keyword = keyword;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperation.java
index 737c5b1..4fcca16 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperation.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperation.java
@@ -32,8 +32,8 @@ public class GridSqlOperation extends GridSqlElement {
      * @param opType Operation type.
      */
     public GridSqlOperation(GridSqlOperationType opType) {
-        super(opType == IN ? new ArrayList<GridSqlElement>() :
-            new ArrayList<GridSqlElement>(opType.childrenCount()));
+        super(opType == IN ? new ArrayList<GridSqlAst>() :
+            new ArrayList<GridSqlAst>(opType.childrenCount()));
 
         this.opType = opType;
     }
@@ -53,7 +53,7 @@ public class GridSqlOperation extends GridSqlElement {
      * @param left Left.
      * @param right Right.
      */
-    public GridSqlOperation(GridSqlOperationType opType, GridSqlElement left, GridSqlElement right) {
+    public GridSqlOperation(GridSqlOperationType opType, GridSqlAst left, GridSqlAst right) {
         this(opType);
 
         addChild(left);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperationType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperationType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperationType.java
index 8d31651..07a6f6b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperationType.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlOperationType.java
@@ -156,7 +156,7 @@ public enum GridSqlOperationType {
         @Override public String getSql(GridSqlOperation operation) {
             assert operation.operationType().childrenCnt == 1;
 
-            return '(' + text + ' ' + operation.child().getSQL() + ')';
+            return '(' + text + ' ' + operation.child(0).getSQL() + ')';
         }
     }
 
@@ -178,7 +178,7 @@ public enum GridSqlOperationType {
         @Override public String getSql(GridSqlOperation operation) {
             assert operation.operationType().childrenCnt == 1;
 
-            return '(' + operation.child().getSQL() + ' ' + text + ')';
+            return '(' + operation.child(0).getSQL() + ' ' + text + ')';
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlParameter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlParameter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlParameter.java
index 0969ad2..f19e9f5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlParameter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlParameter.java
@@ -22,7 +22,7 @@ import java.util.Collections;
 /**
  * Query parameter.
  */
-public class GridSqlParameter extends GridSqlElement implements GridSqlValue {
+public class GridSqlParameter extends GridSqlElement {
     /** Index. */
     private int idx;
 
@@ -30,7 +30,7 @@ public class GridSqlParameter extends GridSqlElement implements GridSqlValue {
      * @param idx Index.
      */
     public GridSqlParameter(int idx) {
-        super(Collections.<GridSqlElement>emptyList());
+        super(Collections.<GridSqlAst>emptyList());
 
         this.idx = idx;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
index 0bb69a8..3798396 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
@@ -33,7 +33,7 @@ public class GridSqlPlaceholder extends GridSqlElement {
      * @param sql SQL.
      */
     public GridSqlPlaceholder(String sql) {
-        super(Collections.<GridSqlElement>emptyList());
+        super(Collections.<GridSqlAst>emptyList());
 
         this.sql = sql;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
index 331aa65..7d4b7f0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuery.java
@@ -23,29 +23,35 @@ import org.h2.util.StatementBuilder;
 import org.h2.util.StringUtils;
 
 /**
- * Select query.
+ * SQL Query AST.
  */
-public abstract class GridSqlQuery extends GridSqlStatement {
+public abstract class GridSqlQuery extends GridSqlStatement implements GridSqlAst {
     /** */
-    protected boolean distinct;
+    public static final int OFFSET_CHILD = 0;
+
+    /** */
+    public static final int LIMIT_CHILD = 1;
 
     /** */
     protected List<GridSqlSortColumn> sort = new ArrayList<>();
 
     /** */
-    protected GridSqlElement offset;
+    private GridSqlAst offset;
+
+    /** */
+    private boolean distinct;
 
     /**
      * @return Offset.
      */
-    public GridSqlElement offset() {
+    public GridSqlAst offset() {
         return offset;
     }
 
     /**
      * @param offset Offset.
      */
-    public void offset(GridSqlElement offset) {
+    public void offset(GridSqlAst offset) {
         this.offset = offset;
     }
 
@@ -93,7 +99,63 @@ public abstract class GridSqlQuery extends GridSqlStatement {
      * @param col Column index.
      * @return Expression for column index.
      */
-    protected abstract GridSqlElement column(int col);
+    protected abstract GridSqlAst column(int col);
+
+    /** {@inheritDoc} */
+    @Override public GridSqlType resultType() {
+        return GridSqlType.RESULT_SET;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <E extends GridSqlAst> E child() {
+        return child(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <E extends GridSqlAst> E child(int childIdx) {
+        switch (childIdx) {
+            case OFFSET_CHILD:
+                return maskNull(offset, GridSqlPlaceholder.EMPTY);
+
+            case LIMIT_CHILD:
+                return maskNull(limit, GridSqlPlaceholder.EMPTY);
+
+            default:
+                throw new IllegalStateException("Child index: " + childIdx);
+        }
+    }
+
+    /**
+     * @param x Element.
+     * @return Empty placeholder if the element is {@code null}.
+     */
+    @SuppressWarnings("unchecked")
+    protected static <E extends GridSqlAst> E maskNull(GridSqlAst x, GridSqlAst dflt) {
+        return (E)(x == null ? dflt : x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <E extends GridSqlAst> void child(int childIdx, E child) {
+        switch (childIdx) {
+            case OFFSET_CHILD:
+                offset = child;
+
+                break;
+
+            case LIMIT_CHILD:
+                limit = child;
+
+                break;
+
+            default:
+                throw new IllegalStateException("Child index: " + childIdx);
+        }
+    }
+
+    /**
+     * @return If this is a simple query with no conditions, expressions, sorting, etc...
+     */
+    public abstract boolean simpleQuery();
 
     /**
      * @param buff Statement builder.
@@ -116,13 +178,13 @@ public abstract class GridSqlQuery extends GridSqlStatement {
                 if (idx < visibleCols)
                     buff.append(idx + 1);
                 else {
-                    GridSqlElement expr = column(idx);
+                    GridSqlAst expr = column(idx);
 
                     if (expr == null) // For plain select should never be null, for union H2 itself can't parse query.
                         throw new IllegalStateException("Failed to build query: " + buff.toString());
 
                     if (expr instanceof GridSqlAlias)
-                        expr = expr.child();
+                        expr = expr.child(0);
 
                     buff.append('=').append(StringUtils.unEnclose(expr.getSQL()));
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3737407b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index d9c546c..f5ad397 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.lang.reflect.Field;
+import java.sql.PreparedStatement;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,8 +28,6 @@ import java.util.List;
 import java.util.Map;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.h2.command.Command;
 import org.h2.command.CommandContainer;
 import org.h2.command.Prepared;
@@ -60,7 +59,6 @@ import org.h2.expression.Parameter;
 import org.h2.expression.Subquery;
 import org.h2.expression.TableFunction;
 import org.h2.expression.ValueExpression;
-import org.h2.index.Index;
 import org.h2.index.ViewIndex;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.result.SortOrder;
@@ -225,6 +223,9 @@ public class GridSqlQueryParser {
     private static final Getter<JavaFunction, FunctionAlias> FUNC_ALIAS = getter(JavaFunction.class, "functionAlias");
 
     /** */
+    private static final Getter<ExpressionColumn, String> SCHEMA_NAME = getter(ExpressionColumn.class, "schemaName");
+
+    /** */
     private static final Getter<JdbcPreparedStatement, Command> COMMAND = getter(JdbcPreparedStatement.class, "command");
 
     /** */
@@ -298,12 +299,30 @@ public class GridSqlQueryParser {
     /** */
     private final IdentityHashMap<Object, Object> h2ObjToGridObj = new IdentityHashMap<>();
 
+    /** */
+    private final Map<String, Integer> optimizedTableFilterOrder;
+
+    /**
+     * We have a counter instead of a simple flag, because
+     * a flag can be reset earlier than needed in case of
+     * deep subquery expression nesting.
+     */
+    private int parsingSubQryExpression;
+
+    /**
+     * @param useOptimizedSubqry If we have to find correct order for table filters in FROM clause.
+     *                           Relies on uniqueness of table filter aliases.
+     */
+    public GridSqlQueryParser(boolean useOptimizedSubqry) {
+        optimizedTableFilterOrder = useOptimizedSubqry ? new HashMap<String, Integer>() : null;
+    }
+
     /**
      * @param stmt Prepared statement.
      * @return Parsed select.
      */
-    public static Prepared prepared(JdbcPreparedStatement stmt) {
-        Command cmd = COMMAND.get(stmt);
+    public static Prepared prepared(PreparedStatement stmt) {
+        Command cmd = COMMAND.get((JdbcPreparedStatement)stmt);
 
         assert cmd instanceof CommandContainer;
 
@@ -311,14 +330,25 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param qry Query expression to parse.
+     * @return Subquery AST.
+     */
+    private GridSqlSubquery parseQueryExpression(Query qry) {
+        parsingSubQryExpression++;
+        GridSqlQuery subQry = parseQuery(qry);
+        parsingSubQryExpression--;
+
+        return new GridSqlSubquery(subQry);
+    }
+
+    /**
      * @param filter Filter.
      */
     private GridSqlElement parseTable(TableFilter filter) {
         GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(filter);
 
         if (res == null) {
-            res = parseTable(filter.getTable(), filter.getIndex(),
-                filter.getSelect() != null ? filter.getSelect().getSQL() : null);
+            res = parseTable(filter.getTable());
 
             String alias = ALIAS.get(filter);
 
@@ -335,7 +365,7 @@ public class GridSqlQueryParser {
     /**
      * @param tbl Table.
      */
-    private GridSqlElement parseTable(Table tbl, @Nullable Index idx, String sql) {
+    private GridSqlElement parseTable(Table tbl) {
         GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(tbl);
 
         if (res == null) {
@@ -344,9 +374,7 @@ public class GridSqlQueryParser {
             else if (tbl instanceof TableView) {
                 Query qry = VIEW_QUERY.get((TableView) tbl);
 
-                Query idxQry = idx instanceof ViewIndex ? ((ViewIndex)idx).getQuery() : null;
-
-                res = new GridSqlSubquery(parse(qry, idxQry));
+                res = new GridSqlSubquery(parseQuery(qry));
             }
             else if (tbl instanceof FunctionTable)
                 res = parseExpression(FUNC_EXPR.get((FunctionTable)tbl), false);
@@ -357,8 +385,7 @@ public class GridSqlQueryParser {
                 res.addChild(parseExpression(RANGE_MAX.get((RangeTable)tbl), false));
             }
             else
-                assert0(false, "Unexpected Table implementation [cls=" + tbl.getClass().getSimpleName() + ", " +
-                    "sql=" + sql + ']');
+                assert0(false, "Unexpected Table implementation [cls=" + tbl.getClass().getSimpleName() + ']');
 
             h2ObjToGridObj.put(tbl, res);
         }
@@ -369,7 +396,7 @@ public class GridSqlQueryParser {
     /**
      * @param select Select.
      */
-    public GridSqlSelect parse(Select select, @Nullable Query idxQry) {
+    private GridSqlSelect parseSelect(Select select) {
         GridSqlSelect res = (GridSqlSelect)h2ObjToGridObj.get(select);
 
         if (res != null)
@@ -384,26 +411,39 @@ public class GridSqlQueryParser {
         Expression where = CONDITION.get(select);
         res.where(parseExpression(where, false));
 
-        GridSqlElement from = null;
+        ArrayList<TableFilter> tableFilters = new ArrayList<>();
 
         TableFilter filter = select.getTopTableFilter();
 
-        if (idxQry instanceof Select)
-            filter = ((Select)idxQry).getTopTableFilter();
-
         do {
             assert0(filter != null, select);
             assert0(filter.getNestedJoin() == null, select);
 
-            GridSqlElement gridFilter = parseTable(filter);
+            // Can use optimized join order only if we are not inside of an expression.
+            if (parsingSubQryExpression == 0 && optimizedTableFilterOrder != null) {
+                String tblAlias = filter.getTableAlias();
+                int idx = optimizedTableFilterOrder.get(tblAlias);
 
-            from = from == null ? gridFilter : new GridSqlJoin(from, gridFilter, filter.isJoinOuter(),
-                parseExpression(filter.getJoinCondition(), false));
+                setElementAt(tableFilters, idx, filter);
+            }
+            else
+                tableFilters.add(filter);
 
             filter = filter.getJoin();
         }
         while (filter != null);
 
+        // Build FROM clause from correctly ordered table filters.
+        GridSqlElement from = null;
+
+        for (int i = 0; i < tableFilters.size(); i++) {
+            TableFilter f = tableFilters.get(i);
+            GridSqlElement gridFilter = parseTable(f);
+
+            from = from == null ? gridFilter : new GridSqlJoin(from, gridFilter, f.isJoinOuter(),
+                parseExpression(f.getJoinCondition(), false));
+        }
+
         res.from(from);
 
         ArrayList<Expression> expressions = select.getExpressions();
@@ -430,6 +470,20 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param list List.
+     * @param idx Index.
+     * @param x Element.
+     */
+    private static <Z> void setElementAt(List<Z> list, int idx, Z x) {
+        while (list.size() <= idx)
+            list.add(null);
+
+        assert0(list.get(idx) == null, "Element already set: " + idx);
+
+        list.set(idx, x);
+    }
+
+    /**
      * @param merge Merge.
      * @see <a href="http://h2database.com/html/grammar.html#merge">H2 merge spec</a>
      */
@@ -443,7 +497,7 @@ public class GridSqlQueryParser {
         h2ObjToGridObj.put(merge, res);
 
         Table srcTbl = MERGE_TABLE.get(merge);
-        GridSqlElement tbl = parseTable(srcTbl, null, merge.getSQL());
+        GridSqlElement tbl = parseTable(srcTbl);
 
         res.into(tbl);
 
@@ -452,7 +506,7 @@ public class GridSqlQueryParser {
         GridSqlColumn[] cols = new GridSqlColumn[srcCols.length];
 
         for (int i = 0; i < srcCols.length; i++) {
-            cols[i] = new GridSqlColumn(srcCols[i], tbl, srcCols[i].getName(), srcCols[i].getSQL());
+            cols[i] = new GridSqlColumn(srcCols[i], tbl, null, null, srcCols[i].getName());
 
             cols[i].resultType(fromColumn(srcCols[i]));
         }
@@ -463,7 +517,7 @@ public class GridSqlQueryParser {
 
         GridSqlColumn[] keys = new GridSqlColumn[srcKeys.length];
         for (int i = 0; i < srcKeys.length; i++)
-            keys[i] = new GridSqlColumn(srcKeys[i], tbl, srcKeys[i].getName(), srcKeys[i].getSQL());
+            keys[i] = new GridSqlColumn(srcKeys[i], tbl, null, null, srcKeys[i].getName());
         res.keys(keys);
 
         List<Expression[]> srcRows = MERGE_ROWS.get(merge);
@@ -481,7 +535,7 @@ public class GridSqlQueryParser {
         }
         else {
             res.rows(Collections.<GridSqlElement[]>emptyList());
-            res.query(parse(MERGE_QUERY.get(merge), null));
+            res.query(parseQuery(MERGE_QUERY.get(merge)));
         }
 
         return res;
@@ -501,7 +555,7 @@ public class GridSqlQueryParser {
         h2ObjToGridObj.put(insert, res);
 
         Table srcTbl = INSERT_TABLE.get(insert);
-        GridSqlElement tbl = parseTable(srcTbl, null, insert.getSQL());
+        GridSqlElement tbl = parseTable(srcTbl);
 
         res.into(tbl).
             direct(INSERT_DIRECT.get(insert)).
@@ -511,7 +565,7 @@ public class GridSqlQueryParser {
         GridSqlColumn[] cols = new GridSqlColumn[srcCols.length];
 
         for (int i = 0; i < srcCols.length; i++) {
-            cols[i] = new GridSqlColumn(srcCols[i], tbl, srcCols[i].getName(), srcCols[i].getSQL());
+            cols[i] = new GridSqlColumn(srcCols[i], tbl, null, null, srcCols[i].getName());
 
             cols[i].resultType(fromColumn(srcCols[i]));
         }
@@ -533,7 +587,7 @@ public class GridSqlQueryParser {
         }
         else {
             res.rows(Collections.<GridSqlElement[]>emptyList());
-            res.query(parse(INSERT_QUERY.get(insert), null));
+            res.query(parseQuery(INSERT_QUERY.get(insert)));
         }
 
         return res;
@@ -581,7 +635,7 @@ public class GridSqlQueryParser {
         LinkedHashMap<String, GridSqlElement> set = new LinkedHashMap<>(srcSet.size());
 
         for (Column c : srcCols) {
-            GridSqlColumn col = new GridSqlColumn(c, tbl, c.getName(), c.getSQL());
+            GridSqlColumn col = new GridSqlColumn(c, tbl, null, null, c.getName());
             col.resultType(fromColumn(c));
             cols.add(col);
             set.put(col.columnName(), parseExpression(srcSet.get(c), true));
@@ -631,65 +685,54 @@ public class GridSqlQueryParser {
     }
 
     /**
-     * @param qry Prepared.
-     * @return Query.
-     */
-    public GridSqlStatement parse(Prepared qry) {
-        return parse(qry, null);
-    }
-
-    /**
-     * @param qry Select.
+     * @param stmt Prepared statement.
+     * @return Parsed AST.
      */
-    public GridSqlStatement parse(Prepared qry, @Nullable Query idxQry) {
-        assert qry != null;
-
-        if (qry instanceof Query)
-            return parse((Query)qry, idxQry);
-
-        if (qry instanceof Merge)
-            return parseMerge((Merge)qry);
+    public final GridSqlStatement parse(Prepared stmt) {
+        if (stmt instanceof Query) {
+            if (optimizedTableFilterOrder != null)
+                collectOptimizedTableFiltersOrder((Query)stmt);
 
-        if (qry instanceof Insert)
-            return parseInsert((Insert)qry);
+            return parseQuery((Query)stmt);
+        }
 
-        if (qry instanceof Delete)
-            return parseDelete((Delete)qry);
+        if (stmt instanceof Merge)
+            return parseMerge((Merge)stmt);
 
-        if (qry instanceof Update)
-            return parseUpdate((Update)qry);
+        if (stmt instanceof Insert)
+            return parseInsert((Insert)stmt);
 
-        if (qry instanceof Explain) {
-            GridSqlStatement stmt = parse(EXPLAIN_COMMAND.get((Explain) qry));
+        if (stmt instanceof Delete)
+            return parseDelete((Delete)stmt);
 
-            if (!(stmt instanceof GridSqlQuery))
-                throw new IgniteSQLException("EXPLAIN is not supported for DML statement: " + qry,
-                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        if (stmt instanceof Update)
+            return parseUpdate((Update)stmt);
 
-            return stmt.explain(true);
-        }
+        if (stmt instanceof Explain)
+            return parse(EXPLAIN_COMMAND.get((Explain)stmt)).explain(true);
 
-        throw new IgniteSQLException("Unsupported statement: " + qry, IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        throw new CacheException("Unsupported SQL statement: " + stmt);
     }
 
     /**
      * @param qry Query.
-     * @return Grid SQL query.
+     * @return Parsed query AST.
      */
-    private GridSqlQuery parse(Query qry, @Nullable Query idxQry) {
+    private GridSqlQuery parseQuery(Query qry) {
         if (qry instanceof Select)
-            return parse((Select)qry, idxQry);
+            return parseSelect((Select)qry);
 
         if (qry instanceof SelectUnion)
-            return parse((SelectUnion)qry);
+            return parseUnion((SelectUnion)qry);
 
         throw new UnsupportedOperationException("Unknown query type: " + qry);
     }
 
     /**
      * @param union Select.
+     * @return Parsed AST.
      */
-    public GridSqlUnion parse(SelectUnion union) {
+    private GridSqlUnion parseUnion(SelectUnion union) {
         GridSqlUnion res = (GridSqlUnion)h2ObjToGridObj.get(union);
 
         if (res != null)
@@ -697,8 +740,8 @@ public class GridSqlQueryParser {
 
         res = new GridSqlUnion();
 
-        res.right(parse(union.getRight(), null));
-        res.left(parse(union.getLeft(), null));
+        res.right(parseQuery(union.getRight()));
+        res.left(parseQuery(union.getLeft()));
 
         res.unionType(union.getUnionType());
 
@@ -736,6 +779,44 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param qry Query.
+     */
+    private void collectOptimizedTableFiltersOrder(Query qry) {
+        if (qry instanceof SelectUnion) {
+            collectOptimizedTableFiltersOrder(((SelectUnion)qry).getLeft());
+            collectOptimizedTableFiltersOrder(((SelectUnion)qry).getRight());
+        }
+        else {
+            Select select = (Select)qry;
+
+            TableFilter filter = select.getTopTableFilter();
+
+            int i = 0;
+
+            do {
+                assert0(filter != null, select);
+                assert0(filter.getNestedJoin() == null, select);
+
+                // Here all the table filters must have generated unique aliases,
+                // thus we can store them in the same map for all the subqueries.
+                optimizedTableFilterOrder.put(filter.getTableAlias(), i++);
+
+                Table tbl = filter.getTable();
+
+                // Go down and collect inside of optimized subqueries.
+                if (tbl instanceof TableView) {
+                    ViewIndex viewIdx = (ViewIndex)filter.getIndex();
+
+                    collectOptimizedTableFiltersOrder(viewIdx.getQuery());
+                }
+
+                filter = filter.getJoin();
+            }
+            while (filter != null);
+        }
+    }
+
+    /**
      * @param expression Expression.
      * @param calcTypes Calculate types for all the expressions.
      * @return Parsed expression.
@@ -746,8 +827,9 @@ public class GridSqlQueryParser {
 
             return new GridSqlColumn(expCol.getColumn(),
                 parseTable(expCol.getTableFilter()),
-                expression.getColumnName(),
-                expression.getSQL());
+                SCHEMA_NAME.get(expCol),
+                expCol.getOriginalTableAliasName(),
+                expCol.getColumnName());
         }
 
         if (expression instanceof Alias)
@@ -812,9 +894,7 @@ public class GridSqlQueryParser {
         if (expression instanceof Subquery) {
             Query qry = ((Subquery)expression).getQuery();
 
-            assert0(qry instanceof Select, expression);
-
-            return new GridSqlSubquery(parse(qry, null));
+            return parseQueryExpression(qry);
         }
 
         if (expression instanceof ConditionIn) {
@@ -856,9 +936,7 @@ public class GridSqlQueryParser {
 
             Query qry = QUERY.get((ConditionInSelect)expression);
 
-            assert0(qry instanceof Select, qry);
-
-            res.addChild(new GridSqlSubquery(parse(qry, null)));
+            res.addChild(parseQueryExpression(qry));
 
             return res;
         }