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/06/17 12:43:35 UTC

[22/50] incubator-ignite git commit: ignite-745 Fixed query metrics for partitioned cache

ignite-745 Fixed query metrics for partitioned cache


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

Branch: refs/heads/ignite-gg-10052
Commit: 1eb3bd24c50d9c6fdb6de3dedd0216bb6e5a81d8
Parents: f4b1123
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 10 19:23:28 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jun 16 13:38:30 2015 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 102 ++++++---
 .../CacheAbstractQueryMetricsSelfTest.java      | 205 ++++++++++++++++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 +++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 +++
 .../cache/GridCacheQueryMetricsSelfTest.java    | 206 -------------------
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 7 files changed, 344 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/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 fed90d5..718e6ba 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
@@ -513,8 +513,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(String space, String clause,
-        Collection<Object> params, String resType, IndexingQueryFilter filters)
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(final String space, final String clause,
+        final Collection<Object> params, final String resType, final IndexingQueryFilter filters)
         throws IgniteCheckedException {
         checkEnabled();
 
@@ -522,12 +522,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
 
-            if (type == null || !type.registered())
-                throw new CacheException("Failed to find SQL table for type: " + resType);
+            return executeQuery(cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
+                @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
+                    TypeDescriptor type = typesByName.get(new TypeName(space, resType));
 
-            return idx.query(space, clause, params, type, filters);
+                    if (type == null || !type.registered())
+                        throw new CacheException("Failed to find SQL table for type: " + resType);
+
+                    return idx.query(space, clause, params, type, filters);
+                }
+            });
         }
         finally {
             busyLock.leaveBusy();
@@ -539,19 +545,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public Iterable<List<?>> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
+    public Iterable<List<?>> queryTwoStep(String space, final GridCacheTwoStepQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            GridCacheContext<Object, Object> cacheCtx = ctx.cache().internalCache(space).context();
-
-            return idx.queryTwoStep(
-                cacheCtx,
-                qry,
-                cacheCtx.keepPortable());
+            final GridCacheContext<Object, Object> cctx = ctx.cache().internalCache(space).context();
+
+            return executeQuery(cctx, new IgniteOutClosureX<Iterable<List<?>>>() {
+                @Override public Iterable<List<?>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(
+                        cctx,
+                        qry,
+                        cctx.keepPortable());
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -563,14 +576,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryTwoStep(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryTwoStep(cctx, qry);
+            return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(cctx, qry);
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -582,14 +602,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry) {
+    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(final GridCacheContext<?,?> cctx, final SqlQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryTwoStep(cctx, qry);
+            return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
+                @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(cctx, qry);
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -698,7 +725,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * Closeable iterator.
      */
-    private static interface ClIter<X> extends AutoCloseable, Iterator<X> {
+    private interface ClIter<X> extends AutoCloseable, Iterator<X> {
         // No-op.
     }
 
@@ -810,24 +837,30 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(String space, String clause, String resType,
-        IndexingQueryFilter filters) throws IgniteCheckedException {
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String space, final String clause,
+        final String resType, final IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
 
-            if (type == null || !type.registered())
-                throw new CacheException("Failed to find SQL table for type: " + resType);
+            return executeQuery(cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
+                @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
+                    TypeDescriptor type = typesByName.get(new TypeName(space, resType));
 
-            return idx.queryText(
-                space,
-                clause,
-                type,
-                filters);
+                    if (type == null || !type.registered())
+                        throw new CacheException("Failed to find SQL table for type: " + resType);
+
+                    return idx.queryText(
+                        space,
+                        clause,
+                        type,
+                        filters);
+                }
+            });
         }
         finally {
             busyLock.leaveBusy();
@@ -842,15 +875,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Field rows.
      * @throws IgniteCheckedException If failed.
      */
-    public GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params,
-        IndexingQueryFilter filters) throws IgniteCheckedException {
+    public GridQueryFieldsResult queryFields(@Nullable final String space, final String clause,
+        final Collection<Object> params, final IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryFields(space, clause, params, filters);
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
+
+            return executeQuery(cctx, new IgniteOutClosureX<GridQueryFieldsResult>() {
+                @Override public GridQueryFieldsResult applyx() throws IgniteCheckedException {
+                    return idx.queryFields(space, clause, params, filters);
+                }
+            });
         }
         finally {
             busyLock.leaveBusy();
@@ -1718,6 +1757,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
         @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
             assert field != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
new file mode 100644
index 0000000..63912bf
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
@@ -0,0 +1,205 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Tests for cache query metrics.
+ */
+public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 2;
+
+    /** Cache mode. */
+    protected CacheMode cacheMode;
+
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration<String, Integer> cacheCfg1 = defaultCacheConfiguration();
+
+        cacheCfg1.setName("A");
+        cacheCfg1.setCacheMode(cacheMode);
+        cacheCfg1.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg1.setIndexedTypes(String.class, Integer.class);
+
+        CacheConfiguration<String, Integer> cacheCfg2 = defaultCacheConfiguration();
+
+        cacheCfg2.setName("B");
+        cacheCfg2.setCacheMode(cacheMode);
+        cacheCfg2.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg2.setIndexedTypes(String.class, Integer.class);
+
+        cfg.setCacheConfiguration(cacheCfg1, cacheCfg2);
+
+        return cfg;
+    }
+
+    /**
+     * Test metrics for SQL queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlFieldsQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer");
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
+     * Test metrics for Scan queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testScanQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        ScanQuery<String, Integer> qry = new ScanQuery<>();
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
+     * Test metrics for SQL cross cache queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlCrossCacheQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer");
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
new file mode 100644
index 0000000..666acfb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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 static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for partitioned cache query metrics.
+ */
+public class CachePartitionedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        cacheMode = PARTITIONED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
new file mode 100644
index 0000000..2be6dd3
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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 static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for replicated cache query metrics.
+ */
+public class CacheReplicatedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        cacheMode = REPLICATED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
deleted file mode 100644
index 24011b4..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * 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 org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * Tests for cache query metrics.
- */
-public class GridCacheQueryMetricsSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int GRID_CNT = 2;
-
-    /** */
-    private static final CacheMode CACHE_MODE = REPLICATED;
-
-    /** */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(disco);
-
-        CacheConfiguration<String, Integer> cacheCfg1 = defaultCacheConfiguration();
-
-        cacheCfg1.setName("A");
-        cacheCfg1.setCacheMode(CACHE_MODE);
-        cacheCfg1.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg1.setIndexedTypes(String.class, Integer.class);
-
-        CacheConfiguration<String, Integer> cacheCfg2 = defaultCacheConfiguration();
-
-        cacheCfg2.setName("B");
-        cacheCfg2.setCacheMode(CACHE_MODE);
-        cacheCfg2.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg2.setIndexedTypes(String.class, Integer.class);
-
-        cfg.setCacheConfiguration(cacheCfg1, cacheCfg2);
-
-        return cfg;
-    }
-
-    /**
-     * Test metrics for SQL queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testSqlFieldsQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
-
-        // Execute query.
-        SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer");
-
-        cache.query(qry).getAll();
-
-        QueryMetrics m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(1, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-
-        // Execute again with the same parameters.
-        cache.query(qry).getAll();
-
-        m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(2, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-    }
-
-    /**
-     * Test metrics for Scan queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testScanQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
-
-        // Execute query.
-        ScanQuery<String, Integer> qry = new ScanQuery<>();
-
-        cache.query(qry).getAll();
-
-        QueryMetrics m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(1, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-
-        // Execute again with the same parameters.
-        cache.query(qry).getAll();
-
-        m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(2, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-    }
-
-    /**
-     * Test metrics for SQL cross cache queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testSqlCrossCacheQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
-
-        // Execute query.
-        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer");
-
-        cache.query(qry).getAll();
-
-        QueryMetrics m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(1, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-
-        // Execute again with the same parameters.
-        cache.query(qry).getAll();
-
-        m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(2, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
index 9da7cf8..a43f573 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
@@ -198,8 +198,8 @@ public class BaseH2CompareQueryTest extends AbstractH2CompareQueryTest {
     /**
      * @throws Exception
      */
-    // TODO: IGNITE-705
     public void testAllExamples() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-705");
 //        compareQueryRes0("select ? limit ? offset ?");
 
 //        compareQueryRes0("select cool1()");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1eb3bd24/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 0815dc6..e37e696 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -112,7 +112,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(BaseH2CompareQueryTest.class);
         suite.addTestSuite(H2CompareBigQueryTest.class);
 
-        suite.addTestSuite(GridCacheQueryMetricsSelfTest.class);
+        // Cache query metrics.
+        suite.addTestSuite(CachePartitionedQueryMetricsSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsSelfTest.class);
 
         //Unmarshallig query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);