You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by al...@apache.org on 2019/12/05 14:21:58 UTC

[ignite] branch ignite-2.8 updated: IGNITE-12188 Fixed CacheGroupMetrics.IndexBuildCountPartitionsLeft metric - Fixes #7078.

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

alexpl pushed a commit to branch ignite-2.8
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-2.8 by this push:
     new c60cdd7  IGNITE-12188 Fixed CacheGroupMetrics.IndexBuildCountPartitionsLeft metric - Fixes #7078.
c60cdd7 is described below

commit c60cdd7120630b5ea93129f1705b35fea2639a80
Author: NSAmelchev <ns...@gmail.com>
AuthorDate: Thu Dec 5 16:56:55 2019 +0300

    IGNITE-12188 Fixed CacheGroupMetrics.IndexBuildCountPartitionsLeft metric - Fixes #7078.
    
    Signed-off-by: Aleksey Plekhanov <pl...@gmail.com>
    (cherry picked from commit 299e3c9e4441763faba8f55b928101f9991c4773)
---
 .../processors/cache/CacheGroupMetricsImpl.java    |   6 +-
 .../processors/query/GridQueryProcessor.java       |   2 +-
 .../query/schema/SchemaIndexCacheVisitorImpl.java  |  40 ++++-
 .../processors/query/h2/IgniteH2Indexing.java      |   2 +-
 .../CacheGroupMetricsWithIndexBuildFailTest.java   | 187 +++++++++++++++++++++
 .../cache/CacheGroupMetricsWithIndexTest.java      |  50 ++++--
 .../cache/index/AbstractIndexingCommonTest.java    |   7 +-
 .../IgniteCacheWithIndexingTestSuite.java          |   2 +
 8 files changed, 266 insertions(+), 30 deletions(-)

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
index e82e451..b1394bc 100644
--- 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
@@ -176,9 +176,9 @@ public class CacheGroupMetricsImpl {
         return idxBuildCntPartitionsLeft.value();
     }
 
-    /** Set number of partitions need processed for finished indexes create or rebuilding. */
-    public void setIndexBuildCountPartitionsLeft(long idxBuildCntPartitionsLeft) {
-        this.idxBuildCntPartitionsLeft.value(idxBuildCntPartitionsLeft);
+    /** Add number of partitions need processed for finished indexes create or rebuilding. */
+    public void addIndexBuildCountPartitionsLeft(long idxBuildCntPartitionsLeft) {
+        this.idxBuildCntPartitionsLeft.add(idxBuildCntPartitionsLeft);
     }
 
     /**
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 a2a51bd..5eadedf 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
@@ -1581,7 +1581,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     SchemaIndexCacheFilter filter = new TableCacheFilter(cctx, op0.tableName());
 
-                    cctx.group().metrics().setIndexBuildCountPartitionsLeft(cctx.topology().localPartitions().size());
+                    cctx.group().metrics().addIndexBuildCountPartitionsLeft(cctx.topology().localPartitions().size());
 
                     visitor = new SchemaIndexCacheVisitorImpl(cctx, filter, cancelTok, op0.parallel());
                 }
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 d45f1a1..5a54b81 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
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.query.schema;
 
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -69,6 +71,12 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
     /** Whether to stop the process. */
     private volatile boolean stop;
 
+    /** Count of partitions to be processed. */
+    private final AtomicInteger partsCnt = new AtomicInteger();
+
+    /** Logger. */
+    protected IgniteLogger log;
+
     static {
         int parallelism = IgniteSystemProperties.getInteger(INDEX_REBUILDING_PARALLELISM, 0);
 
@@ -110,6 +118,8 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
             cctx = ((GridNearCacheAdapter)cctx.cache()).dht().context();
 
         this.cctx = cctx;
+
+        log = cctx.kernalContext().log(getClass());
     }
 
     /** {@inheritDoc} */
@@ -121,6 +131,8 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         if (parts.isEmpty())
             return;
 
+        partsCnt.set(parts.size());
+
         GridCompoundFuture<Void, Void> fut = null;
 
         if (parallelism > 1) {
@@ -132,7 +144,18 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
             fut.markInitialized();
         }
 
-        processPartitions(parts, clo, 0);
+        try {
+            processPartitions(parts, clo, 0);
+        }
+        catch (Throwable e) {
+            U.error(log, "Error during parallel index create/rebuild.", e);
+
+            stop = true;
+
+            resetPartitionsCount();
+
+            throw e;
+        }
 
         if (fut != null)
             fut.get();
@@ -234,7 +257,8 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         finally {
             part.release();
 
-            cctx.group().metrics().decrementIndexBuildCountPartitionsLeft();
+            if (partsCnt.getAndUpdate(v -> v > 0 ? v - 1 : 0) > 0)
+                cctx.group().metrics().decrementIndexBuildCountPartitionsLeft();
         }
     }
 
@@ -280,6 +304,16 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
             throw new IgniteCheckedException("Index creation was cancelled.");
     }
 
+    /**
+     * Resets value of partitions count to be processed and update metrics.
+     */
+    private void resetPartitionsCount() {
+        int cnt = partsCnt.getAndSet(0);
+
+        if (cnt > 0)
+            cctx.group().metrics().addIndexBuildCountPartitionsLeft(-cnt);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(SchemaIndexCacheVisitorImpl.class, this);
@@ -335,7 +369,7 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
 
                 stop = true;
 
-                cctx.group().metrics().setIndexBuildCountPartitionsLeft(0);
+                resetPartitionsCount();
             }
             finally {
                 fut.onDone(err);
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 63b39e6..687ed76 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
@@ -1963,7 +1963,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         markIndexRebuild(cctx.name(), true);
 
         if (cctx.group().metrics() != null)
-            cctx.group().metrics().setIndexBuildCountPartitionsLeft(cctx.topology().localPartitions().size());
+            cctx.group().metrics().addIndexBuildCountPartitionsLeft(cctx.topology().localPartitions().size());
 
         GridWorker worker = new GridWorker(ctx.igniteInstanceName(), "index-rebuild-worker-" + cctx.name(), log) {
             @Override protected void body() {
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexBuildFailTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexBuildFailTest.java
new file mode 100644
index 0000000..4098616
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexBuildFailTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.nio.file.Path;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingSpi;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+import static org.apache.ignite.IgniteSystemProperties.INDEX_REBUILDING_PARALLELISM;
+import static org.apache.ignite.internal.processors.cache.CacheGroupMetricsImpl.CACHE_GROUP_METRICS_PREFIX;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+
+/**
+ * Tests cache group metrics for index build fail case.
+ */
+public class CacheGroupMetricsWithIndexBuildFailTest extends AbstractIndexingCommonTest {
+    /** Group name. */
+    private static final String GROUP_NAME = "TEST_GROUP";
+
+    /** {@code True} if fail index build. */
+    private final AtomicBoolean failIndexRebuild = new AtomicBoolean();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration().setPersistenceEnabled(true).setMaxSize(10 * 1024 * 1024)
+            )
+        );
+
+        cfg.setIndexingSpi(new TestIndexingSpi());
+
+        return cfg;
+    }
+
+    /**
+     * @return Default cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(String cacheName) {
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>(cacheName);
+
+        ccfg.setGroupName(GROUP_NAME);
+        ccfg.setIndexedTypes(Integer.class, Integer.class);
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        GridQueryProcessor.idxCls = null;
+    }
+
+    /** */
+    @Test
+    @WithSystemProperty(key = INDEX_REBUILDING_PARALLELISM, value = "4")
+    public void testIndexRebuildCountPartitionsLeft() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+
+        ignite0.cluster().active(true);
+
+        String cacheName1 = "cache1";
+        String cacheName2 = "cache2";
+
+        IgniteCache<Integer, Integer> cache1 = ignite0.getOrCreateCache(cacheConfiguration(cacheName1));
+        IgniteCache<Integer, Integer> cache2 = ignite0.getOrCreateCache(cacheConfiguration(cacheName2));
+
+        cache1.put(1, 1);
+        cache2.put(1, 1);
+
+        int parts1 = ignite0.cachex(cacheName1).configuration().getAffinity().partitions();
+        int parts2 = ignite0.cachex(cacheName2).configuration().getAffinity().partitions();
+
+        List<Path> idxPaths = getIndexBinPaths(cacheName1);
+
+        stopAllGrids();
+
+        idxPaths.forEach(idxPath -> assertTrue(U.delete(idxPath)));
+
+        GridQueryProcessor.idxCls = BlockingIndexing.class;
+
+        IgniteEx ignite = startGrid(0);
+
+        ignite.cluster().active(true);
+
+        MetricRegistry grpMreg = ignite.context().metric().registry(metricName(CACHE_GROUP_METRICS_PREFIX, GROUP_NAME));
+
+        LongMetric indexBuildCountPartitionsLeft = grpMreg.findMetric("IndexBuildCountPartitionsLeft");
+
+        assertEquals(parts1 + parts2, indexBuildCountPartitionsLeft.value());
+
+        failIndexRebuild.set(true);
+
+        ((AbstractIndexingCommonTest.BlockingIndexing)ignite.context().query().getIndexing()).stopBlock(cacheName1);
+
+        GridTestUtils.assertThrows(log, () -> ignite.cache(cacheName1).indexReadyFuture().get(30_000),
+            IgniteSpiException.class, "Test exception.");
+
+        assertEquals(parts2, indexBuildCountPartitionsLeft.value());
+
+        failIndexRebuild.set(false);
+
+        ((AbstractIndexingCommonTest.BlockingIndexing)ignite.context().query().getIndexing()).stopBlock(cacheName2);
+
+        ignite.cache(cacheName2).indexReadyFuture().get(30_000);
+
+        assertEquals(0, indexBuildCountPartitionsLeft.value());
+    }
+
+    /** */
+    private class TestIndexingSpi extends IgniteSpiAdapter implements IndexingSpi {
+        /** {@inheritDoc} */
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
+            @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
+            throws IgniteSpiException {
+            if (failIndexRebuild.get())
+                throw new IgniteSpiException("Test exception.");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void spiStop() throws IgniteSpiException {
+            // No-op.
+        }
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexTest.java
index 627091b..57f69c7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsWithIndexTest.java
@@ -35,7 +35,10 @@ 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.IgniteEx;
+import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest.BlockingIndexing;
 import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.metric.LongMetric;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -56,6 +59,9 @@ public class CacheGroupMetricsWithIndexTest extends CacheGroupMetricsTest {
     private static final String CACHE_NAME = "cache1";
 
     /** */
+    private static final String GROUP_NAME_2 = "group2";
+
+    /** */
     private static final String OBJECT_NAME = "MyObject";
 
     /** */
@@ -81,8 +87,7 @@ public class CacheGroupMetricsWithIndexTest extends CacheGroupMetricsTest {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
         for (CacheConfiguration cacheCfg : cfg.getCacheConfiguration()) {
-            if (GROUP_NAME.equals(cacheCfg.getGroupName()) && CACHE_NAME.equals(cacheCfg.getName())) {
-
+            if (GROUP_NAME.equals(cacheCfg.getGroupName()) || GROUP_NAME_2.equals(cacheCfg.getGroupName())) {
                 QueryEntity qryEntity = new QueryEntity(Long.class.getCanonicalName(), OBJECT_NAME);
 
                 qryEntity.setKeyFieldName(KEY_NAME);
@@ -124,6 +129,8 @@ public class CacheGroupMetricsWithIndexTest extends CacheGroupMetricsTest {
         stopAllGrids();
 
         cleanPersistenceDir();
+
+        GridQueryProcessor.idxCls = null;
     }
 
     /**
@@ -133,22 +140,23 @@ public class CacheGroupMetricsWithIndexTest extends CacheGroupMetricsTest {
     public void testIndexRebuildCountPartitionsLeft() throws Exception {
         pds = true;
 
-        Ignite ignite = startGrid(0);
+        GridQueryProcessor.idxCls = BlockingIndexing.class;
+
+        IgniteEx ignite = startGrid(0);
 
         ignite.cluster().active(true);
 
-        IgniteCache<Object, Object> cache1 = ignite.cache(CACHE_NAME);
+        String cacheName2 = "cache2";
+        String cacheName3 = "cache3";
 
-        for (int i = 0; i < 100_000; i++) {
-            Long id = (long)i;
+        IgniteCache<Long, Object> cache2 = ignite.cache(cacheName2);
+        IgniteCache<Long, Object> cache3 = ignite.cache(cacheName3);
 
-            BinaryObjectBuilder o = ignite.binary().builder(OBJECT_NAME)
-                .setField(KEY_NAME, id)
-                .setField(COLUMN1_NAME, i / 2)
-                .setField(COLUMN2_NAME, "str" + Integer.toHexString(i));
+        cache2.put(1L, 1L);
+        cache3.put(1L, 1L);
 
-            cache1.put(id, o.build());
-        }
+        int parts2 = ignite.cachex(cacheName2).configuration().getAffinity().partitions();
+        int parts3 = ignite.cachex(cacheName3).configuration().getAffinity().partitions();
 
         ignite.cluster().active(false);
 
@@ -163,15 +171,23 @@ public class CacheGroupMetricsWithIndexTest extends CacheGroupMetricsTest {
 
         ignite.cluster().active(true);
 
-        MetricRegistry grpMreg = cacheGroupMetrics(0, GROUP_NAME).get2();
+        MetricRegistry grpMreg = cacheGroupMetrics(0, GROUP_NAME_2).get2();
 
         LongMetric indexBuildCountPartitionsLeft = grpMreg.findMetric("IndexBuildCountPartitionsLeft");
 
-        Assert.assertTrue("Timeout wait start rebuild index",
-            waitForCondition(() -> indexBuildCountPartitionsLeft.value() > 0, 30_000));
+        assertEquals(parts2 + parts3, indexBuildCountPartitionsLeft.value());
 
-        Assert.assertTrue("Timeout wait finished rebuild index",
-            GridTestUtils.waitForCondition(() -> indexBuildCountPartitionsLeft.value() == 0, 30_000));
+        ((BlockingIndexing)ignite.context().query().getIndexing()).stopBlock(cacheName2);
+
+        ignite.cache(cacheName2).indexReadyFuture().get(30_000);
+
+        assertEquals(parts3, indexBuildCountPartitionsLeft.value());
+
+        ((BlockingIndexing)ignite.context().query().getIndexing()).stopBlock(cacheName3);
+
+        ignite.cache(cacheName3).indexReadyFuture().get(30_000);
+
+        assertEquals(0, indexBuildCountPartitionsLeft.value());
     }
 
     /**
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java
index 30fefc5..99694dd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java
@@ -103,7 +103,7 @@ public class AbstractIndexingCommonTest extends GridCommonAbstractTest {
      * <p>
      * Blocks the indexes rebuilding until unblocked via {@link #stopBlock(String)}.
      */
-    protected static class BlockingIndexing extends IgniteH2Indexing {
+    public static class BlockingIndexing extends IgniteH2Indexing {
         /** */
         private final ConcurrentHashMap<String, CountDownLatch> latches = new ConcurrentHashMap<>();
 
@@ -125,10 +125,7 @@ public class AbstractIndexingCommonTest extends GridCommonAbstractTest {
          * @param cacheName Cache name.
          */
         public void stopBlock(String cacheName) {
-            CountDownLatch latch = latches.get(cacheName);
-
-            if (latch == null)
-                throw new IgniteException("Cache wasn't start index rebuild yet. [cacheName=" + cacheName + ']');
+            CountDownLatch latch = latches.computeIfAbsent(cacheName, l -> new CountDownLatch(1));
 
             latch.countDown();
         }
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 003ffed..ff38fc0 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.CacheGroupMetricsWithIndexBuildFailTest;
 import org.apache.ignite.internal.processors.cache.CacheGroupMetricsWithIndexTest;
 import org.apache.ignite.internal.processors.cache.CacheIndexStreamerTest;
 import org.apache.ignite.internal.processors.cache.CacheOperationsWithExpirationTest;
@@ -97,6 +98,7 @@ import org.junit.runners.Suite;
     ClusterReadOnlyModeSqlTest.class,
 
     CacheGroupMetricsWithIndexTest.class,
+    CacheGroupMetricsWithIndexBuildFailTest.class,
 
     RebuildIndexLogMessageTest.class,