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 2017/05/24 13:36:26 UTC

[21/31] ignite git commit: IGNITE-5282: SQL: Correct top-bottom propagation of "keepBinary" flag. This closes #1993.

IGNITE-5282: SQL: Correct top-bottom propagation of "keepBinary" flag. This closes #1993.


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

Branch: refs/heads/ignite-5075
Commit: 33cb5e89c011cc0029898fd87e6b47e75809cf0c
Parents: eea7a3b
Author: devozerov <vo...@gridgain.com>
Authored: Wed May 24 00:02:26 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed May 24 00:02:26 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheProxy.java      | 15 +++++-----
 .../processors/query/GridQueryIndexing.java     | 11 +++++---
 .../processors/query/GridQueryProcessor.java    | 27 +++++++++---------
 .../query/h2/DmlStatementsProcessor.java        | 21 ++++++++------
 .../processors/query/h2/IgniteH2Indexing.java   | 29 +++++++++-----------
 .../cache/GridCacheCrossCacheQuerySelfTest.java | 12 ++++----
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  4 +--
 7 files changed, 62 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 40aafeb..a7d0d19 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
@@ -774,20 +774,19 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
             convertToBinary(qry);
 
-            final CacheOperationContext opCtxCall = ctx.operationContextPerCall();
+            CacheOperationContext opCtxCall = ctx.operationContextPerCall();
 
-            if (qry instanceof ContinuousQuery)
-                return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal(),
-                    opCtxCall != null && opCtxCall.isKeepBinary());
+            boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary();
 
-            if (qry instanceof SqlQuery) {
-                boolean keepBinary = opCtxCall != null && opCtxCall.isKeepBinary();
+            if (qry instanceof ContinuousQuery)
+                return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal(), keepBinary);
 
+            if (qry instanceof SqlQuery)
                 return (QueryCursor<R>)ctx.kernalContext().query().querySql(ctx, (SqlQuery)qry, keepBinary);
-            }
 
             if (qry instanceof SqlFieldsQuery)
-                return (FieldsQueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry);
+                return (FieldsQueryCursor<R>)ctx.kernalContext().query().querySqlFields(ctx, (SqlFieldsQuery)qry,
+                    keepBinary);
 
             if (qry instanceof ScanQuery)
                 return query((ScanQuery)qry, null, projection(qry.isLocal()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 1f5e701..4b9c0e2 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
@@ -68,23 +68,25 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?,?> cctx, SqlQuery qry)
-        throws IgniteCheckedException;
+    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?,?> cctx, SqlQuery qry,
+        boolean keepBinary) throws IgniteCheckedException;
 
     /**
      * Parses SQL query into two step query and executes it.
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @param cancel Query cancel.
      * @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
     public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        GridQueryCancel cancel) throws IgniteCheckedException;
+        boolean keepBinary, GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Perform a MERGE statement using data streamer as receiver.
@@ -116,12 +118,13 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @param filter Cache name and key filter.
      * @param cancel Query cancel.
      * @return Cursor.
      */
     public FieldsQueryCursor<List<?>> queryLocalSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
+        boolean keepBinary, IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Executes text query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 ced78cf..a40c9e9 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
@@ -1651,10 +1651,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
     @SuppressWarnings("unchecked")
-    public FieldsQueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
+    public FieldsQueryCursor<List<?>> querySqlFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry,
+        final boolean keepBinary) {
         checkxEnabled();
 
         if (qry.isReplicatedOnly() && qry.getPartitions() != null)
@@ -1677,7 +1679,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
                         GridQueryCancel cancel = new GridQueryCancel();
 
-                        final FieldsQueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry,
+                        final FieldsQueryCursor<List<?>> cursor = idx.queryLocalSqlFields(cctx, qry, keepBinary,
                             idx.backupFilter(requestTopVer.get(), qry.getPartitions()), cancel);
 
                         Iterable<List<?>> iterExec = new Iterable<List<?>>() {
@@ -1702,7 +1704,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             else {
                 clo = new IgniteOutClosureX<FieldsQueryCursor<List<?>>>() {
                     @Override public FieldsQueryCursor<List<?>> applyx() throws IgniteCheckedException {
-                        return idx.queryDistributedSqlFields(cctx, qry, null);
+                        return idx.queryDistributedSqlFields(cctx, qry, keepBinary, null);
                     }
                 };
             }
@@ -1767,16 +1769,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if ((qry.isReplicatedOnly() && cctx.isReplicatedAffinityNode()) || cctx.isLocal() || qry.isLocal())
             return queryLocalSql(cctx, qry, keepBinary);
 
-        return queryDistributedSql(cctx, qry);
+        return queryDistributedSql(cctx, qry, keepBinary);
     }
 
     /**
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryDistributedSql(final GridCacheContext<?,?> cctx,
-        final SqlQuery qry) {
+    private <K,V> QueryCursor<Cache.Entry<K,V>> queryDistributedSql(final GridCacheContext<?,?> cctx,
+        final SqlQuery qry, final boolean keepBinary) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
@@ -1786,7 +1789,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.queryDistributedSql(cctx, qry);
+                        return idx.queryDistributedSql(cctx, qry, keepBinary);
                     }
                 }, true);
         }
@@ -1804,11 +1807,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(
-        final GridCacheContext<?, ?> cctx,
-        final SqlQuery qry,
-        final boolean keepBinary
-    ) {
+    private <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(final GridCacheContext<?, ?> cctx, final SqlQuery qry,
+        final boolean keepBinary) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
@@ -1827,7 +1827,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             qry.getArgs(),
                             cctx.name());
 
-                        return idx.queryLocalSql(cctx, qry, idx.backupFilter(requestTopVer.get(), qry.getPartitions()), keepBinary);
+                        return idx.queryLocalSql(cctx, qry, idx.backupFilter(requestTopVer.get(), qry.getPartitions()),
+                            keepBinary);
                     }
                 }, true);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 db7bfd6..e40c328 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
@@ -141,7 +141,8 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     private UpdateResult updateSqlFields(String cacheName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
-        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
+        boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
         Object[] errKeys = null;
 
         long items = 0;
@@ -196,13 +197,13 @@ public class DmlStatementsProcessor {
     /**
      * @param cacheName Cache name.
      * @param stmt Prepared statement.
-     * @param fieldsQry Initial query.
+     * @param fieldsQry Initial query
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String cacheName, PreparedStatement stmt,
+    QueryCursorImpl<List<?>> updateSqlFieldsDistributed(String cacheName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
         UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, false, null, cancel);
 
@@ -218,14 +219,16 @@ public class DmlStatementsProcessor {
      * Execute DML statement on local cache.
      * @param cacheName Cache name.
      * @param stmt Prepared statement.
+     * @param fieldsQry Fields query.
      * @param filters Cache name and key filter.
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    GridQueryFieldsResult updateLocalSqlFields(String cacheName, PreparedStatement stmt,
-        SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
+    GridQueryFieldsResult updateSqlFieldsLocal(String cacheName, PreparedStatement stmt,
+        SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel)
+        throws IgniteCheckedException {
         UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, true, filters, cancel);
 
         return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
@@ -266,8 +269,8 @@ public class DmlStatementsProcessor {
 
             final ArrayList<List<?>> data = new ArrayList<>(plan.rowsNum);
 
-            final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,
-                F.asList(args), null, false, 0, null);
+            final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry, F.asList(args),
+                null, false, 0, null);
 
             QueryCursorImpl<List<?>> stepCur = new QueryCursorImpl<>(new Iterable<List<?>>() {
                 @Override public Iterator<List<?>> iterator() {
@@ -315,8 +318,10 @@ public class DmlStatementsProcessor {
 
     /**
      * Actually perform SQL DML operation locally.
+     *
      * @param cctx Cache context.
      * @param prepStmt Prepared statement for DML query.
+     * @param fieldsQry Fields query.
      * @param filters Cache name and key filter.
      * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
      * @return Pair [number of successfully processed items; keys that have failed to be processed]
@@ -351,7 +356,7 @@ public class DmlStatementsProcessor {
                 .setPageSize(fieldsQry.getPageSize())
                 .setTimeout(fieldsQry.getTimeout(), TimeUnit.MILLISECONDS);
 
-            cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(cctx, newFieldsQry, cancel);
+            cur = (QueryCursorImpl<List<?>>) idx.queryDistributedSqlFields(cctx, newFieldsQry, true, cancel);
         }
         else {
             final GridQueryFieldsResult res = idx.queryLocalSqlFields(cctx.name(), plan.selectQry,

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 0ce905b..d3ee6ff 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
@@ -1027,8 +1027,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     @SuppressWarnings("unchecked")
     public GridQueryFieldsResult queryLocalSqlFields(final String cacheName, final String qry,
         @Nullable final Collection<Object> params, final IndexingQueryFilter filter, boolean enforceJoinOrder,
-        final int timeout, final GridQueryCancel cancel)
-        throws IgniteCheckedException {
+        final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException {
         final Connection conn = connectionForCache(cacheName);
 
         setupConnection(conn, false, enforceJoinOrder);
@@ -1046,7 +1045,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             fldsQry.setEnforceJoinOrder(enforceJoinOrder);
             fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
 
-            return dmlProc.updateLocalSqlFields(cacheName, stmt, fldsQry, filter, cancel);
+            return dmlProc.updateSqlFieldsLocal(cacheName, stmt, fldsQry, filter, cancel);
         }
         else if (DdlStatementsProcessor.isDdlStatement(p))
             throw new IgniteSQLException("DDL statements are supported for the whole cluster only",
@@ -1334,19 +1333,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryLocalSqlFields(final GridCacheContext<?, ?> cctx,
-        final SqlFieldsQuery qry, final IndexingQueryFilter filter, final GridQueryCancel cancel)
-        throws IgniteCheckedException {
+        final SqlFieldsQuery qry, final boolean keepBinary, final IndexingQueryFilter filter,
+        final GridQueryCancel cancel) throws IgniteCheckedException {
 
         if (cctx.config().getQueryParallelism() > 1) {
             qry.setDistributedJoins(true);
 
             assert qry.isLocal();
 
-            return queryDistributedSqlFields(cctx, qry, cancel);
+            return queryDistributedSqlFields(cctx, qry, keepBinary, cancel);
         }
         else {
-            final boolean keepBinary = cctx.keepBinary();
-
             final String cacheName = cctx.name();
             final String sql = qry.getSql();
             final Object[] args = qry.getArgs();
@@ -1379,7 +1376,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             assert qry.isLocal();
 
-            return queryDistributedSql(cctx, qry);
+            return queryDistributedSql(cctx, qry, keepBinary);
         }
         else {
             String cacheName = cctx.name();
@@ -1495,7 +1492,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?, ?> cctx, SqlQuery qry) {
+    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(GridCacheContext<?, ?> cctx,
+        SqlQuery qry, boolean keepBinary) {
         String type = qry.getType();
         String cacheName = cctx.name();
 
@@ -1525,7 +1523,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (qry.getTimeout() > 0)
             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
 
-        final QueryCursor<List<?>> res = queryDistributedSqlFields(cctx, fqry, null);
+        final QueryCursor<List<?>> res = queryDistributedSqlFields(cctx, fqry, keepBinary, null);
 
         final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
             @Override public Iterator<Cache.Entry<K, V>> iterator() {
@@ -1567,7 +1565,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> queryDistributedSqlFields(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry,
-        GridQueryCancel cancel) {
+        boolean keepBinary, GridQueryCancel cancel) {
         final String cacheName = cctx.name();
         final String sqlQry = qry.getSql();
 
@@ -1657,7 +1655,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 if (twoStepQry == null) {
                     if (DmlStatementsProcessor.isDmlStatement(prepared)) {
                         try {
-                            return dmlProc.updateSqlFieldsTwoStep(cctx.name(), stmt, qry, cancel);
+                            return dmlProc.updateSqlFieldsDistributed(cctx.name(), stmt, qry, cancel);
                         }
                         catch (IgniteCheckedException e) {
                             throw new IgniteSQLException("Failed to execute DML statement [stmt=" + sqlQry +
@@ -1727,9 +1725,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             cancel = new GridQueryCancel();
 
         QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-            runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder, qry.getTimeout(), cancel,
-                    qry.getArgs(), qry.getPartitions()),
-            cancel);
+            runQueryTwoStep(cctx, twoStepQry, keepBinary, enforceJoinOrder, qry.getTimeout(), cancel,
+                qry.getArgs(), qry.getPartitions()), cancel);
 
         cursor.fieldsMeta(meta);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/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 42a71cd..6344cba 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.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).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.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).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.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).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.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).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.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).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.querySqlFields(cache.context(), qry).getAll()) {
+        for (List<?> o : qryProc.querySqlFields(cache.context(), qry, false).getAll()) {
             X.println("___ -> " + o);
 
             assertEquals(top--, o.get(0));

http://git-wip-us.apache.org/repos/asf/ignite/blob/33cb5e89/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 7ba7d56..97db2c4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -468,8 +468,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
                 time = now;
                 range *= 3;
 
-                GridQueryFieldsResult res = spi.queryLocalSqlFields("A", sql, Arrays.<Object>asList(1, range), null,
-                    false, 0, null);
+                GridQueryFieldsResult res = spi.queryLocalSqlFields("A", sql, Arrays.<Object>asList(1, range),
+                    null, false, 0, null);
 
                 assert res.iterator().hasNext();