You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2015/07/08 21:21:52 UTC

[27/50] incubator-ignite git commit: ignite-1031 Query metrics updated incorrectly

ignite-1031 Query metrics updated incorrectly


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

Branch: refs/heads/ignite-1026
Commit: c04cba9e82d3f9a7a5d851047de92322c54cc568
Parents: 9d0aa6f
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 7 19:28:38 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 7 19:28:38 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 11 files changed, 361 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 5b82c34..c2425f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -36,6 +36,7 @@ import org.jetbrains.annotations.*;
 
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.*;
 
 /**
@@ -413,12 +414,19 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     @SuppressWarnings("IfMayBeConditional")
     private <R> CacheQueryFuture<R> execute(@Nullable IgniteReducer<T, R> rmtReducer,
         @Nullable IgniteClosure<T, R> rmtTransform, @Nullable Object... args) {
-        Collection<ClusterNode> nodes = nodes();
+        Collection<ClusterNode> nodes;
+
+        try {
+            nodes = nodes();
+        }
+        catch (IgniteCheckedException e) {
+            return queryErrorFuture(cctx, e, log);
+        }
 
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (nodes.isEmpty())
-            return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), new ClusterGroupEmptyCheckedException());
+            return queryErrorFuture(cctx, new ClusterGroupEmptyCheckedException(), log);
 
         if (log.isDebugEnabled())
             log.debug("Executing query [query=" + this + ", nodes=" + nodes + ']');
@@ -429,7 +437,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                 cctx.deploy().registerClasses(args);
             }
             catch (IgniteCheckedException e) {
-                return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
+                return queryErrorFuture(cctx, e, log);
             }
         }
 
@@ -457,7 +465,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     /**
      * @return Nodes to execute on.
      */
-    private Collection<ClusterNode> nodes() {
+    private Collection<ClusterNode> nodes() throws IgniteCheckedException {
         CacheMode cacheMode = cctx.config().getCacheMode();
 
         switch (cacheMode) {
@@ -466,6 +474,10 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                     U.warn(log, "Ignoring query projection because it's executed over LOCAL cache " +
                         "(only local node will be queried): " + this);
 
+                if (type == SCAN && cctx.config().getCacheMode() == LOCAL &&
+                    partition() != null && partition() >= cctx.affinity().partitions())
+                    throw new IgniteCheckedException("Invalid partition number: " + partition());
+
                 return Collections.singletonList(cctx.localNode());
 
             case REPLICATED:
@@ -513,6 +525,21 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         });
     }
 
+    /**
+     * @param cctx Cache context.
+     * @param e Exception.
+     * @param log Logger.
+     */
+    private static <T> GridCacheQueryErrorFuture<T> queryErrorFuture(GridCacheContext<?, ?> cctx,
+        Exception e, IgniteLogger log) {
+
+        GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
+
+        GridQueryProcessor.onExecuted(cctx, metrics, null, e, 0, 0, log);
+
+        return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheQueryAdapter.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/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 e080c6d..6af0f40 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
@@ -1453,6 +1453,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             throw (IgniteCheckedException)err;
         }
+        catch (Exception e) {
+            err = e;
+
+            throw new IgniteCheckedException(e);
+        }
         finally {
             GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/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
index 63912bf..3eb96b6 100644
--- 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
@@ -33,7 +33,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
  */
 public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstractTest {
     /** Grid count. */
-    private static final int GRID_CNT = 2;
+    protected int gridCnt;
 
     /** Cache mode. */
     protected CacheMode cacheMode;
@@ -43,7 +43,7 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
+        startGridsMultiThreaded(gridCnt);
     }
 
     /** {@inheritDoc} */
@@ -122,6 +122,57 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
+     * Test metrics for failed SQL queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlFieldsQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from UNKNOWN");
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, 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.
@@ -163,6 +214,57 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
+     * Test metrics for failed Scan queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testScanQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        ScanQuery<String, Integer> qry = new ScanQuery<>(Integer.MAX_VALUE);
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, 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.
@@ -202,4 +304,55 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
         assertTrue(m.maximumTime() >= 0);
         assertTrue(m.minimumTime() >= 0);
     }
+
+    /**
+     * Test metrics for failed SQL cross cache queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlCrossCacheQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"G\".Integer");
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, 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/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
new file mode 100644
index 0000000..d28d3a0
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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 local cache query metrics.
+ */
+public class CacheLocalQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = LOCAL;
+
+        super.beforeTest();
+    }
+}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/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
deleted file mode 100644
index 2be6dd3..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,32 +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 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/c04cba9e/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 181ff0c..2d7d0ce 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
@@ -116,8 +116,11 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(H2CompareBigQueryTest.class);
 
         // Cache query metrics.
-        suite.addTestSuite(CachePartitionedQueryMetricsSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsSelfTest.class);
+        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
 
         //Unmarshallig query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);