You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2016/08/26 09:16:15 UTC

[11/50] ignite git commit: ignite-2795 Support 'copyOnRead' for SQL queries

ignite-2795 Support 'copyOnRead' for SQL queries


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

Branch: refs/heads/ignite-3443
Commit: 612eb3daffe608995aac28eed019b3e6ef9d66d3
Parents: 47374d9
Author: Aleksei Scherbakov <al...@gmail.com>
Authored: Fri Aug 19 13:28:39 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Aug 19 13:30:45 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |   8 +
 .../query/h2/GridH2ResultSetIterator.java       |  62 ++++-
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |  11 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  34 ++-
 .../h2/twostep/GridReduceQueryExecutor.java     |   3 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   | 226 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite2.java         |   2 +
 8 files changed, 338 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/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 e0ecf1c..8b66727 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
@@ -1718,6 +1718,14 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return {@code True} if the value for the cache object has to be copied because
+     * of {@link CacheConfiguration#isCopyOnRead()}.
+     */
+    public boolean needValueCopy() {
+        return affNode && cacheCfg.isCopyOnRead() && cacheCfg.getMemoryMode() != OFFHEAP_VALUES;
+    }
+
+    /**
      * Converts temporary offheap object to heap-based.
      *
      * @param obj Object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
index 3603bb5..e0680d3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
@@ -17,24 +17,49 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.lang.reflect.Field;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.NoSuchElementException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-
+import org.h2.jdbc.JdbcResultSet;
+import org.h2.result.ResultInterface;
+import org.h2.value.Value;
 
 /**
  * Iterator over result set.
  */
 public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAdapter<T> {
     /** */
+    private static final Field RESULT_FIELD;
+
+    /**
+     * Initialize.
+     */
+    static {
+        try {
+            RESULT_FIELD = JdbcResultSet.class.getDeclaredField("result");
+
+            RESULT_FIELD.setAccessible(true);
+        }
+        catch (NoSuchFieldException e) {
+            throw new IllegalStateException("Check H2 version in classpath.", e);
+        }
+    }
+
+    /** */
     private static final long serialVersionUID = 0L;
 
     /** */
+    private final ResultInterface res;
+
+    /** */
     private final ResultSet data;
 
     /** */
@@ -49,12 +74,20 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
     /**
      * @param data Data array.
      * @param closeStmt If {@code true} closes result set statement when iterator is closed.
+     * @param needCpy {@code True} if need copy cache object's value.
      * @throws IgniteCheckedException If failed.
      */
-    protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt) throws IgniteCheckedException {
+    protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt, boolean needCpy) throws IgniteCheckedException {
         this.data = data;
         this.closeStmt = closeStmt;
 
+        try {
+            res = needCpy ? (ResultInterface)RESULT_FIELD.get(data) : null;
+        }
+        catch (IllegalAccessException e) {
+            throw new IllegalStateException(e); // Must not happen.
+        }
+
         if (data != null) {
             try {
                 row = new Object[data.getMetaData().getColumnCount()];
@@ -78,8 +111,27 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
             if (!data.next())
                 return false;
 
-            for (int c = 0; c < row.length; c++)
-                row[c] = data.getObject(c + 1);
+            if (res != null) {
+                Value[] values = res.currentRow();
+
+                for (int c = 0; c < row.length; c++) {
+                    Value val = values[c];
+
+                    if (val instanceof GridH2ValueCacheObject) {
+                        GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)values[c];
+
+                        GridCacheContext cctx = valCacheObj.getCacheContext();
+
+                        row[c] = valCacheObj.getObject(cctx != null && cctx.needValueCopy());
+                    }
+                    else
+                        row[c] = val.getObject();
+                }
+            }
+            else {
+                for (int c = 0; c < row.length; c++)
+                    row[c] = data.getObject(c + 1);
+            }
 
             return true;
         }
@@ -134,6 +186,6 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString((Class<GridH2ResultSetIterator>)getClass(), this);
+        return S.toString(GridH2ResultSetIterator.class, this);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/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 e1b21e2..2e8b93f 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
@@ -2484,7 +2484,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @throws IgniteCheckedException If failed.
          */
         protected FieldsIterator(ResultSet data) throws IgniteCheckedException {
-            super(data, false);
+            super(data, false, true);
         }
 
         /** {@inheritDoc} */
@@ -2509,7 +2509,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @throws IgniteCheckedException If failed.
          */
         protected KeyValIterator(ResultSet data) throws IgniteCheckedException {
-            super(data, false);
+            super(data, false, true);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
index 80e8504..a80a15b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
@@ -21,7 +21,6 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Types;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -123,7 +122,15 @@ public class GridH2ValueCacheObject extends Value {
 
     /** {@inheritDoc} */
     @Override public Object getObject() {
-        return obj.isPlatformType() ? obj.value(objectContext(), false) : obj;
+        return getObject(false);
+    }
+
+    /**
+     * @param cpy Copy flag.
+     * @return Value.
+     */
+    public Object getObject(boolean cpy) {
+        return obj.isPlatformType() ? obj.value(objectContext(), cpy) : obj;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/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 bb5e419..d51306c 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
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest;
@@ -900,6 +901,9 @@ public class GridMapQueryExecutor {
         private final int rowCnt;
 
         /** */
+        private boolean cpNeeded;
+
+        /** */
         private volatile boolean closed;
 
         /**
@@ -908,11 +912,12 @@ public class GridMapQueryExecutor {
          * @param qrySrcNodeId Query source node.
          * @param qry Query.
          */
-        private QueryResult(ResultSet rs, GridCacheContext<?,?> cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) {
+        private QueryResult(ResultSet rs, GridCacheContext<?, ?> cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) {
             this.rs = rs;
             this.cctx = cctx;
             this.qry = qry;
             this.qrySrcNodeId = qrySrcNodeId;
+            this.cpNeeded = cctx.isLocalNode(qrySrcNodeId);
 
             try {
                 res = (ResultInterface)RESULT_FIELD.get(rs);
@@ -944,6 +949,33 @@ public class GridMapQueryExecutor {
 
                 Value[] row = res.currentRow();
 
+                if (cpNeeded) {
+                    boolean copied = false;
+
+                    for (int j = 0; j < row.length; j++) {
+                        Value val = row[j];
+
+                        if (val instanceof GridH2ValueCacheObject) {
+                            GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)val;
+
+                            GridCacheContext cctx = valCacheObj.getCacheContext();
+
+                            if (cctx != null && cctx.needValueCopy()) {
+                                row[j] = new GridH2ValueCacheObject(valCacheObj.getCacheContext(), valCacheObj.getCacheObject()) {
+                                    @Override public Object getObject() {
+                                        return getObject(true);
+                                    }
+                                };
+
+                                copied = true;
+                            }
+                        }
+                    }
+
+                    if (i == 0 && !copied)
+                        cpNeeded = false; // No copy on read caches, skip next checks.
+                }
+
                 assert row != null;
 
                 if (readEvt) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/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 04449ac..6a576ee 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
@@ -31,7 +31,6 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableMap;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
@@ -1320,7 +1319,7 @@ public class GridReduceQueryExecutor {
          * @throws IgniteCheckedException If failed.
          */
         protected Iter(ResultSet data) throws IgniteCheckedException {
-            super(data, true);
+            super(data, true, false);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
new file mode 100644
index 0000000..e47e893
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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.cache;
+
+import java.util.List;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests modification of values returned by query iterators with enabled copy on read.
+ */
+public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int KEYS = 100;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if ("client".equals(cfg.getGridName()))
+            cfg.setClientMode(true);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        CacheConfiguration<Integer, Value> cc = new CacheConfiguration<>();
+
+        cc.setCopyOnRead(true);
+        cc.setIndexedTypes(Integer.class, Value.class);
+
+        cfg.setCacheConfiguration(cc);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        for (int i = 0; i < KEYS; i++)
+            cache.put(i, new Value("before"));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        cache.removeAll();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * Tests two step query from dedicated client.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTwoStepSqlClientQuery() throws Exception {
+        try (Ignite client = startGrid("client")) {
+            IgniteCache<Integer, Value> cache = client.cache(null);
+
+            List<Cache.Entry<Integer, Value>> all = cache.query(
+                new SqlQuery<Integer, Value>(Value.class, "select * from Value")).getAll();
+
+            assertEquals(KEYS, all.size());
+
+            for (Cache.Entry<Integer, Value> entry : all)
+                entry.getValue().str = "after";
+
+            check(cache);
+
+            QueryCursor<List<?>> qry = cache.query(new SqlFieldsQuery("select _val from Value"));
+
+            List<List<?>> all0 = qry.getAll();
+
+            assertEquals(KEYS, all0.size());
+
+            for (List<?> entry : all0)
+                ((Value)entry.get(0)).str = "after";
+
+            check(cache);
+        }
+    }
+
+    /**
+     * Test two step query without local reduce phase.
+     */
+    public void testTwoStepSkipReduceSqlQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        List<Cache.Entry<Integer, Value>> all = cache.query(
+            new SqlQuery<Integer, Value>(Value.class, "select * from Value").setPageSize(3)).getAll();
+
+        assertEquals(KEYS, all.size());
+
+        for (Cache.Entry<Integer, Value> entry : all)
+            entry.getValue().str = "after";
+
+        check(cache);
+    }
+
+    /**
+     * Test two step query value copy.
+     */
+    public void testTwoStepReduceSqlQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        QueryCursor<List<?>> qry = cache.query(new SqlFieldsQuery("select _val from Value order by _key"));
+
+        List<List<?>> all = qry.getAll();
+
+        assertEquals(KEYS, all.size());
+
+        for (List<?> entry : all)
+            ((Value)entry.get(0)).str = "after";
+
+        check(cache);
+    }
+
+    /**
+     * Tests local sql query.
+     */
+    public void testLocalSqlQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        SqlQuery<Integer, Value> qry = new SqlQuery<>(Value.class.getSimpleName(), "select * from Value");
+        qry.setLocal(true);
+
+        List<Cache.Entry<Integer, Value>> all = cache.query(qry).getAll();
+
+        assertFalse(all.isEmpty());
+
+        for (Cache.Entry<Integer, Value> entry : all)
+            entry.getValue().str = "after";
+
+        check(cache);
+    }
+
+    /**
+     * Tests local sql query.
+     */
+    public void testLocalSqlFieldsQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        QueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("select _val from Value").setLocal(true));
+
+        List<List<?>> all = cur.getAll();
+
+        assertFalse(all.isEmpty());
+
+        for (List<?> entry : all)
+            ((Value)entry.get(0)).str = "after";
+
+        check(cache);
+    }
+
+    /** */
+    private static class Value {
+        /** */
+        private String str;
+
+        /**
+         * @param str String.
+         */
+        public Value(String str) {
+            this.str = str;
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private void check(IgniteCache<Integer, Value> cache) {
+        int cnt = 0;
+
+        // Value should be not modified by previous assignment.
+        for (Cache.Entry<Integer, Value> entry : cache) {
+            cnt++;
+
+            assertEquals("before", entry.getValue().str);
+        }
+
+        assertEquals(KEYS, cnt);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/612eb3da/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 7cb1adb..f96c1d5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLos
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
@@ -95,6 +96,7 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
         // Other.
         suite.addTestSuite(CacheQueryNewClientSelfTest.class);
         suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class);
+        suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class);
 
         return suite;
     }