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 2017/05/10 15:54:47 UTC

[1/2] ignite git commit: IGNITE-5054: Preliminar refactoring for decoupled SQL schema. This closes #1923.

Repository: ignite
Updated Branches:
  refs/heads/master 51069c5ca -> 00ff6563f


IGNITE-5054: Preliminar refactoring for decoupled SQL schema. This closes #1923.


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

Branch: refs/heads/master
Commit: 3bc828afda9929d46f963497f93c7ad6ff3f8a0f
Parents: 68b6838
Author: devozerov <vo...@gridgain.com>
Authored: Wed May 10 18:53:20 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 10 18:53:20 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |   7 +
 .../processors/cache/IgniteCacheProxy.java      |  41 +-----
 .../processors/query/GridQueryIndexing.java     |  36 ++---
 .../processors/query/GridQueryProcessor.java    | 132 +++++++++++--------
 .../query/h2/DmlStatementsProcessor.java        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  14 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |  12 +-
 7 files changed, 122 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 92c144c..8d562c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -414,6 +414,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return {@code true} If this is a replicated cache and we are on a data node.
+     */
+    public boolean isReplicatedAffinityNode() {
+        return isReplicated() && affinityNode();
+    }
+
+    /**
      * @throws IgniteCheckedException If failed to wait.
      */
     public void awaitStarted() throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index dfe817e..c3a24fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -680,7 +680,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
      * @return Local node cluster group.
      */
     private ClusterGroup projection(boolean loc) {
-        if (loc || ctx.isLocal() || isReplicatedDataNode())
+        if (loc || ctx.isLocal() || ctx.isReplicatedAffinityNode())
             return ctx.kernalContext().grid().cluster().forLocal();
 
         if (ctx.isReplicated())
@@ -774,37 +774,13 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                     opCtxCall != null && opCtxCall.isKeepBinary());
 
             if (qry instanceof SqlQuery) {
-                final SqlQuery p = (SqlQuery)qry;
+                boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary();
 
-                if (p.isReplicatedOnly() && p.getPartitions() != null)
-                    throw new CacheException("Partitions are not supported in replicated only mode.");
-
-                if (p.isDistributedJoins() && p.getPartitions() != null)
-                    throw new CacheException(
-                        "Using both partitions and distributed JOINs is not supported for the same query");
-
-                if ((p.isReplicatedOnly() && isReplicatedDataNode()) || ctx.isLocal() || qry.isLocal())
-                     return (QueryCursor<R>)ctx.kernalContext().query().queryLocal(ctx, p,
-                                opCtxCall != null && opCtxCall.isKeepBinary());
-
-                return (QueryCursor<R>)ctx.kernalContext().query().queryTwoStep(ctx, p);
+                return (QueryCursor<R>)ctx.kernalContext().query().querySql(ctx, (SqlQuery)qry, keepBinary);
             }
 
-            if (qry instanceof SqlFieldsQuery) {
-                SqlFieldsQuery p = (SqlFieldsQuery)qry;
-
-                if (p.isReplicatedOnly() && p.getPartitions() != null)
-                    throw new CacheException("Partitions are not supported in replicated only mode.");
-
-                if (p.isDistributedJoins() && p.getPartitions() != null)
-                    throw new CacheException(
-                        "Using both partitions and distributed JOINs is not supported for the same query");
-
-                if ((p.isReplicatedOnly() && isReplicatedDataNode()) || ctx.isLocal() || qry.isLocal())
-                    return (QueryCursor<R>)ctx.kernalContext().query().queryLocalFields(ctx, p);
-
-                return (QueryCursor<R>)ctx.kernalContext().query().queryTwoStep(ctx, p);
-            }
+            if (qry instanceof SqlFieldsQuery)
+                return (QueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry);
 
             if (qry instanceof ScanQuery)
                 return query((ScanQuery)qry, null, projection(qry.isLocal()));
@@ -891,13 +867,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /**
-     * @return {@code true} If this is a replicated cache and we are on a data node.
-     */
-    private boolean isReplicatedDataNode() {
-        return ctx.isReplicated() && ctx.affinityNode();
-    }
-
-    /**
      * Checks query.
      *
      * @param qry Query

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 0afba59..8180cd0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -67,11 +67,10 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
-     * @param cancel Query cancel.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry, GridQueryCancel cancel)
+    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?,?> cctx, SqlQuery qry)
         throws IgniteCheckedException;
 
     /**
@@ -79,23 +78,12 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param cancel Query cancel.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry)
-        throws IgniteCheckedException;
-
-    /**
-     * Queries individual fields (generally used by JDBC drivers).
-     *
-     * @param cctx Cache context.
-     * @param qry Query.
-     * @param filter Space name and key filter.
-     * @param cancel Query cancel.
-     * @return Cursor.
-     */
-    public <K, V> QueryCursor<List<?>> queryLocalSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
+    public QueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+        GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Perform a MERGE statement using data streamer as receiver.
@@ -107,8 +95,8 @@ public interface GridQueryIndexing {
      * @return Query result.
      * @throws IgniteCheckedException If failed.
      */
-    public long streamUpdateQuery(final String spaceName, final String qry,
-         @Nullable final Object[] params, IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException;
+    public long streamUpdateQuery(final String spaceName, final String qry, @Nullable final Object[] params,
+        IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException;
 
     /**
      * Executes regular query.
@@ -123,6 +111,18 @@ public interface GridQueryIndexing {
         IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException;
 
     /**
+     * Queries individual fields (generally used by JDBC drivers).
+     *
+     * @param cctx Cache context.
+     * @param qry Query.
+     * @param filter Space name and key filter.
+     * @param cancel Query cancel.
+     * @return Cursor.
+     */
+    public QueryCursor<List<?>> queryLocalSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+        IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
+
+    /**
      * Executes text query.
      *
      * @param spaceName Space name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 448639b..13525f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1643,25 +1643,70 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Query SQL fields.
+     *
      * @param cctx Cache context.
      * @param qry Query.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> queryTwoStep(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
+    @SuppressWarnings("unchecked")
+    public QueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
         checkxEnabled();
 
+        if (qry.isReplicatedOnly() && qry.getPartitions() != null)
+            throw new CacheException("Partitions are not supported in replicated only mode.");
+
+        if (qry.isDistributedJoins() && qry.getPartitions() != null)
+            throw new CacheException(
+                "Using both partitions and distributed JOINs is not supported for the same query");
+
+        boolean loc = (qry.isReplicatedOnly() && cctx.isReplicatedAffinityNode()) || cctx.isLocal() || qry.isLocal();
+
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
-                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
-                    return idx.queryTwoStep(cctx, qry, null);
-                }
-            }, true);
+            IgniteOutClosureX<QueryCursor<List<?>>> clo;
+
+            if (loc) {
+                clo = new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                    @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                        GridQueryCancel cancel = new GridQueryCancel();
+
+                        final QueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry,
+                            idx.backupFilter(requestTopVer.get(), qry.getPartitions()), cancel);
+
+                        Iterable<List<?>> iterExec = new Iterable<List<?>>() {
+                            @Override public Iterator<List<?>> iterator() {
+                                sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx.name());
+
+                                return cursor.iterator();
+                            }
+                        };
+
+                        return new QueryCursorImpl<List<?>>(iterExec, cancel) {
+                            @Override public List<GridQueryFieldMetadata> fieldsMeta() {
+                                if (cursor instanceof QueryCursorImpl)
+                                    return ((QueryCursorEx)cursor).fieldsMeta();
+
+                                return super.fieldsMeta();
+                            }
+                        };
+                    }
+                };
+            }
+            else {
+                clo = new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                    @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                        return idx.queryDistributedSqlFields(cctx, qry, null);
+                    }
+                };
+            }
+
+            return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, clo, true);
         }
         catch (IgniteCheckedException e) {
-            throw new IgniteException(e);
+            throw new CacheException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -1699,11 +1744,35 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Execute distributed SQL query.
+     *
+     * @param cctx Cache context.
+     * @param qry Query.
+     * @param keepBinary Keep binary flag.
+     * @return Cursor.
+     */
+    public <K, V> QueryCursor<Cache.Entry<K,V>> querySql(final GridCacheContext<?,?> cctx, final SqlQuery qry,
+        boolean keepBinary) {
+        if (qry.isReplicatedOnly() && qry.getPartitions() != null)
+            throw new CacheException("Partitions are not supported in replicated only mode.");
+
+        if (qry.isDistributedJoins() && qry.getPartitions() != null)
+            throw new CacheException(
+                "Using both partitions and distributed JOINs is not supported for the same query");
+
+        if ((qry.isReplicatedOnly() && cctx.isReplicatedAffinityNode()) || cctx.isLocal() || qry.isLocal())
+            return queryLocalSql(cctx, qry, keepBinary);
+
+        return queryDistributedSql(cctx, qry);
+    }
+
+    /**
      * @param cctx Cache context.
      * @param qry Query.
      * @return Cursor.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(final GridCacheContext<?,?> cctx, final SqlQuery qry) {
+    public <K,V> QueryCursor<Cache.Entry<K,V>> queryDistributedSql(final GridCacheContext<?,?> cctx,
+        final SqlQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
@@ -1713,7 +1782,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx,
                 new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
                     @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
-                        return idx.queryTwoStep(cctx, qry);
+                        return idx.queryDistributedSql(cctx, qry);
                     }
                 }, true);
         }
@@ -1731,7 +1800,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocal(
+    public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(
         final GridCacheContext<?, ?> cctx,
         final SqlQuery qry,
         final boolean keepBinary
@@ -1923,49 +1992,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param cctx Cache context.
-     * @param qry Query.
-     * @return Iterator.
-     */
-    @SuppressWarnings("unchecked")
-    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?, ?> cctx, final SqlFieldsQuery qry) {
-        if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to execute query (grid is stopping).");
-
-        try {
-            return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
-                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
-                    GridQueryCancel cancel = new GridQueryCancel();
-
-                    final QueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry,
-                        idx.backupFilter(requestTopVer.get(), qry.getPartitions()), cancel);
-
-                    return new QueryCursorImpl<List<?>>(new Iterable<List<?>>() {
-                        @Override public Iterator<List<?>> iterator() {
-                            sendQueryExecutedEvent(qry.getSql(), qry.getArgs(), cctx.name());
-
-                            return cursor.iterator();
-                        }
-                    }, cancel) {
-                        @Override public List<GridQueryFieldMetadata> fieldsMeta() {
-                            if (cursor instanceof QueryCursorImpl)
-                                return ((QueryCursorEx)cursor).fieldsMeta();
-
-                            return super.fieldsMeta();
-                        }
-                    };
-                }
-            }, true);
-        }
-        catch (IgniteCheckedException e) {
-            throw new CacheException(e);
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
      * @param space Space.
      * @param key Key.
      * @throws IgniteCheckedException Thrown in case of any errors.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index d4109f3..5fee5ed 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -350,7 +350,7 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>) idx.queryTwoStep(cctx, newFieldsQry, cancel);
+            cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(cctx, newFieldsQry, cancel);
         }
         else {
             final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/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 d62d7d8..1c49fc7 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
@@ -115,7 +115,6 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2SystemIndexFactory;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2ProxyIndex;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
@@ -166,7 +165,6 @@ import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Cursor;
 import org.h2.index.Index;
-import org.h2.index.SpatialIndex;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.jdbc.JdbcStatement;
@@ -1335,7 +1333,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public <K, V> QueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
+    @Override public QueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
         final SqlFieldsQuery qry, final IndexingQueryFilter filter, final GridQueryCancel cancel)
         throws IgniteCheckedException {
 
@@ -1344,7 +1342,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             assert qry.isLocal();
 
-            return queryTwoStep(cctx, qry, cancel);
+            return queryDistributedSqlFields(cctx, qry, cancel);
         }
         else {
             final boolean keepBinary = cctx.keepBinary();
@@ -1381,7 +1379,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             assert qry.isLocal();
 
-            return queryTwoStep(cctx, qry);
+            return queryDistributedSql(cctx, qry);
         }
         else {
             String space = cctx.name();
@@ -1497,7 +1495,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlQuery qry) {
+    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?, ?> cctx, SqlQuery qry) {
         String type = qry.getType();
         String space = cctx.name();
 
@@ -1527,7 +1525,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (qry.getTimeout() > 0)
             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
 
-        final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry, null);
+        final QueryCursor<List<?>> res = queryDistributedSqlFields(cctx, fqry, null);
 
         final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
             @Override public Iterator<Cache.Entry<K, V>> iterator() {
@@ -1568,7 +1566,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
+    @Override public QueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
         GridQueryCancel cancel) {
         final String space = cctx.name();
         final String sqlQry = qry.getSql();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bc828af/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
index 43cdec0..42a71cd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
@@ -127,7 +127,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
         SqlFieldsQuery qry = new SqlFieldsQuery("select f.productId, p.name, f.price " +
             "from FactPurchase f, \"replicated-prod\".DimProduct p where p.id = f.productId ");
 
-        for (List<?> o : qryProc.queryTwoStep(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
             X.println("___ -> " + o);
 
             set1.add((Integer)o.get(0));
@@ -141,7 +141,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
 
         qry = new SqlFieldsQuery("select productId from FactPurchase group by productId");
 
-        for (List<?> o : qryProc.queryTwoStep(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
             X.println("___ -> " + o);
 
             assertTrue(set0.add((Integer) o.get(0)));
@@ -160,7 +160,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
             "where p.id = f.productId " +
             "group by f.productId, p.name");
 
-        for (List<?> o : qryProc.queryTwoStep(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
             X.println("___ -> " + o);
 
             assertTrue(names.add((String)o.get(0)));
@@ -177,7 +177,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
             "group by f.productId, p.name " +
             "having s >= 15");
 
-        for (List<?> o : qryProc.queryTwoStep(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
             X.println("___ -> " + o);
 
             assertTrue(i(o, 1) >= 15);
@@ -190,7 +190,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
         qry = new SqlFieldsQuery("select top 3 distinct productId " +
             "from FactPurchase f order by productId desc ");
 
-        for (List<?> o : qryProc.queryTwoStep(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
             X.println("___ -> " + o);
 
             assertEquals(top--, o.get(0));
@@ -203,7 +203,7 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
         qry = new SqlFieldsQuery("select distinct productId " +
             "from FactPurchase f order by productId desc limit 2 offset 1");
 
-        for (List<?> o : qryProc.queryTwoStep(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry).getAll()) {
             X.println("___ -> " + o);
 
             assertEquals(top--, o.get(0));


[2/2] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/master
Commit: 00ff6563f781d99f86f7b5049ad655e804f3f034
Parents: 3bc828a 51069c5
Author: devozerov <vo...@gridgain.com>
Authored: Wed May 10 18:54:38 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 10 18:54:38 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheEntryInfo.java    | 68 +++-----------------
 .../GridDhtPartitionSupplyMessage.java          | 30 +--------
 2 files changed, 12 insertions(+), 86 deletions(-)
----------------------------------------------------------------------