You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2019/06/04 11:42:24 UTC

[ignite] branch master updated: IGNITE-11512 Add counter left partition for index rebuild in CacheGroupMetricsMXBean

This is an automated email from the ASF dual-hosted git repository.

irakov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 8ccbd2f  IGNITE-11512 Add counter left partition for index rebuild in CacheGroupMetricsMXBean
8ccbd2f is described below

commit 8ccbd2fbbf4e2a421d3d5cf5ebbbe4272306fd34
Author: a-polyakov <po...@gmail.com>
AuthorDate: Tue Jun 4 14:41:10 2019 +0300

    IGNITE-11512 Add counter left partition for index rebuild in CacheGroupMetricsMXBean
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../java/org/apache/ignite/cache/CacheMetrics.java |   2 +
 .../processors/cache/CacheGroupContext.java        |  12 +
 .../processors/cache/CacheGroupMetrics.java        |  30 +++
 .../processors/cache/CacheGroupMetricsImpl.java    |  52 +++++
 .../cache/CacheGroupMetricsMXBeanImpl.java         |   5 +
 .../processors/query/GridQueryProcessor.java       |   3 +
 .../query/schema/SchemaIndexCacheVisitorImpl.java  |   6 +
 .../ignite/mxbean/CacheGroupMetricsMXBean.java     |   7 +-
 .../cache/CacheGroupMetricsMBeanTest.java          |  31 ++-
 .../processors/query/h2/IgniteH2Indexing.java      |   3 +
 .../cache/CacheGroupMetricsMBeanWithIndexTest.java | 244 +++++++++++++++++++++
 .../IgniteCacheWithIndexingTestSuite.java          |   5 +-
 12 files changed, 382 insertions(+), 18 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
index aa3f5b7..759af8f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -315,6 +315,7 @@ public interface CacheMetrics {
 
     /**
      * Gets number of non-{@code null} values in the cache.
+     * Note this method will always return {@code 0}
      *
      * @return Number of non-{@code null} values in the cache.
      * @deprecated Can overflow. Use {@link CacheMetrics#getCacheSize()} instead.
@@ -331,6 +332,7 @@ public interface CacheMetrics {
 
     /**
      * Gets number of keys in the cache, possibly with {@code null} values.
+     * Note this method will always return {@code 0}
      *
      * @return Number of keys in the cache.
      * @deprecated Can overflow. Use {@link CacheMetrics#getCacheSize()} instead.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index 60b8196..0be1dff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -186,6 +186,9 @@ public class CacheGroupContext {
     /** */
     private volatile boolean hasAtomicCaches;
 
+    /** Store cache group metrics. */
+    private final CacheGroupMetricsImpl metrics;
+
     /**
      * @param ctx Context.
      * @param grpId Group ID.
@@ -247,6 +250,8 @@ public class CacheGroupContext {
 
         log = ctx.kernalContext().log(getClass());
 
+        metrics = new CacheGroupMetricsImpl();
+
         mxBean = new CacheGroupMetricsMXBeanImpl(this);
 
         if (systemCache()) {
@@ -1290,4 +1295,11 @@ public class CacheGroupContext {
     public boolean hasAtomicCaches() {
         return hasAtomicCaches;
     }
+
+    /**
+     * @return Metrics.
+     */
+    public CacheGroupMetricsImpl metrics0() {
+        return metrics;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetrics.java
new file mode 100644
index 0000000..408738e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetrics.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * Cache group metrics
+ */
+public interface CacheGroupMetrics {
+    /**
+     * @return Number of partitions need processed for finished indexes create or rebuilding.
+     * It is calculated as the number of local partition minus the processed.
+     * A value of 0 indicates that the index is built.
+     */
+    public long getIndexBuildCountPartitionsLeft();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsImpl.java
new file mode 100644
index 0000000..eab6858
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsImpl.java
@@ -0,0 +1,52 @@
+/*
+ * 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.concurrent.atomic.AtomicLong;
+
+/**
+ * Cache group metrics
+ */
+public class CacheGroupMetricsImpl implements CacheGroupMetrics {
+    /** Number of partitions need processed for finished indexes create or rebuilding. */
+    private final AtomicLong idxBuildCntPartitionsLeft;
+
+    /** */
+    public CacheGroupMetricsImpl() {
+        idxBuildCntPartitionsLeft = new AtomicLong();
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public long getIndexBuildCountPartitionsLeft() {
+        return idxBuildCntPartitionsLeft.get();
+    }
+
+    /** Set number of partitions need processed for finished indexes create or rebuilding. */
+    public void setIndexBuildCountPartitionsLeft(long idxBuildCntPartitionsLeft) {
+        this.idxBuildCntPartitionsLeft.set(idxBuildCntPartitionsLeft);
+    }
+
+    /**
+     * Commit the complete index building for partition.
+     * @return Decrement number of partitions need processed for finished indexes create or rebuilding.
+     */
+    public long decrementIndexBuildCountPartitionsLeft() {
+        return idxBuildCntPartitionsLeft.decrementAndGet();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
index 85689f7..589b223 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMXBeanImpl.java
@@ -379,6 +379,11 @@ public class CacheGroupMetricsMXBeanImpl implements CacheGroupMetricsMXBean {
         return database().forGroupPageStores(ctx, PageStore::getSparseSize);
     }
 
+    /** {@inheritDoc} */
+    @Override public long getIndexBuildCountPartitionsLeft() {
+        return ctx.metrics0().getIndexBuildCountPartitionsLeft();
+    }
+
     /**
      * @return Database.
      */
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 10d9de5..24596e1 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
@@ -1468,6 +1468,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     SchemaIndexCacheFilter filter = new TableCacheFilter(cctx, op0.tableName());
 
+                    if (cctx.group().metrics0() != null)
+                        cctx.group().metrics0().setIndexBuildCountPartitionsLeft(cctx.topology().localPartitions().size());
+
                     visitor = new SchemaIndexCacheVisitorImpl(cctx, filter, cancelTok, op0.parallel());
                 }
                 else
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
index d7bc721..e2f660a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
@@ -220,6 +220,9 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         }
         finally {
             part.release();
+
+            if (cctx.group().metrics0() != null)
+                cctx.group().metrics0().decrementIndexBuildCountPartitionsLeft();
         }
     }
 
@@ -319,6 +322,9 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
                 U.error(log, "Error during parallel index create/rebuild.", e);
 
                 stop = true;
+
+                if (cctx.group().metrics0() != null)
+                    cctx.group().metrics0().setIndexBuildCountPartitionsLeft(0);
             }
             finally {
                 fut.onDone(err);
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
index cf8aa76..f2147b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheGroupMetricsMXBean.java
@@ -21,12 +21,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheGroupMetrics;
 
 /**
  * This interface defines JMX view on {@link CacheGroupContext}.
  */
 @MXBeanDescription("MBean that provides access to cache group descriptor.")
-public interface CacheGroupMetricsMXBean {
+public interface CacheGroupMetricsMXBean extends CacheGroupMetrics {
     /**
      * Gets cache group id.
      *
@@ -191,4 +192,8 @@ public interface CacheGroupMetricsMXBean {
      */
     @MXBeanDescription("Storage space allocated for group adjusted for possible sparsity, in bytes.")
     public long getSparseStorageSize();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Count of partitions need processed for finished indexes create or rebuilding.")
+    @Override public long getIndexBuildCountPartitionsLeft();
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanTest.java
index 6bb60b2..c448b4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanTest.java
@@ -59,7 +59,7 @@ import org.junit.Test;
  */
 public class CacheGroupMetricsMBeanTest extends GridCommonAbstractTest implements Serializable {
     /** */
-    private boolean pds = false;
+    protected boolean pds = false;
 
     /** */
     private static class RoundRobinVariableSizeAffinityFunction implements AffinityFunction {
@@ -108,21 +108,20 @@ public class CacheGroupMetricsMBeanTest extends GridCommonAbstractTest implement
     /**
      * Partition assignment for cache1 with given affinity function:
      *
-     *  P/N 0 1 2
-     *  ---------
-     *  0 | P
-     *  1 |   P B
-     *  2 | B B P
-     *  3 |   P
-     *  4 | B   P
-     *  5 | P B B
-     *  6 |     P
-     *  7 | P B
-     *  8 | B P B
-     *  9 | P
-     *
+     * P/N 0 1 2
+     * ---------
+     * 0 | P
+     * 1 |   P B
+     * 2 | B B P
+     * 3 |   P
+     * 4 | B   P
+     * 5 | P B B
+     * 6 |     P
+     * 7 | P B
+     * 8 | B P B
+     * 9 | P
      */
-    private static final int [][] assignmentMapArr =
+    private static final int[][] assignmentMapArr =
         new int[][] {{0}, {1, 2}, {2, 0, 1}, {1}, {2, 0}, {0, 1, 2}, {2}, {0, 1}, {1, 2, 0}, {0}};
 
     /** {@inheritDoc} */
@@ -188,7 +187,7 @@ public class CacheGroupMetricsMBeanTest extends GridCommonAbstractTest implement
      * @param cacheOrGrpName Cache group name.
      * @return MBean instance.
      */
-    private CacheGroupMetricsMXBean mxBean(int nodeIdx, String cacheOrGrpName) throws MalformedObjectNameException {
+    protected CacheGroupMetricsMXBean mxBean(int nodeIdx, String cacheOrGrpName) throws MalformedObjectNameException {
         ObjectName mbeanName = U.makeMBeanName(getTestIgniteInstanceName(nodeIdx), "Cache groups", cacheOrGrpName);
 
         MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer();
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 b4238c2..8efb078 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
@@ -1817,6 +1817,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         markIndexRebuild(cctx.name(), true);
 
+        if (cctx.group().metrics0() != null)
+            cctx.group().metrics0().setIndexBuildCountPartitionsLeft(cctx.topology().localPartitions().size());
+
         GridWorker worker = new GridWorker(ctx.igniteInstanceName(), "index-rebuild-worker-" + cctx.name(), log) {
             @Override protected void body() {
                 try {
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanWithIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanWithIndexTest.java
new file mode 100644
index 0000000..c1192bb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsMBeanWithIndexTest.java
@@ -0,0 +1,244 @@
+/*
+ * 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.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.FileFilterUtils;
+import org.apache.commons.io.filefilter.IOFileFilter;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.mxbean.CacheGroupMetricsMXBean;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/**
+ * Cache group JMX metrics test.
+ */
+public class CacheGroupMetricsMBeanWithIndexTest extends CacheGroupMetricsMBeanTest {
+    /**
+     *
+     */
+    private static final String GROUP_NAME = "group1";
+
+    /**
+     *
+     */
+    private static final String CACHE_NAME = "cache1";
+
+    /**
+     *
+     */
+    private static final String OBJECT_NAME = "MyObject";
+
+    /**
+     *
+     */
+    private static final String TABLE = "\"" + CACHE_NAME + "\"." + OBJECT_NAME;
+
+    /**
+     *
+     */
+    private static final String KEY_NAME = "id";
+
+    /**
+     *
+     */
+    private static final String COLUMN1_NAME = "col1";
+
+    /**
+     *
+     */
+    private static final String COLUMN2_NAME = "col2";
+
+    /**
+     *
+     */
+    private static final String COLUMN3_NAME = "col3";
+
+    /**
+     *
+     */
+    private static final String INDEX_NAME = "testindex001";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        for (CacheConfiguration cacheCfg : cfg.getCacheConfiguration()) {
+            if (GROUP_NAME.equals(cacheCfg.getGroupName()) && CACHE_NAME.equals(cacheCfg.getName())) {
+
+                QueryEntity qryEntity = new QueryEntity(Long.class.getCanonicalName(), OBJECT_NAME);
+
+                qryEntity.setKeyFieldName(KEY_NAME);
+
+                LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+                fields.put(KEY_NAME, Long.class.getCanonicalName());
+
+                fields.put(COLUMN1_NAME, Integer.class.getCanonicalName());
+
+                fields.put(COLUMN2_NAME, String.class.getCanonicalName());
+
+                qryEntity.setFields(fields);
+
+                ArrayList<QueryIndex> indexes = new ArrayList<>();
+
+                indexes.add(new QueryIndex(COLUMN1_NAME));
+
+                indexes.add(new QueryIndex(COLUMN2_NAME));
+
+                qryEntity.setIndexes(indexes);
+
+                cacheCfg.setQueryEntities(Collections.singletonList(qryEntity));
+            }
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * Test number of partitions need to finished indexes rebuilding.
+     */
+    @Test
+    public void testIndexRebuildCountPartitionsLeft() throws Exception {
+        pds = true;
+
+        Ignite ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        IgniteCache<Object, Object> cache1 = ignite.cache(CACHE_NAME);
+
+        for (int i = 0; i < 100_000; i++) {
+            Long id = (long)i;
+
+            BinaryObjectBuilder o = ignite.binary().builder(OBJECT_NAME)
+                .setField(KEY_NAME, id)
+                .setField(COLUMN1_NAME, i / 2)
+                .setField(COLUMN2_NAME, "str" + Integer.toHexString(i));
+
+            cache1.put(id, o.build());
+        }
+
+        ignite.cluster().active(false);
+
+        File dir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
+
+        IOFileFilter filter = FileFilterUtils.nameFileFilter("index.bin");
+
+        Collection<File> idxBinFiles = FileUtils.listFiles(dir, filter, TrueFileFilter.TRUE);
+
+        for (File indexBin : idxBinFiles)
+            U.delete(indexBin);
+
+        ignite.cluster().active(true);
+
+        CacheGroupMetricsMXBean mxBean0Grp1 = mxBean(0, GROUP_NAME);
+
+        Assert.assertTrue("Timeout wait start rebuild index",
+            GridTestUtils.waitForCondition(() -> mxBean0Grp1.getIndexBuildCountPartitionsLeft() > 0, 30_000)
+        );
+
+        Assert.assertTrue("Timeout wait finished rebuild index",
+            GridTestUtils.waitForCondition(() -> mxBean0Grp1.getIndexBuildCountPartitionsLeft() == 0, 30_000)
+        );
+    }
+
+    /**
+     * Test number of partitions need to finished create indexes.
+     */
+    @Test
+    public void testIndexCreateCountPartitionsLeft() throws Exception {
+        pds = true;
+
+        Ignite ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        IgniteCache<Object, Object> cache1 = ignite.cache(CACHE_NAME);
+
+        String addColSql = "ALTER TABLE " + TABLE + " ADD COLUMN " + COLUMN3_NAME + " BIGINT";
+
+        cache1.query(new SqlFieldsQuery(addColSql)).getAll();
+
+        for (int i = 0; i < 100_000; i++) {
+            Long id = (long)i;
+
+            BinaryObjectBuilder o = ignite.binary().builder(OBJECT_NAME)
+                .setField(KEY_NAME, id)
+                .setField(COLUMN1_NAME, i / 2)
+                .setField(COLUMN2_NAME, "str" + Integer.toHexString(i))
+                .setField(COLUMN3_NAME, id * 10);
+
+            cache1.put(id, o.build());
+        }
+
+        CacheGroupMetricsMXBean mxBean0Grp1 = mxBean(0, GROUP_NAME);
+
+        GridTestUtils.runAsync(() -> {
+            String createIdxSql = "CREATE INDEX " + INDEX_NAME + " ON " + TABLE + "(" + COLUMN3_NAME + ")";
+
+            cache1.query(new SqlFieldsQuery(createIdxSql)).getAll();
+
+            String selectIdxSql = "select * from information_schema.indexes where index_name='" + INDEX_NAME + "'";
+
+            List<List<?>> all = cache1.query(new SqlFieldsQuery(selectIdxSql)).getAll();
+
+            Assert.assertEquals("Index not found", 1, all.size());
+        });
+
+        Assert.assertTrue("Timeout wait start rebuild index",
+            GridTestUtils.waitForCondition(() -> mxBean0Grp1.getIndexBuildCountPartitionsLeft() > 0, 30_000)
+        );
+
+        Assert.assertTrue("Timeout wait finished rebuild index",
+            GridTestUtils.waitForCondition(() -> mxBean0Grp1.getIndexBuildCountPartitionsLeft() == 0, 30_000)
+        );
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index 973d499..a7dc9da 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.BinaryTypeMismatchLoggingTest
 import org.apache.ignite.internal.processors.cache.BinaryTypeRegistrationTest;
 import org.apache.ignite.internal.processors.cache.CacheBinaryKeyConcurrentQueryTest;
 import org.apache.ignite.internal.processors.cache.CacheConfigurationP2PTest;
+import org.apache.ignite.internal.processors.cache.CacheGroupMetricsMBeanWithIndexTest;
 import org.apache.ignite.internal.processors.cache.CacheIndexStreamerTest;
 import org.apache.ignite.internal.processors.cache.CacheOperationsWithExpirationTest;
 import org.apache.ignite.internal.processors.cache.CacheQueryAfterDynamicCacheStartFailureTest;
@@ -86,7 +87,9 @@ import org.junit.runners.Suite;
 
     BinaryTypeRegistrationTest.class,
 
-    ClusterReadOnlyModeSqlTest.class
+    ClusterReadOnlyModeSqlTest.class,
+
+    CacheGroupMetricsMBeanWithIndexTest.class
 })
 public class IgniteCacheWithIndexingTestSuite {
 }