You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/08/07 11:24:43 UTC

[4/7] incubator-ignite git commit: # ignite-1142

# ignite-1142

Signed-off-by: Yakov Zhdanov <yz...@gridgain.com>


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

Branch: refs/heads/ignite-gg-9615
Commit: d7dd4a0272aff8e00324254ac97f47393d05f70c
Parents: 63944d4
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Aug 6 16:24:56 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Aug 6 16:27:59 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheSqlQuery.java          |  33 ++-
 .../cache/query/GridCacheTwoStepQuery.java      |  34 +--
 .../processors/query/h2/IgniteH2Indexing.java   |  27 +-
 .../processors/query/h2/sql/GridSqlElement.java |  18 +-
 .../query/h2/sql/GridSqlFunction.java           |  17 +-
 .../processors/query/h2/sql/GridSqlQuery.java   |   4 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  94 ++++---
 .../query/h2/sql/GridSqlQuerySplitter.java      | 117 +++++----
 .../processors/query/h2/sql/GridSqlSelect.java  |  76 +++---
 .../processors/query/h2/sql/GridSqlType.java    |  24 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 211 ++++++---------
 .../query/h2/twostep/GridThreadLocalTable.java  | 262 +++++++++++++++++++
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   2 +-
 14 files changed, 614 insertions(+), 307 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/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 7a0e140..d5eb379 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
@@ -26,6 +26,7 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 
 import java.nio.*;
+import java.util.*;
 
 /**
  * Query.
@@ -38,9 +39,6 @@ public class GridCacheSqlQuery implements Message {
     public static final Object[] EMPTY_PARAMS = {};
 
     /** */
-    private String alias;
-
-    /** */
     @GridToStringInclude
     private String qry;
 
@@ -52,6 +50,11 @@ public class GridCacheSqlQuery implements Message {
     /** */
     private byte[] paramsBytes;
 
+    /** */
+    @GridToStringInclude
+    @GridDirectTransient
+    private LinkedHashMap<String, ?> columns;
+
     /**
      * For {@link Message}.
      */
@@ -60,24 +63,32 @@ public class GridCacheSqlQuery implements Message {
     }
 
     /**
-     * @param alias Alias.
      * @param qry Query.
      * @param params Query parameters.
      */
-    public GridCacheSqlQuery(String alias, String qry, Object[] params) {
+    public GridCacheSqlQuery(String qry, Object[] params) {
         A.ensure(!F.isEmpty(qry), "qry must not be empty");
 
-        this.alias = alias;
         this.qry = qry;
 
         this.params = F.isEmpty(params) ? EMPTY_PARAMS : params;
     }
 
     /**
-     * @return Alias.
+     * @return Columns.
      */
-    public String alias() {
-        return alias;
+    public LinkedHashMap<String, ?> columns() {
+        return columns;
+    }
+
+    /**
+     * @param columns Columns.
+     * @return {@code this}.
+     */
+    public GridCacheSqlQuery columns(LinkedHashMap<String, ?> columns) {
+        this.columns = columns;
+
+        return this;
     }
 
     /**
@@ -138,7 +149,7 @@ public class GridCacheSqlQuery implements Message {
 
         switch (writer.state()) {
             case 0:
-                if (!writer.writeString("alias", alias))
+                if (!writer.writeString("alias", null))
                     return false;
 
                 writer.incrementState();
@@ -169,7 +180,7 @@ public class GridCacheSqlQuery implements Message {
 
         switch (reader.state()) {
             case 0:
-                alias = reader.readString("alias");
+                reader.readString("alias");
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 1dacd10..83a79e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -17,10 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.util.*;
@@ -34,11 +31,11 @@ public class GridCacheTwoStepQuery {
 
     /** */
     @GridToStringInclude
-    private Map<String, GridCacheSqlQuery> mapQrys;
+    private List<GridCacheSqlQuery> mapQrys = new ArrayList<>();
 
     /** */
     @GridToStringInclude
-    private GridCacheSqlQuery reduce;
+    private GridCacheSqlQuery rdc;
 
     /** */
     private int pageSize = DFLT_PAGE_SIZE;
@@ -51,13 +48,14 @@ public class GridCacheTwoStepQuery {
 
     /**
      * @param spaces All spaces accessed in query.
-     * @param qry Reduce query.
-     * @param params Reduce query parameters.
+     * @param rdc Reduce query.
      */
-    public GridCacheTwoStepQuery(Set<String> spaces, String qry, Object ... params) {
+    public GridCacheTwoStepQuery(Set<String> spaces, GridCacheSqlQuery rdc) {
+        assert rdc != null;
+
         this.spaces = spaces;
 
-        reduce = new GridCacheSqlQuery(null, qry, params);
+        this.rdc = rdc;
     }
 
     /**
@@ -89,32 +87,24 @@ public class GridCacheTwoStepQuery {
     }
 
     /**
-     * @param alias Alias.
      * @param qry SQL Query.
-     * @param params Query parameters.
      */
-    public void addMapQuery(String alias, String qry, Object ... params) {
-        A.ensure(!F.isEmpty(alias), "alias must not be empty");
-
-        if (mapQrys == null)
-            mapQrys = new GridLeanMap<>();
-
-        if (mapQrys.put(alias, new GridCacheSqlQuery(alias, qry, params)) != null)
-            throw new IgniteException("Failed to add query, alias already exists: " + alias + ".");
+    public void addMapQuery(GridCacheSqlQuery qry) {
+        mapQrys.add(qry);
     }
 
     /**
      * @return Reduce query.
      */
     public GridCacheSqlQuery reduceQuery() {
-        return reduce;
+        return rdc;
     }
 
     /**
      * @return Map queries.
      */
-    public Collection<GridCacheSqlQuery> mapQueries() {
-        return mapQrys.values();
+    public List<GridCacheSqlQuery> mapQueries() {
+        return mapQrys;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/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 c76dbe7..dc61d76 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
@@ -599,6 +599,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             String name = rsMeta.getColumnLabel(i);
             String type = rsMeta.getColumnClassName(i);
 
+            if (type == null) // Expression always returns NULL.
+                type = Void.class.getName();
+
             meta.add(new SqlFieldMetadata(schemaName, typeName, name, type));
         }
 
@@ -852,6 +855,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             throw new CacheException("Failed to parse query: " + sqlQry, e);
         }
 
+        try {
+            bindParameters(stmt, F.asList(qry.getArgs()));
+        }
+        catch (IgniteCheckedException e) {
+            throw new CacheException("Failed to bind parameters: [qry=" + sqlQry + ", params=" +
+                Arrays.deepToString(qry.getArgs()) + "]", e);
+        }
+
         GridCacheTwoStepQuery twoStepQry;
         List<GridQueryFieldMetadata> meta;
 
@@ -1318,14 +1329,20 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             }
         }
 
-        executeStatement("INFORMATION_SCHEMA", "SHUTDOWN");
-
         for (Connection c : conns)
             U.close(c, log);
 
         conns.clear();
         schemas.clear();
 
+        try (Connection c = DriverManager.getConnection(dbUrl);
+             Statement s = c.createStatement()) {
+            s.execute("SHUTDOWN");
+        }
+        catch (SQLException e) {
+            U.error(log, "Failed to shutdown database.", e);
+        }
+
         if (log.isDebugEnabled())
             log.debug("Cache query index stopped.");
     }
@@ -1341,9 +1358,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         createSchema(schema);
 
-        executeStatement(schema, "CREATE ALIAS " + GridSqlQuerySplitter.TABLE_FUNC_NAME +
-            " NOBUFFER FOR \"" + GridReduceQueryExecutor.class.getName() + ".mergeTableFunction\"");
-
         createSqlFunctions(schema, ccfg.getSqlFunctionClasses());
     }
 
@@ -1881,8 +1895,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @param type Type.
          */
         SqlFieldMetadata(@Nullable String schemaName, @Nullable String typeName, String name, String type) {
-            assert name != null;
-            assert type != null;
+            assert name != null && type != null : schemaName + " | " + typeName + " | " + name + " | " + type;
 
             this.schemaName = schemaName;
             this.typeName = typeName;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/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 44705de..0f98a33 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
@@ -27,7 +27,7 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     protected List<GridSqlElement> children;
 
     /** */
-    private GridSqlType expressionResultType;
+    private GridSqlType resultType;
 
     /**
      * @param children Initial child list.
@@ -41,15 +41,18 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     /**
      * @return Optional expression result type (if this is an expression and result type is known).
      */
-    public GridSqlType expressionResultType() {
-        return expressionResultType;
+    public GridSqlType resultType() {
+        return resultType;
     }
 
     /**
      * @param type Optional expression result type (if this is an expression and result type is known).
+     * @return {@code this}.
      */
-    public void expressionResultType(GridSqlType type) {
-        expressionResultType = type;
+    public GridSqlElement resultType(GridSqlType type) {
+        resultType = type;
+
+        return this;
     }
 
     /**
@@ -110,4 +113,9 @@ public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     @Override public Iterator<GridSqlElement> iterator() {
         return children.iterator();
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return getSQL();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/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 c41bbb7..77039b0 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
@@ -50,9 +50,6 @@ public class GridSqlFunction extends GridSqlElement {
     /** */
     protected final GridSqlFunctionType type;
 
-    /**  */
-    private String castType;
-
     /**
      * @param type Function type.
      */
@@ -87,16 +84,6 @@ public class GridSqlFunction extends GridSqlElement {
         this(schema, TYPE_MAP.get(name), name);
     }
 
-    /**
-     * @param castType Type for {@link GridSqlFunctionType#CAST} function.
-     * @return {@code this}.
-     */
-    public GridSqlFunction setCastType(String castType) {
-        this.castType = castType;
-
-        return this;
-    }
-
     /** {@inheritDoc} */
     @Override public String getSQL() {
         StatementBuilder buff = new StatementBuilder();
@@ -123,12 +110,16 @@ public class GridSqlFunction extends GridSqlElement {
         buff.append('(');
 
         if (type == CAST) {
+            String castType = resultType().sql();
+
             assert !F.isEmpty(castType) : castType;
             assert size() == 1;
 
             buff.append(child().getSQL()).append(" AS ").append(castType);
         }
         else if (type == CONVERT) {
+            String castType = resultType().sql();
+
             assert !F.isEmpty(castType) : castType;
             assert size() == 1;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/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 ad13dfe..329304a 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
@@ -134,7 +134,7 @@ public abstract class GridSqlQuery {
      * @param col Column index.
      * @return Expression for column index.
      */
-    protected abstract GridSqlElement expression(int col);
+    protected abstract GridSqlElement column(int col);
 
     /**
      * @param buff Statement builder.
@@ -157,7 +157,7 @@ public abstract class GridSqlQuery {
                 if (idx < visibleCols)
                     buff.append(idx + 1);
                 else {
-                    GridSqlElement expr = expression(idx);
+                    GridSqlElement 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());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/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 4267b4a..a52f3b0 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
@@ -22,7 +22,6 @@ import org.h2.command.*;
 import org.h2.command.dml.*;
 import org.h2.engine.*;
 import org.h2.expression.*;
-import org.h2.expression.Parameter;
 import org.h2.jdbc.*;
 import org.h2.result.*;
 import org.h2.table.*;
@@ -205,12 +204,12 @@ public class GridSqlQueryParser {
                 res = new GridSqlSubquery(parse(qry));
             }
             else if (tbl instanceof FunctionTable)
-                res = parseExpression(FUNC_EXPR.get((FunctionTable)tbl));
+                res = parseExpression(FUNC_EXPR.get((FunctionTable)tbl), false);
             else if (tbl instanceof RangeTable) {
                 res = new GridSqlFunction(GridSqlFunctionType.SYSTEM_RANGE);
 
-                res.addChild(parseExpression(RANGE_MIN.get((RangeTable)tbl)));
-                res.addChild(parseExpression(RANGE_MAX.get((RangeTable)tbl)));
+                res.addChild(parseExpression(RANGE_MIN.get((RangeTable)tbl), false));
+                res.addChild(parseExpression(RANGE_MAX.get((RangeTable)tbl), false));
             }
             else
                 assert0(false, filter.getSelect().getSQL());
@@ -242,7 +241,7 @@ public class GridSqlQueryParser {
         res.distinct(select.isDistinct());
 
         Expression where = CONDITION.get(select);
-        res.where(parseExpression(where));
+        res.where(parseExpression(where, false));
 
         Set<TableFilter> allFilters = new HashSet<>(select.getTopFilters());
 
@@ -257,7 +256,7 @@ public class GridSqlQueryParser {
             GridSqlElement gridFilter = parseTable(filter);
 
             from = from == null ? gridFilter : new GridSqlJoin(from, gridFilter, filter.isJoinOuter(),
-                parseExpression(filter.getJoinCondition()));
+                parseExpression(filter.getJoinCondition(), false));
 
             allFilters.remove(filter);
 
@@ -272,7 +271,7 @@ public class GridSqlQueryParser {
         ArrayList<Expression> expressions = select.getExpressions();
 
         for (int i = 0; i < expressions.size(); i++)
-            res.addSelectExpression(parseExpression(expressions.get(i)), i < select.getColumnCount());
+            res.addColumn(parseExpression(expressions.get(i), true), i < select.getColumnCount());
 
         int[] grpIdx = GROUP_INDEXES.get(select);
 
@@ -286,8 +285,8 @@ public class GridSqlQueryParser {
 
         processSortOrder(select.getSortOrder(), res);
 
-        res.limit(parseExpression(select.getLimit()));
-        res.offset(parseExpression(select.getOffset()));
+        res.limit(parseExpression(select.getLimit(), false));
+        res.offset(parseExpression(select.getOffset(), false));
 
         return res;
     }
@@ -346,8 +345,8 @@ public class GridSqlQueryParser {
 
         res.unionType(union.getUnionType());
 
-        res.limit(parseExpression(union.getLimit()));
-        res.offset(parseExpression(union.getOffset()));
+        res.limit(parseExpression(union.getLimit(), false));
+        res.offset(parseExpression(union.getOffset(), false));
 
         processSortOrder(UNION_SORT.get(union), res);
 
@@ -358,22 +357,29 @@ public class GridSqlQueryParser {
 
     /**
      * @param expression Expression.
+     * @param calcTypes Calculate types for all the expressions.
+     * @return Parsed expression.
      */
-    private GridSqlElement parseExpression(@Nullable Expression expression) {
+    private GridSqlElement parseExpression(@Nullable Expression expression, boolean calcTypes) {
         if (expression == null)
             return null;
 
         GridSqlElement res = (GridSqlElement)h2ObjToGridObj.get(expression);
 
         if (res == null) {
-            res = parseExpression0(expression);
+            res = parseExpression0(expression, calcTypes);
 
-            if (expression.getType() != Value.UNKNOWN) {
-                Column c = new Column(null, expression.getType(), expression.getPrecision(), expression.getScale(),
-                    expression.getDisplaySize());
+            if (calcTypes) {
+                GridSqlType type = GridSqlType.UNKNOWN;
 
-                res.expressionResultType(new GridSqlType(c.getType(), c.getScale(), c.getPrecision(), c.getDisplaySize(),
-                    c.getCreateSQL()));
+                if (expression.getType() != Value.UNKNOWN) {
+                    Column c = new Column(null, expression.getType(), expression.getPrecision(), expression.getScale(),
+                        expression.getDisplaySize());
+
+                    type = new GridSqlType(c.getType(), c.getScale(), c.getPrecision(), c.getDisplaySize(), c.getCreateSQL());
+                }
+
+                res.resultType(type);
             }
 
             h2ObjToGridObj.put(expression, res);
@@ -384,8 +390,10 @@ public class GridSqlQueryParser {
 
     /**
      * @param expression Expression.
+     * @param calcTypes Calculate types for all the expressions.
+     * @return Parsed expression.
      */
-    private GridSqlElement parseExpression0(Expression expression) {
+    private GridSqlElement parseExpression0(Expression expression, boolean calcTypes) {
         if (expression instanceof ExpressionColumn) {
             TableFilter tblFilter = ((ExpressionColumn)expression).getTableFilter();
 
@@ -395,7 +403,8 @@ public class GridSqlQueryParser {
         }
 
         if (expression instanceof Alias)
-            return new GridSqlAlias(expression.getAlias(), parseExpression(expression.getNonAliasExpression()), true);
+            return new GridSqlAlias(expression.getAlias(),
+                parseExpression(expression.getNonAliasExpression(), calcTypes), true);
 
         if (expression instanceof ValueExpression)
             return new GridSqlConst(expression.getValue(null));
@@ -408,12 +417,13 @@ public class GridSqlQueryParser {
             if (type == Operation.NEGATE) {
                 assert OPERATION_RIGHT.get(operation) == null;
 
-                return new GridSqlOperation(GridSqlOperationType.NEGATE, parseExpression(OPERATION_LEFT.get(operation)));
+                return new GridSqlOperation(GridSqlOperationType.NEGATE,
+                    parseExpression(OPERATION_LEFT.get(operation), calcTypes));
             }
 
             return new GridSqlOperation(OPERATION_OP_TYPES[type],
-                parseExpression(OPERATION_LEFT.get(operation)),
-                parseExpression(OPERATION_RIGHT.get(operation)));
+                parseExpression(OPERATION_LEFT.get(operation), calcTypes),
+                parseExpression(OPERATION_RIGHT.get(operation), calcTypes));
         }
 
         if (expression instanceof Comparison) {
@@ -423,18 +433,18 @@ public class GridSqlQueryParser {
 
             assert opType != null : COMPARISON_TYPE.get(cmp);
 
-            GridSqlElement left = parseExpression(COMPARISON_LEFT.get(cmp));
+            GridSqlElement left = parseExpression(COMPARISON_LEFT.get(cmp), calcTypes);
 
             if (opType.childrenCount() == 1)
                 return new GridSqlOperation(opType, left);
 
-            GridSqlElement right = parseExpression(COMPARISON_RIGHT.get(cmp));
+            GridSqlElement right = parseExpression(COMPARISON_RIGHT.get(cmp), calcTypes);
 
             return new GridSqlOperation(opType, left, right);
         }
 
         if (expression instanceof ConditionNot)
-            return new GridSqlOperation(NOT, parseExpression(expression.getNotIfPossible(null)));
+            return new GridSqlOperation(NOT, parseExpression(expression.getNotIfPossible(null), calcTypes));
 
         if (expression instanceof ConditionAndOr) {
             ConditionAndOr andOr = (ConditionAndOr)expression;
@@ -444,7 +454,7 @@ public class GridSqlQueryParser {
             assert type == ConditionAndOr.AND || type == ConditionAndOr.OR;
 
             return new GridSqlOperation(type == ConditionAndOr.AND ? AND : OR,
-                parseExpression(ANDOR_LEFT.get(andOr)), parseExpression(ANDOR_RIGHT.get(andOr)));
+                parseExpression(ANDOR_LEFT.get(andOr), calcTypes), parseExpression(ANDOR_RIGHT.get(andOr), calcTypes));
         }
 
         if (expression instanceof Subquery) {
@@ -458,12 +468,12 @@ public class GridSqlQueryParser {
         if (expression instanceof ConditionIn) {
             GridSqlOperation res = new GridSqlOperation(IN);
 
-            res.addChild(parseExpression(LEFT_CI.get((ConditionIn)expression)));
+            res.addChild(parseExpression(LEFT_CI.get((ConditionIn)expression), calcTypes));
 
             List<Expression> vals = VALUE_LIST_CI.get((ConditionIn)expression);
 
             for (Expression val : vals)
-                res.addChild(parseExpression(val));
+                res.addChild(parseExpression(val, calcTypes));
 
             return res;
         }
@@ -471,12 +481,12 @@ public class GridSqlQueryParser {
         if (expression instanceof ConditionInConstantSet) {
             GridSqlOperation res = new GridSqlOperation(IN);
 
-            res.addChild(parseExpression(LEFT_CICS.get((ConditionInConstantSet) expression)));
+            res.addChild(parseExpression(LEFT_CICS.get((ConditionInConstantSet)expression), calcTypes));
 
             List<Expression> vals = VALUE_LIST_CICS.get((ConditionInConstantSet)expression);
 
             for (Expression val : vals)
-                res.addChild(parseExpression(val));
+                res.addChild(parseExpression(val, calcTypes));
 
             return res;
         }
@@ -490,7 +500,7 @@ public class GridSqlQueryParser {
             assert0(!all, expression);
             assert0(compareType == Comparison.EQUAL, expression);
 
-            res.addChild(parseExpression(LEFT_CIS.get((ConditionInSelect) expression)));
+            res.addChild(parseExpression(LEFT_CIS.get((ConditionInSelect) expression), calcTypes));
 
             Query qry = QUERY.get((ConditionInSelect)expression);
 
@@ -506,8 +516,9 @@ public class GridSqlQueryParser {
 
             boolean regexp = REGEXP_CL.get((CompareLike)expression);
 
-            return new GridSqlOperation(regexp ? REGEXP : LIKE, parseExpression(LEFT.get((CompareLike) expression)),
-                parseExpression(RIGHT.get((CompareLike) expression)));
+            return new GridSqlOperation(regexp ? REGEXP : LIKE,
+                parseExpression(LEFT.get((CompareLike)expression), calcTypes),
+                parseExpression(RIGHT.get((CompareLike)expression), calcTypes));
         }
 
         if (expression instanceof Function) {
@@ -524,13 +535,16 @@ public class GridSqlQueryParser {
                         res.addChild(GridSqlPlaceholder.EMPTY);
                     }
                     else
-                        res.addChild(parseExpression(arg));
+                        res.addChild(parseExpression(arg, calcTypes));
                 }
             }
 
-            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT)
-                res.setCastType(new Column(null, f.getType(), f.getPrecision(), f.getScale(), f.getDisplaySize())
-                    .getCreateSQL());
+            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT) {
+                Column c = new Column(null, f.getType(), f.getPrecision(), f.getScale(), f.getDisplaySize());
+
+                res.resultType(new GridSqlType(c.getType(), c.getScale(), c.getPrecision(),
+                    c.getDisplaySize(), c.getCreateSQL()));
+            }
 
             return res;
         }
@@ -544,7 +558,7 @@ public class GridSqlQueryParser {
 
             if (f.getArgs() != null) {
                 for (Expression arg : f.getArgs())
-                    res.addChild(parseExpression(arg));
+                    res.addChild(parseExpression(arg, calcTypes));
             }
 
             return res;
@@ -560,7 +574,7 @@ public class GridSqlQueryParser {
             Expression on = ON.get((Aggregate)expression);
 
             if (on != null)
-                res.addChild(parseExpression(on));
+                res.addChild(parseExpression(on, calcTypes));
 
             return res;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 502366d..2f8bcdd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.query.h2.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.h2.jdbc.*;
-import org.h2.value.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -35,20 +34,20 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlPlacehol
  */
 public class GridSqlQuerySplitter {
     /** */
-    private static final String TABLE_PREFIX = "__T";
+    private static final String TABLE_SCHEMA = "PUBLIC";
 
     /** */
-    private static final String COLUMN_PREFIX = "__C";
+    private static final String TABLE_PREFIX = "__T";
 
     /** */
-    public static final String TABLE_FUNC_NAME = "__Z0";
+    private static final String COLUMN_PREFIX = "__C";
 
     /**
      * @param idx Index of table.
-     * @return Table name.
+     * @return Table.
      */
-    private static String table(int idx) {
-        return TABLE_PREFIX + idx;
+    public static GridSqlTable table(int idx) {
+        return new GridSqlTable(TABLE_SCHEMA, TABLE_PREFIX + idx);
     }
 
     /**
@@ -88,7 +87,9 @@ public class GridSqlQuerySplitter {
 
         int c = 0;
 
-        for (GridSqlElement expr : left.select(true)) {
+        for (GridSqlElement expr : left.columns(true)) {
+            GridSqlType type = expr.resultType();
+
             String colName;
 
             if (expr instanceof GridSqlAlias)
@@ -101,12 +102,14 @@ public class GridSqlQuerySplitter {
                 expr = alias(colName, expr);
 
                 // Set generated alias to the expression.
-                left.setSelectExpression(c, expr);
+                left.setColumn(c, expr);
             }
 
             GridSqlColumn col = column(colName);
 
-            wrapQry.addSelectExpression(col, true);
+            col.resultType(type);
+
+            wrapQry.addColumn(col, true);
 
             c++;
         }
@@ -137,17 +140,15 @@ public class GridSqlQuerySplitter {
         // nullifying or updating things, have to make sure that we will not need them in the original form later.
         final GridSqlSelect mapQry = wrapUnion(collectAllSpaces(GridSqlQueryParser.parse(stmt), spaces));
 
-        final String mergeTable = TABLE_FUNC_NAME + "()"; // table(0); TODO IGNITE-1142
-
         final boolean explain = mapQry.explain();
 
         mapQry.explain(false);
 
-        GridSqlSelect rdcQry = new GridSqlSelect().from(new GridSqlFunction(null, TABLE_FUNC_NAME)); // table(mergeTable)); TODO IGNITE-1142
+        GridSqlSelect rdcQry = new GridSqlSelect().from(table(0));
 
         // Split all select expressions into map-reduce parts.
         List<GridSqlElement> mapExps = F.addAll(new ArrayList<GridSqlElement>(mapQry.allColumns()),
-            mapQry.select(false));
+            mapQry.columns(false));
 
         GridSqlElement[] rdcExps = new GridSqlElement[mapQry.visibleColumns()];
 
@@ -159,16 +160,16 @@ public class GridSqlQuerySplitter {
             aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocated);
 
         // Fill select expressions.
-        mapQry.clearSelect();
+        mapQry.clearColumns();
 
         for (GridSqlElement exp : mapExps) // Add all map expressions as visible.
-            mapQry.addSelectExpression(exp, true);
+            mapQry.addColumn(exp, true);
 
         for (GridSqlElement rdcExp : rdcExps) // Add corresponding visible reduce columns.
-            rdcQry.addSelectExpression(rdcExp, true);
+            rdcQry.addColumn(rdcExp, true);
 
         for (int i = rdcExps.length; i < mapExps.size(); i++)  // Add all extra map columns as invisible reduce columns.
-            rdcQry.addSelectExpression(column(((GridSqlAlias)mapExps.get(i)).alias()), false);
+            rdcQry.addColumn(column(((GridSqlAlias)mapExps.get(i)).alias()), false);
 
         // -- GROUP BY
         if (mapQry.groupColumns() != null && !collocated)
@@ -214,11 +215,12 @@ public class GridSqlQuerySplitter {
         }
 
         // Build resulting two step query.
-        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, rdcQry.getSQL(),
-            findParams(rdcQry, params, new ArrayList<>()).toArray());
+        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, new GridCacheSqlQuery(rdcQry.getSQL(),
+            findParams(rdcQry, params, new ArrayList<>()).toArray()));
 
-        res.addMapQuery(mergeTable, mapQry.getSQL(),
-            findParams(mapQry, params, new ArrayList<>(params.length)).toArray());
+        res.addMapQuery(new GridCacheSqlQuery(mapQry.getSQL(),
+            findParams(mapQry, params, new ArrayList<>(params.length)).toArray())
+            .columns(collectColumns(mapExps)));
 
         res.explain(explain);
 
@@ -226,6 +228,37 @@ public class GridSqlQuerySplitter {
     }
 
     /**
+     * @param cols Columns from SELECT clause.
+     * @return Map of columns with types.
+     */
+    private static LinkedHashMap<String,?> collectColumns(List<GridSqlElement> cols) {
+        LinkedHashMap<String, GridSqlType> res = new LinkedHashMap<>(cols.size(), 1f, false);
+
+        for (int i = 0; i < cols.size(); i++) {
+            GridSqlElement col = cols.get(i);
+            GridSqlType t = col.resultType();
+
+            if (t == null)
+                throw new NullPointerException("Column type.");
+
+            if (t == GridSqlType.UNKNOWN)
+                throw new IllegalStateException("Unknown type: " + col);
+
+            String alias;
+
+            if (col instanceof GridSqlAlias)
+                alias = ((GridSqlAlias)col).alias();
+            else
+                alias = columnName(i);
+
+            if (res.put(alias, t) != null)
+                throw new IllegalStateException("Alias already exists: " + alias);
+        }
+
+        return res;
+    }
+
+    /**
      * @param qry Query.
      * @param spaces Space names.
      * @return Query.
@@ -242,7 +275,7 @@ public class GridSqlQuerySplitter {
 
             collectAllSpacesInFrom(select.from(), spaces);
 
-            for (GridSqlElement el : select.select(false))
+            for (GridSqlElement el : select.columns(false))
                 collectAllSpacesInSubqueries(el, spaces);
 
             collectAllSpacesInSubqueries(select.where(), spaces);
@@ -325,7 +358,7 @@ public class GridSqlQuerySplitter {
         if (params.length == 0)
             return target;
 
-        for (GridSqlElement el : qry.select(false))
+        for (GridSqlElement el : qry.columns(false))
             findParams(el, params, target);
 
         findParams(qry.from(), params, target);
@@ -422,11 +455,6 @@ public class GridSqlQuerySplitter {
             if (idx < rdcSelect.length) { // SELECT __C0 AS original_alias
                 GridSqlElement rdcEl = column(mapColAlias);
 
-                GridSqlType type = el.expressionResultType();
-
-                if (type != null && type.type() == Value.UUID) // There is no JDBC type UUID, so conversion to bytes occurs.
-                    rdcEl = function(CAST).setCastType("UUID").addChild(rdcEl); // TODO IGNITE-1142 - remove this cast when table function removed
-
                 if (colNames.add(rdcColAlias)) // To handle column name duplication (usually wildcard for few tables).
                     rdcEl = alias(rdcColAlias, rdcEl);
 
@@ -499,6 +527,8 @@ public class GridSqlQuerySplitter {
     ) {
         GridSqlAggregateFunction agg = parentExpr.child(aggIdx);
 
+        assert agg.resultType() != null;
+
         GridSqlElement mapAgg, rdcAgg;
 
         // Create stubbed map alias to fill it with correct expression later.
@@ -513,7 +543,8 @@ public class GridSqlQuerySplitter {
         switch (agg.type()) {
             case AVG: // SUM( AVG(CAST(x AS DOUBLE))*COUNT(x) )/SUM( COUNT(x) ).
                 //-- COUNT(x) map
-                GridSqlElement cntMapAgg = aggregate(agg.distinct(), COUNT).addChild(agg.child());
+                GridSqlElement cntMapAgg = aggregate(agg.distinct(), COUNT)
+                    .resultType(GridSqlType.BIGINT).addChild(agg.child());
 
                 // Add generated alias to COUNT(x).
                 // Using size as index since COUNT will be added as the last select element to the map query.
@@ -524,8 +555,8 @@ public class GridSqlQuerySplitter {
                 mapSelect.add(cntMapAgg);
 
                 //-- AVG(CAST(x AS DOUBLE)) map
-                mapAgg = aggregate(agg.distinct(), AVG).addChild( // Add function argument.
-                    function(CAST).setCastType("DOUBLE").addChild(agg.child()));
+                mapAgg = aggregate(agg.distinct(), AVG).resultType(GridSqlType.DOUBLE).addChild(
+                    function(CAST).resultType(GridSqlType.DOUBLE).addChild(agg.child()));
 
                 //-- SUM( AVG(x)*COUNT(x) )/SUM( COUNT(x) ) reduce
                 GridSqlElement sumUpRdc = aggregate(false, SUM).addChild(
@@ -542,20 +573,20 @@ public class GridSqlQuerySplitter {
             case SUM: // SUM( SUM(x) )
             case MAX: // MAX( MAX(x) )
             case MIN: // MIN( MIN(x) )
-                mapAgg = aggregate(agg.distinct(), agg.type()).addChild(agg.child());
+                mapAgg = aggregate(agg.distinct(), agg.type()).resultType(agg.resultType()).addChild(agg.child());
                 rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias()));
 
                 break;
 
             case COUNT_ALL: // CAST(SUM( COUNT(*) ) AS BIGINT)
             case COUNT: // CAST(SUM( COUNT(x) ) AS BIGINT)
-                mapAgg = aggregate(agg.distinct(), agg.type());
+                mapAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT);
 
                 if (agg.type() == COUNT)
                     mapAgg.addChild(agg.child());
 
                 rdcAgg = aggregate(false, SUM).addChild(column(mapAggAlias.alias()));
-                rdcAgg = function(CAST).setCastType("BIGINT").addChild(rdcAgg);
+                rdcAgg = function(CAST).resultType(GridSqlType.BIGINT).addChild(rdcAgg);
 
                 break;
 
@@ -564,9 +595,11 @@ public class GridSqlQuerySplitter {
         }
 
         assert !(mapAgg instanceof GridSqlAlias);
+        assert mapAgg.resultType() != null;
 
         // Fill the map alias with aggregate.
         mapAggAlias.child(0, mapAgg);
+        mapAggAlias.resultType(mapAgg.resultType());
 
         // Replace in original expression aggregate with reduce aggregate.
         parentExpr.child(aggIdx, rdcAgg);
@@ -595,7 +628,11 @@ public class GridSqlQuerySplitter {
      * @return Alias.
      */
     private static GridSqlAlias alias(String alias, GridSqlElement child) {
-        return new GridSqlAlias(alias, child);
+        GridSqlAlias res = new GridSqlAlias(alias, child);
+
+        res.resultType(child.resultType());
+
+        return res;
     }
 
     /**
@@ -615,12 +652,4 @@ public class GridSqlQuerySplitter {
     private static GridSqlFunction function(GridSqlFunctionType type) {
         return new GridSqlFunction(type);
     }
-
-    /**
-     * @param name Table name.
-     * @return Table.
-     */
-    private static GridSqlTable table(String name) {
-        return new GridSqlTable(null, name);
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
index fb2643e..6705c48 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
@@ -26,10 +26,10 @@ import java.util.*;
  */
 public class GridSqlSelect extends GridSqlQuery {
     /** */
-    private List<GridSqlElement> allExprs = new ArrayList<>();
+    private List<GridSqlElement> cols = new ArrayList<>();
 
     /** */
-    private List<GridSqlElement> select = new ArrayList<>();
+    private int visibleCols;
 
     /** */
     private int[] grpCols;
@@ -45,19 +45,19 @@ public class GridSqlSelect extends GridSqlQuery {
 
     /** {@inheritDoc} */
     @Override public int visibleColumns() {
-        return select.size();
+        return visibleCols;
     }
 
     /**
      * @return Number of columns is select including invisible ones.
      */
     public int allColumns() {
-        return allExprs.size();
+        return cols.size();
     }
 
     /** {@inheritDoc} */
-    @Override protected GridSqlElement expression(int col) {
-        return allExprs.get(col);
+    @Override protected GridSqlElement column(int col) {
+        return cols.get(col);
     }
 
     /** {@inheritDoc} */
@@ -67,7 +67,7 @@ public class GridSqlSelect extends GridSqlQuery {
         if (distinct)
             buff.append(" DISTINCT");
 
-        for (GridSqlElement expression : select) {
+        for (GridSqlElement expression : columns(true)) {
             buff.appendExceptFirst(",");
             buff.append('\n');
             buff.append(expression.getSQL());
@@ -86,14 +86,14 @@ public class GridSqlSelect extends GridSqlQuery {
             for (int grpCol : grpCols) {
                 buff.appendExceptFirst(", ");
 
-                addAlias(buff, allExprs.get(grpCol));
+                addAlias(buff, cols.get(grpCol));
             }
         }
 
         if (havingCol >= 0) {
             buff.append("\nHAVING ");
 
-            addAlias(buff, allExprs.get(havingCol));
+            addAlias(buff, cols.get(havingCol));
         }
 
         getSortLimitSQL(buff);
@@ -114,52 +114,59 @@ public class GridSqlSelect extends GridSqlQuery {
 
     /**
      * @param visibleOnly If only visible expressions needed.
-     * @return Select phrase expressions.
+     * @return Select clause expressions.
      */
-    public Iterable<GridSqlElement> select(boolean visibleOnly) {
-        return visibleOnly ? select : allExprs;
+    public Iterable<GridSqlElement> columns(boolean visibleOnly) {
+        assert visibleCols <= cols.size();
+
+        return visibleOnly && visibleCols != cols.size() ?
+            cols.subList(0, visibleCols) : cols;
     }
 
     /**
-     * Clears select list.
+     * Clears select expressions list.
+     * @return {@code this}.
      */
-    public void clearSelect() {
-        select = new ArrayList<>();
-        allExprs = new ArrayList<>();
+    public GridSqlSelect clearColumns() {
+        visibleCols = 0;
+        cols = new ArrayList<>();
+
+        return this;
     }
 
     /**
      * @param expression Expression.
      * @param visible Expression is visible in select phrase.
+     * @return {@code this}.
      */
-    public void addSelectExpression(GridSqlElement expression, boolean visible) {
+    public GridSqlSelect addColumn(GridSqlElement expression, boolean visible) {
         if (expression == null)
             throw new NullPointerException();
 
         if (visible) {
-            if (select.size() != allExprs.size())
+            if (visibleCols != cols.size())
                 throw new IllegalStateException("Already started adding invisible columns.");
 
-            select.add(expression);
+            visibleCols++;
         }
-        else if (select.isEmpty())
-            throw new IllegalStateException("No visible columns.");
 
-        allExprs.add(expression);
+        cols.add(expression);
+
+        return this;
     }
 
     /**
      * @param colIdx Column index.
      * @param expression Expression.
+     * @return {@code this}.
      */
-    public void setSelectExpression(int colIdx, GridSqlElement expression) {
+    public GridSqlSelect setColumn(int colIdx, GridSqlElement expression) {
         if (expression == null)
             throw new NullPointerException();
 
-        if (colIdx < select.size()) // Assuming that all the needed expressions were already added.
-            select.set(colIdx, expression);
+        cols.set(colIdx, expression);
 
-        allExprs.set(colIdx, expression);
+        return this;
     }
 
     /**
@@ -171,9 +178,12 @@ public class GridSqlSelect extends GridSqlQuery {
 
     /**
      * @param grpCols Group columns.
+     * @return {@code this}.
      */
-    public void groupColumns(int[] grpCols) {
+    public GridSqlSelect groupColumns(int[] grpCols) {
         this.grpCols = grpCols;
+
+        return this;
     }
 
     /**
@@ -202,9 +212,12 @@ public class GridSqlSelect extends GridSqlQuery {
 
     /**
      * @param where New where.
+     * @return {@code this}.
      */
-    public void where(GridSqlElement where) {
+    public GridSqlSelect where(GridSqlElement where) {
         this.where = where;
+
+        return this;
     }
 
     /**
@@ -226,16 +239,19 @@ public class GridSqlSelect extends GridSqlQuery {
      * @return Having.
      */
     public GridSqlElement having() {
-        return havingCol >= 0 ? allExprs.get(havingCol) : null;
+        return havingCol >= 0 ? column(havingCol) : null;
     }
 
     /**
      * @param col Index of HAVING column.
+     * @return {@code this}.
      */
-    public void havingColumn(int col) {
+    public GridSqlSelect havingColumn(int col) {
         assert col >= -1 : col;
 
         havingCol = col;
+
+        return this;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
index 1dbcd46..aeee562 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
@@ -17,10 +17,27 @@
 
 package org.apache.ignite.internal.processors.query.h2.sql;
 
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.h2.value.*;
+
 /**
  * SQL Data type based on H2.
  */
-public class GridSqlType {
+public final class GridSqlType {
+    /** */
+    public static final GridSqlType UNKNOWN = new GridSqlType(Value.UNKNOWN, 0, 0, 0, null);
+
+    /** */
+    public static final GridSqlType BIGINT = new GridSqlType(Value.LONG, 0, ValueLong.PRECISION,
+        ValueLong.DISPLAY_SIZE, "BIGINT");
+
+    /** */
+    public static final GridSqlType DOUBLE = new GridSqlType(Value.DOUBLE, 0, ValueDouble.PRECISION,
+        ValueDouble.DISPLAY_SIZE, "DOUBLE");
+
+    /** */
+    public static final GridSqlType UUID = new GridSqlType(Value.UUID, 0, Integer.MAX_VALUE, 36, "UUID");
+
     /** H2 type. */
     private final int type;
 
@@ -91,4 +108,9 @@ public class GridSqlType {
     public String sql() {
         return sql;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridSqlType.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUnion.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUnion.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUnion.java
index 721c288..2900470 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUnion.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlUnion.java
@@ -41,7 +41,7 @@ public class GridSqlUnion extends GridSqlQuery {
     }
 
     /** {@inheritDoc} */
-    @Override protected GridSqlElement expression(int col) {
+    @Override protected GridSqlElement column(int col) {
         throw new IllegalStateException();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 5510e9e..03500e6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.query.h2.*;
+import org.apache.ignite.internal.processors.query.h2.sql.*;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -36,16 +37,11 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.plugin.extensions.communication.*;
-import org.h2.command.*;
 import org.h2.command.ddl.*;
-import org.h2.command.dml.*;
 import org.h2.engine.*;
-import org.h2.expression.*;
-import org.h2.index.*;
 import org.h2.jdbc.*;
 import org.h2.result.*;
 import org.h2.table.*;
-import org.h2.tools.*;
 import org.h2.util.*;
 import org.h2.value.*;
 import org.jetbrains.annotations.*;
@@ -55,9 +51,9 @@ import javax.cache.*;
 import java.lang.reflect.*;
 import java.sql.*;
 import java.util.*;
-import java.util.Set;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
 
 import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.*;
 
@@ -84,7 +80,10 @@ public class GridReduceQueryExecutor {
     private final ConcurrentMap<Long, QueryRun> runs = new ConcurrentHashMap8<>();
 
     /** */
-    private static ThreadLocal<GridMergeTable> curFunTbl = new ThreadLocal<>();
+    private volatile List<GridThreadLocalTable> fakeTbls = Collections.emptyList();
+
+    /** */
+    private final Lock fakeTblsLock = new ReentrantLock();
 
     /** */
     private static final Constructor<JdbcResultSet> CONSTRUCTOR;
@@ -475,11 +474,13 @@ public class GridReduceQueryExecutor {
                 nodes = Collections.singleton(F.rand(nodes));
             }
 
+            int tblIdx = 0;
+
             for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
                 GridMergeTable tbl;
 
                 try {
-                    tbl = createFunctionTable(r.conn, mapQry, qry.explain()); // createTable(r.conn, mapQry); TODO
+                    tbl = createMergeTable(r.conn, mapQry, qry.explain());
                 }
                 catch (IgniteCheckedException e) {
                     throw new IgniteException(e);
@@ -492,7 +493,7 @@ public class GridReduceQueryExecutor {
 
                 r.tbls.add(tbl);
 
-                curFunTbl.set(tbl);
+                fakeTable(r.conn, tblIdx++).setInnerTable(tbl);
             }
 
             r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
@@ -512,7 +513,7 @@ public class GridReduceQueryExecutor {
                     mapQrys = new ArrayList<>(qry.mapQueries().size());
 
                     for (GridCacheSqlQuery mapQry : qry.mapQueries())
-                        mapQrys.add(new GridCacheSqlQuery(mapQry.alias(), "EXPLAIN " + mapQry.query(), mapQry.parameters()));
+                        mapQrys.add(new GridCacheSqlQuery("EXPLAIN " + mapQry.query(), mapQry.parameters()));
                 }
 
                 if (nodes.size() != 1 || !F.first(nodes).isLocal()) { // Marshall params for remotes.
@@ -565,8 +566,6 @@ public class GridReduceQueryExecutor {
                 for (GridMergeTable tbl : r.tbls) {
                     if (!tbl.getScanIndex(null).fetchedAll()) // We have to explicitly cancel queries on remote nodes.
                         send(nodes, new GridQueryCancelRequest(qryReqId), null);
-
-//                dropTable(r.conn, tbl.getName()); TODO
                 }
 
                 if (retry) {
@@ -600,7 +599,8 @@ public class GridReduceQueryExecutor {
                 if (!runs.remove(qryReqId, r))
                     U.warn(log, "Query run was already removed: " + qryReqId);
 
-                curFunTbl.remove();
+                for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++)
+                    fakeTable(null, i).setInnerTable(null); // Drop all merge tables.
             }
         }
     }
@@ -626,6 +626,54 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @param idx Table index.
+     * @return Table name.
+     */
+    private static String table(int idx) {
+        return GridSqlQuerySplitter.table(idx).getSQL();
+    }
+
+    /**
+     * Gets or creates new fake table for index.
+     *
+     * @param idx Index of table.
+     * @return Table.
+     */
+    private GridThreadLocalTable fakeTable(Connection c, int idx) {
+        List<GridThreadLocalTable> tbls = fakeTbls;
+
+        assert tbls.size() >= idx;
+
+        if (tbls.size() == idx) { // If table for such index does not exist, create one.
+            fakeTblsLock.lock();
+
+            try {
+                if ((tbls = fakeTbls).size() == idx) { // Double check inside of lock.
+                    try (Statement stmt = c.createStatement()) {
+                        stmt.executeUpdate("CREATE TABLE " + table(idx) +
+                            "(fake BOOL) ENGINE \"" + GridThreadLocalTable.Engine.class.getName() + '"');
+                    }
+                    catch (SQLException e) {
+                        throw new IllegalStateException(e);
+                    }
+
+                    List<GridThreadLocalTable> newTbls = new ArrayList<>(tbls.size() + 1);
+
+                    newTbls.addAll(tbls);
+                    newTbls.add(GridThreadLocalTable.Engine.getCreated());
+
+                    fakeTbls = tbls = newTbls;
+                }
+            }
+            finally {
+                fakeTblsLock.unlock();
+            }
+        }
+
+        return tbls.get(idx);
+    }
+
+    /**
      * Calculates data nodes for replicated caches on unstable topology.
      *
      * @param cctx Cache context for main space.
@@ -858,16 +906,18 @@ public class GridReduceQueryExecutor {
         throws IgniteCheckedException {
         List<List<?>> lists = new ArrayList<>();
 
-        for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
-            ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + mapQry.alias(), null);
+        for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) {
+            ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + table(i), null);
 
             lists.add(F.asList(getPlan(rs)));
         }
 
+        int tblIdx = 0;
+
         for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
-            GridMergeTable tbl = createFunctionTable(c, mapQry, false);
+            GridMergeTable tbl = createMergeTable(c, mapQry, false);
 
-            curFunTbl.set(tbl); // Now it will be only a single table.
+            fakeTable(c, tblIdx++).setInnerTable(tbl);
         }
 
         GridCacheSqlQuery rdc = qry.reduceQuery();
@@ -961,118 +1011,12 @@ public class GridReduceQueryExecutor {
 
     /**
      * @param conn Connection.
-     * @param tblName Table name.
-     * @throws SQLException If failed.
-     */
-    private void dropTable(Connection conn, String tblName) throws SQLException {
-        try (Statement s = conn.createStatement()) {
-            s.execute("DROP TABLE " + tblName);
-        }
-    }
-
-    /**
-     * @return Merged result set.
-     */
-    public static ResultSet mergeTableFunction(JdbcConnection c) throws Exception {
-        GridMergeTable tbl = curFunTbl.get();
-
-        Session ses = (Session)c.getSession();
-
-        String url = c.getMetaData().getURL();
-
-        // URL is either "jdbc:default:connection" or "jdbc:columnlist:connection"
-        final Cursor cursor = url.charAt(5) == 'c' ? null : tbl.getScanIndex(ses).find(ses, null, null);
-
-        final Column[] cols = tbl.getColumns();
-
-        SimpleResultSet rs = new SimpleResultSet(cursor == null ? null : new SimpleRowSource() {
-            @Override public Object[] readRow() throws SQLException {
-                if (!cursor.next())
-                    return null;
-
-                Row r = cursor.get();
-
-                Object[] row = new Object[cols.length];
-
-                for (int i = 0; i < row.length; i++)
-                    row[i] = r.getValue(i).getObject();
-
-                return row;
-            }
-
-            @Override public void close() {
-                // No-op.
-            }
-
-            @Override public void reset() throws SQLException {
-                throw new SQLException("Unsupported.");
-            }
-        }) {
-            @Override public byte[] getBytes(int colIdx) throws SQLException {
-                assert cursor != null;
-
-                return cursor.get().getValue(colIdx - 1).getBytes();
-            }
-
-            @Override public <T> T getObject(int columnIndex, Class<T> type) throws SQLException {
-                throw new UnsupportedOperationException();
-            }
-
-            @Override public <T> T getObject(String columnLabel, Class<T> type) throws SQLException {
-                throw new UnsupportedOperationException();
-            }
-        };
-
-        for (Column col : cols)
-            rs.addColumn(col.getName(), DataType.convertTypeToSQLType(col.getType()),
-                MathUtils.convertLongToInt(col.getPrecision()), col.getScale());
-
-        return rs;
-    }
-
-    /**
-     * @param asQuery Query.
-     * @return List of columns.
-     */
-    private static ArrayList<Column> generateColumnsFromQuery(org.h2.command.dml.Query asQuery) {
-        int columnCount = asQuery.getColumnCount();
-        ArrayList<Expression> expressions = asQuery.getExpressions();
-        ArrayList<Column> cols = new ArrayList<>();
-        for (int i = 0; i < columnCount; i++) {
-            Expression expr = expressions.get(i);
-            int type = expr.getType();
-            String name = expr.getAlias();
-            long precision = expr.getPrecision();
-            int displaySize = expr.getDisplaySize();
-            DataType dt = DataType.getDataType(type);
-            if (precision > 0 && (dt.defaultPrecision == 0 ||
-                (dt.defaultPrecision > precision && dt.defaultPrecision < Byte.MAX_VALUE))) {
-                // dont' set precision to MAX_VALUE if this is the default
-                precision = dt.defaultPrecision;
-            }
-            int scale = expr.getScale();
-            if (scale > 0 && (dt.defaultScale == 0 ||
-                (dt.defaultScale > scale && dt.defaultScale < precision))) {
-                scale = dt.defaultScale;
-            }
-            if (scale > precision) {
-                precision = scale;
-            }
-            Column col = new Column(name, type, precision, scale, displaySize);
-            cols.add(col);
-        }
-
-        return cols;
-    }
-
-    /**
-     * @param conn Connection.
      * @param qry Query.
      * @param explain Explain.
      * @return Table.
      * @throws IgniteCheckedException
      */
-    private GridMergeTable createFunctionTable(JdbcConnection conn, GridCacheSqlQuery qry, boolean explain)
+    private GridMergeTable createMergeTable(JdbcConnection conn, GridCacheSqlQuery qry, boolean explain)
         throws IgniteCheckedException {
         try {
             Session ses = (Session)conn.getSession();
@@ -1084,17 +1028,24 @@ public class GridReduceQueryExecutor {
             data.create = true;
 
             if (!explain) {
-                Prepared prepare = ses.prepare(qry.query(), false);
+                LinkedHashMap<String,?> colsMap = qry.columns();
+
+                assert colsMap != null;
+
+                ArrayList<Column> cols = new ArrayList<>(colsMap.size());
+
+                for (Map.Entry<String,?> e : colsMap.entrySet()) {
+                    String alias = e.getKey();
+                    GridSqlType t = (GridSqlType)e.getValue();
 
-                List<org.h2.expression.Parameter> parsedParams = prepare.getParameters();
+                    assert !F.isEmpty(alias);
 
-                for (int i = Math.min(parsedParams.size(), qry.parameters().length); --i >= 0; ) {
-                    Object val = qry.parameters()[i];
+                    Column c = new Column(alias, t.type(), t.precision(), t.scale(), t.displaySize());
 
-                    parsedParams.get(i).setValue(DataType.convertToValue(ses, val, Value.UNKNOWN));
+                    cols.add(c);
                 }
 
-                data.columns = generateColumnsFromQuery((Query)prepare);
+                data.columns = cols;
             }
             else
                 data.columns = planColumns();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridThreadLocalTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridThreadLocalTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridThreadLocalTable.java
new file mode 100644
index 0000000..c468371
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridThreadLocalTable.java
@@ -0,0 +1,262 @@
+/*
+ * 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.twostep;
+
+import org.h2.api.*;
+import org.h2.command.ddl.*;
+import org.h2.engine.*;
+import org.h2.index.*;
+import org.h2.result.*;
+import org.h2.schema.*;
+import org.h2.table.*;
+import org.h2.value.*;
+
+import java.util.*;
+
+/**
+ * Thread local table wrapper for another table instance.
+ */
+public class GridThreadLocalTable extends Table {
+    /** Delegate table */
+    private final ThreadLocal<Table> tbl = new ThreadLocal<>();
+
+    /**
+     * @param schema Schema.
+     * @param id ID.
+     * @param name Table name.
+     * @param persistIndexes Persist indexes.
+     * @param persistData Persist data.
+     */
+    public GridThreadLocalTable(Schema schema, int id, String name, boolean persistIndexes, boolean persistData) {
+        super(schema, id, name, persistIndexes, persistData);
+    }
+
+    /**
+     * @param t Table or {@code null} to reset existing.
+     */
+    public void setInnerTable(Table t) {
+        if (t == null)
+            tbl.remove();
+        else
+            tbl.set(t);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index getPrimaryKey() {
+        return tbl.get().getPrimaryKey();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Column getRowIdColumn() {
+        return tbl.get().getRowIdColumn();
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlanItem getBestPlanItem(Session session, int[] masks, TableFilter filter, SortOrder sortOrder) {
+        return tbl.get().getBestPlanItem(session, masks, filter, sortOrder);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Value getDefaultValue(Session session, Column column) {
+        return tbl.get().getDefaultValue(session, column);
+    }
+
+    /** {@inheritDoc} */
+    @Override public SearchRow getTemplateSimpleRow(boolean singleColumn) {
+        return tbl.get().getTemplateSimpleRow(singleColumn);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Row getTemplateRow() {
+        return tbl.get().getTemplateRow();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Column getColumn(String columnName) {
+        return tbl.get().getColumn(columnName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Column getColumn(int index) {
+        return tbl.get().getColumn(index);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index getIndexForColumn(Column column) {
+        return tbl.get().getIndexForColumn(column);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Column[] getColumns() {
+        return tbl.get().getColumns();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void setColumns(Column[] columns) {
+        throw new IllegalStateException("Cols: " + Arrays.asList(columns));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void lock(Session session, boolean exclusive, boolean force) {
+        tbl.get().lock(session, exclusive, force);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(Session session) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void unlock(Session s) {
+        tbl.get().unlock(s);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index addIndex(Session session, String indexName, int indexId, IndexColumn[] cols,
+        IndexType indexType, boolean create, String indexComment) {
+        return tbl.get().addIndex(session, indexName, indexId, cols, indexType, create, indexComment);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeRow(Session session, Row row) {
+        tbl.get().removeRow(session, row);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void truncate(Session session) {
+        tbl.get().truncate(session);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addRow(Session session, Row row) {
+        tbl.get().addRow(session, row);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkSupportAlter() {
+        tbl.get().checkSupportAlter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTableType() {
+        return tbl.get().getTableType();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index getUniqueIndex() {
+        return tbl.get().getUniqueIndex();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index getScanIndex(Session session) {
+        return tbl.get().getScanIndex(session);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ArrayList<Index> getIndexes() {
+        return tbl.get().getIndexes();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isLockedExclusively() {
+        return tbl.get().isLockedExclusively();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getMaxDataModificationId() {
+        return tbl.get().getMaxDataModificationId();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDeterministic() {
+        return tbl.get().isDeterministic();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canGetRowCount() {
+        return tbl.get().canGetRowCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canDrop() {
+        return tbl.get().canDrop();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCount(Session session) {
+        return tbl.get().getRowCount(session);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getRowCountApproximation() {
+        return tbl.get().getRowCountApproximation();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getDiskSpaceUsed() {
+        return tbl.get().getDiskSpaceUsed();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCreateSQL() {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDropSQL() {
+        return tbl.get().getDropSQL();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkRename() {
+        tbl.get().checkRename();
+    }
+
+    /**
+     * Engine.
+     */
+    public static class Engine implements TableEngine {
+        /** */
+        private static ThreadLocal<GridThreadLocalTable> createdTbl = new ThreadLocal<>();
+
+        /**
+         * @return Created table.
+         */
+        public static GridThreadLocalTable getCreated() {
+            GridThreadLocalTable tbl = createdTbl.get();
+
+            assert tbl != null;
+
+            createdTbl.remove();
+
+            return tbl;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Table createTable(CreateTableData d) {
+            assert createdTbl.get() == null;
+
+            GridThreadLocalTable tbl = new GridThreadLocalTable(d.schema, d.id, d.tableName, d.persistIndexes,
+                d.persistData);
+
+            createdTbl.set(tbl);
+
+            return tbl;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7dd4a02/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index ccb3115..18bfd57 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -316,7 +316,7 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
         if (cacheMode() == PARTITIONED) {
             assertEquals(2, res.size());
 
-            assertTrue(((String)res.get(1).get(0)).contains(GridSqlQuerySplitter.TABLE_FUNC_NAME));
+            assertTrue(((String)res.get(1).get(0)).contains(GridSqlQuerySplitter.table(0).getSQL()));
         }
         else
             assertEquals(1, res.size());