You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/05/08 19:22:23 UTC

[04/50] [abbrv] incubator-ignite git commit: ignite-855

ignite-855


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

Branch: refs/heads/ignite-37
Commit: eab2e538a53042c2a8f252d22ff50957a48ae364
Parents: 07a4258
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed May 6 11:21:57 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed May 6 11:21:57 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheSqlQuery.java          |   2 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  17 +++
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +
 .../processors/query/h2/sql/GridSqlQuery.java   |  20 ++++
 .../query/h2/sql/GridSqlQueryParser.java        |  10 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  11 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |   2 +-
 .../processors/query/h2/sql/GridSqlUnion.java   |   2 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   3 +
 .../h2/twostep/GridReduceQueryExecutor.java     | 119 +++++++++++++++----
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  21 ++++
 11 files changed, 184 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 0c1c822..7a0e140 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
@@ -64,7 +64,7 @@ public class GridCacheSqlQuery implements Message {
      * @param qry Query.
      * @param params Query parameters.
      */
-    GridCacheSqlQuery(String alias, String qry, Object[] params) {
+    public GridCacheSqlQuery(String alias, String qry, Object[] params) {
         A.ensure(!F.isEmpty(qry), "qry must not be empty");
 
         this.alias = alias;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 ce12a85..53fc7a3 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
@@ -43,6 +43,9 @@ public class GridCacheTwoStepQuery {
     /** */
     private int pageSize = DFLT_PAGE_SIZE;
 
+    /** */
+    private boolean explain;
+
     /**
      * @param qry Reduce query.
      * @param params Reduce query parameters.
@@ -52,6 +55,20 @@ public class GridCacheTwoStepQuery {
     }
 
     /**
+     * @return If this is explain query.
+     */
+    public boolean explain() {
+        return explain;
+    }
+
+    /**
+     * @param explain If this is explain query.
+     */
+    public void explain(boolean explain) {
+        this.explain = explain;
+    }
+
+    /**
      * @param pageSize Page size.
      */
     public void pageSize(int pageSize) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 9579de3..975378c 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
@@ -1321,6 +1321,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             throw new IgniteCheckedException("Cache already registered: " + U.maskName(ccfg.getName()));
 
         createSchema(schema);
+
+        executeStatement(schema, "CREATE ALIAS " + GridSqlQuerySplitter.TABLE_FUNC_NAME +
+            " NOBUFFER FOR \"" + GridReduceQueryExecutor.class.getName() + ".mergeTableFunction\"");
+
         createSqlFunctions(schema, ccfg.getSqlFunctionClasses());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 c40632f..207588e 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
@@ -37,6 +37,26 @@ public abstract class GridSqlQuery implements Cloneable {
     /** */
     protected GridSqlElement limit;
 
+    /** */
+    private boolean explain;
+
+    /**
+     * @param explain Explain.
+     * @return {@code this}.
+     */
+    public GridSqlQuery explain(boolean explain) {
+        this.explain = explain;
+
+        return this;
+    }
+
+    /**
+     * @return {@code true} If explain.
+     */
+    public boolean explain() {
+        return explain;
+    }
+
     /**
      * @return Offset.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 62ec72f..74e4748 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
@@ -158,6 +158,9 @@ public class GridSqlQueryParser {
     private static final Getter<SelectUnion, SortOrder> UNION_SORT = getter(SelectUnion.class, "sort");
 
     /** */
+    private static final Getter<Explain, Prepared> EXPLAIN_COMMAND = getter(Explain.class, "command");
+
+    /** */
     private static volatile Getter<Command,Prepared> prepared;
 
     /** */
@@ -182,7 +185,7 @@ public class GridSqlQueryParser {
 
         Prepared statement = p.get(cmd);
 
-        return new GridSqlQueryParser().parse((Query)statement);
+        return new GridSqlQueryParser().parse(statement);
     }
 
     /**
@@ -321,13 +324,16 @@ public class GridSqlQueryParser {
     /**
      * @param qry Select.
      */
-    public GridSqlQuery parse(Query qry) {
+    public GridSqlQuery parse(Prepared qry) {
         if (qry instanceof Select)
             return parse((Select)qry);
 
         if (qry instanceof SelectUnion)
             return parse((SelectUnion)qry);
 
+        if (qry instanceof Explain)
+            return parse(EXPLAIN_COMMAND.get((Explain)qry)).explain(true);
+
         throw new UnsupportedOperationException("Unknown query type: " + qry);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 73d7751..6c7e2e2 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
@@ -78,7 +78,7 @@ public class GridSqlQuerySplitter {
         if (params == null)
             params = GridCacheSqlQuery.EMPTY_PARAMS;
 
-        GridSqlQuery qry0 = GridSqlQueryParser.parse(stmt);
+        final GridSqlQuery qry0 = GridSqlQueryParser.parse(stmt);
 
         GridSqlSelect srcQry;
 
@@ -87,6 +87,8 @@ public class GridSqlQuerySplitter {
         else { // Handle UNION.
             srcQry = new GridSqlSelect().from(new GridSqlSubquery(qry0));
 
+            srcQry.explain(qry0.explain());
+
             GridSqlSelect left = leftest(qry0);
 
             int c = 0;
@@ -127,7 +129,10 @@ public class GridSqlQuerySplitter {
 
         // Create map and reduce queries.
         GridSqlSelect mapQry = srcQry.clone();
-        GridSqlSelect rdcQry = new GridSqlSelect().from(new GridSqlFunction("PUBLIC", TABLE_FUNC_NAME)); // table(mergeTable)); TODO
+
+        mapQry.explain(false);
+
+        GridSqlSelect rdcQry = new GridSqlSelect().from(new GridSqlFunction(null, TABLE_FUNC_NAME)); // table(mergeTable)); TODO
 
         // Split all select expressions into map-reduce parts.
         List<GridSqlElement> mapExps = F.addAll(
@@ -213,6 +218,8 @@ public class GridSqlQuerySplitter {
         res.addMapQuery(mergeTable, mapQry.getSQL(),
             findParams(mapQry, params, new ArrayList<>(params.length)).toArray());
 
+        res.explain(qry0.explain());
+
         return res;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 0786eac..9972bba 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
@@ -68,7 +68,7 @@ public class GridSqlSelect extends GridSqlQuery {
 
     /** {@inheritDoc} */
     @Override public String getSQL() {
-        StatementBuilder buff = new StatementBuilder("SELECT");
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN SELECT" : "SELECT");
 
         if (distinct)
             buff.append(" DISTINCT");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 23c18ed..96beb6b 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
@@ -47,7 +47,7 @@ public class GridSqlUnion extends GridSqlQuery {
 
     /** {@inheritDoc} */
     @Override public String getSQL() {
-        StatementBuilder buff = new StatementBuilder();
+        StatementBuilder buff = new StatementBuilder(explain() ? "EXPLAIN \n" : "");
 
         buff.append('(').append(left.getSQL()).append(')');
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 66b3636..f15a2da 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -141,6 +141,9 @@ public class GridMapQueryExecutor {
 
             ClusterNode node = ctx.discovery().node(nodeId);
 
+            if (node == null)
+                return; // Node left, ignore.
+
             boolean processed = true;
 
             if (msg instanceof GridQueryRequest)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 e175a43..09a238f 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
@@ -28,13 +28,13 @@ import org.apache.ignite.internal.processors.cache.*;
 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.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 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.Query;
 import org.h2.engine.*;
@@ -152,9 +152,6 @@ public class GridReduceQueryExecutor {
                 }
             }
         }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
-
-        h2.executeStatement("PUBLIC", "CREATE ALIAS " + GridSqlQuerySplitter.TABLE_FUNC_NAME +
-            " NOBUFFER FOR \"" + GridReduceQueryExecutor.class.getName() + ".mergeTableFunction\"");
     }
 
     /**
@@ -167,6 +164,9 @@ public class GridReduceQueryExecutor {
 
             ClusterNode node = ctx.discovery().node(nodeId);
 
+            if (node == null)
+                return; // Node left, ignore.
+
             boolean processed = true;
 
             if (msg instanceof GridQueryNextPageResponse)
@@ -280,15 +280,16 @@ public class GridReduceQueryExecutor {
 
         String space = cctx.name();
 
-        r.conn = h2.connectionForSpace(space);
+        r.conn = (JdbcConnection)h2.connectionForSpace(space);
 
         // TODO Add topology version.
         ClusterGroup dataNodes = ctx.grid().cluster().forDataNodes(space);
 
-        if (cctx.isReplicated()) {
-            assert dataNodes.node(ctx.localNodeId()) == null : "We must be on a client node.";
+        if (cctx.isReplicated() || qry.explain()) {
+            assert qry.explain() || dataNodes.node(ctx.localNodeId()) == null : "We must be on a client node.";
 
-            dataNodes = dataNodes.forRandom(); // Select random data node to run query on a replicated data.
+            // Select random data node to run query on a replicated data or get EXPLAIN PLAN from a single node.
+            dataNodes = dataNodes.forRandom();
         }
 
         final Collection<ClusterNode> nodes = dataNodes.nodes();
@@ -297,7 +298,7 @@ public class GridReduceQueryExecutor {
             GridMergeTable tbl;
 
             try {
-                tbl = createFunctionTable((JdbcConnection)r.conn, mapQry); // createTable(r.conn, mapQry); TODO
+                tbl = createFunctionTable(r.conn, mapQry, qry.explain()); // createTable(r.conn, mapQry); TODO
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
@@ -318,20 +319,32 @@ public class GridReduceQueryExecutor {
         runs.put(qryReqId, r);
 
         try {
+            Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();
+
+            if (qry.explain()) {
+                mapQrys = new ArrayList<>(qry.mapQueries().size());
+
+                for (GridCacheSqlQuery mapQry : qry.mapQueries())
+                    mapQrys.add(new GridCacheSqlQuery(mapQry.alias(), "EXPLAIN " + mapQry.query(), mapQry.parameters()));
+            }
+
             if (nodes.size() != 1 || !F.first(nodes).isLocal()) { // Marshall params for remotes.
                 Marshaller m = ctx.config().getMarshaller();
 
-                for (GridCacheSqlQuery mapQry : qry.mapQueries())
+                for (GridCacheSqlQuery mapQry : mapQrys)
                     mapQry.marshallParams(m);
             }
 
-            send(nodes, new GridQueryRequest(qryReqId, r.pageSize, space, qry.mapQueries()));
+            send(nodes, new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys));
 
             r.latch.await();
 
             if (r.rmtErr != null)
                 throw new CacheException("Failed to run map query remotely.", r.rmtErr);
 
+            if (qry.explain())
+                return explainPlan(r.conn, space, qry);
+
             GridCacheSqlQuery rdc = qry.reduceQuery();
 
             final ResultSet res = h2.executeSqlQueryWithTimer(space, r.conn, rdc.query(), F.asList(rdc.parameters()));
@@ -362,6 +375,55 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @param c Connection.
+     * @param space Space.
+     * @param qry Query.
+     * @return Cursor for plans.
+     * @throws IgniteCheckedException if failed.
+     */
+    private QueryCursor<List<?>> explainPlan(JdbcConnection c, String space, GridCacheTwoStepQuery qry)
+        throws IgniteCheckedException {
+        List<List<?>> lists = new ArrayList<>();
+
+        for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
+            ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + mapQry.alias(), null);
+
+            lists.add(F.asList(getPlan(rs)));
+        }
+
+        for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
+            GridMergeTable tbl = createFunctionTable(c, mapQry, false);
+
+            curFunTbl.set(tbl); // Now it will be only a single table.
+        }
+
+        GridCacheSqlQuery rdc = qry.reduceQuery();
+
+        ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "EXPLAIN " + rdc.query(), F.asList(rdc.parameters()));
+
+        lists.add(F.asList(getPlan(rs)));
+
+        return new QueryCursorImpl<>(lists.iterator());
+    }
+
+    /**
+     * @param rs Result set.
+     * @return Plan.
+     * @throws IgniteCheckedException If failed.
+     */
+    private String getPlan(ResultSet rs) throws IgniteCheckedException {
+        try {
+            if (!rs.next())
+                throw new IllegalStateException();
+
+            return rs.getString(1);
+        }
+        catch (SQLException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
      * @param nodes Nodes.
      * @param msg Message.
      * @throws IgniteCheckedException If failed.
@@ -502,10 +564,12 @@ public class GridReduceQueryExecutor {
     /**
      * @param conn Connection.
      * @param qry Query.
+     * @param explain Explain.
      * @return Table.
      * @throws IgniteCheckedException
      */
-    private GridMergeTable createFunctionTable(JdbcConnection conn, GridCacheSqlQuery qry) throws IgniteCheckedException {
+    private GridMergeTable createFunctionTable(JdbcConnection conn, GridCacheSqlQuery qry, boolean explain)
+        throws IgniteCheckedException {
         try {
             Session ses = (Session)conn.getSession();
 
@@ -515,17 +579,21 @@ public class GridReduceQueryExecutor {
             data.schema = ses.getDatabase().getSchema(ses.getCurrentSchemaName());
             data.create = true;
 
-            Query prepare = (Query)ses.prepare(qry.query(), false);
+            if (!explain) {
+                Prepared prepare = ses.prepare(qry.query(), false);
 
-            List<org.h2.expression.Parameter> parsedParams = prepare.getParameters();
+                List<org.h2.expression.Parameter> parsedParams = prepare.getParameters();
 
-            for (int i = Math.min(parsedParams.size(), qry.parameters().length); --i >= 0; ) {
-                Object val = qry.parameters()[i];
+                for (int i = Math.min(parsedParams.size(), qry.parameters().length); --i >= 0; ) {
+                    Object val = qry.parameters()[i];
 
-                parsedParams.get(i).setValue(DataType.convertToValue(ses, val, Value.UNKNOWN));
-            }
+                    parsedParams.get(i).setValue(DataType.convertToValue(ses, val, Value.UNKNOWN));
+                }
 
-            data.columns = generateColumnsFromQuery(prepare);
+                data.columns = generateColumnsFromQuery((Query)prepare);
+            }
+            else
+                data.columns = planColumns();
 
             return new GridMergeTable(data);
         }
@@ -537,6 +605,17 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @return Columns.
+     */
+    private static ArrayList<Column> planColumns() {
+        ArrayList<Column> res = new ArrayList<>(1);
+
+        res.add(new Column("PLAN", Value.STRING));
+
+        return res;
+    }
+
+    /**
      * @param conn Connection.
      * @param qry Query.
      * @return Table.
@@ -573,7 +652,7 @@ public class GridReduceQueryExecutor {
         private CountDownLatch latch;
 
         /** */
-        private Connection conn;
+        private JdbcConnection conn;
 
         /** */
         private int pageSize;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eab2e538/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 c2722fa..3159589 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
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.datastructures.*;
 import org.apache.ignite.internal.processors.query.*;
+import org.apache.ignite.internal.processors.query.h2.sql.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.spi.discovery.*;
@@ -298,6 +299,26 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
         }
     }
 
+    /**
+     *
+     */
+    public void testExplain() {
+        List<List<?>> res = grid(0).cache(null).query(new SqlFieldsQuery(
+            "explain select p.age, p.name, o.name " +
+            "from Person p, Organization o where p.orgId = o.id")).getAll();
+
+        for (List<?> row : res)
+            X.println("____ : " + row);
+
+        if (cacheMode() == PARTITIONED) {
+            assertEquals(2, res.size());
+
+            assertTrue(((String)res.get(1).get(0)).contains(GridSqlQuerySplitter.TABLE_FUNC_NAME));
+        }
+        else
+            assertEquals(1, res.size());
+    }
+
     /** @throws Exception If failed. */
     public void testExecuteWithMetaData() throws Exception {
         QueryCursorImpl<List<?>> cursor = (QueryCursorImpl<List<?>>)grid(0).cache(null).query(new SqlFieldsQuery(