You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ap...@apache.org on 2024/02/21 12:03:52 UTC

(ignite-3) branch main updated: IGNITE-21433 Ignore STOPPING indexes during Catalog validation (#3223)

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

apolovtsev pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new e07a4fea64 IGNITE-21433 Ignore STOPPING indexes during Catalog validation (#3223)
e07a4fea64 is described below

commit e07a4fea643c8ae695358c2fe8eedc0a3eb19ebe
Author: Alexander Polovtcev <al...@gmail.com>
AuthorDate: Wed Feb 21 14:03:45 2024 +0200

    IGNITE-21433 Ignore STOPPING indexes during Catalog validation (#3223)
---
 .../internal/catalog/CatalogManagerImpl.java       |   4 +-
 .../catalog/CatalogParamsValidationUtils.java      |   2 +-
 .../ignite/internal/catalog/CatalogService.java    |   9 +-
 .../commands/AlterTableDropColumnCommand.java      |  42 ++-
 .../internal/catalog/commands/CatalogUtils.java    |   2 +-
 .../catalog/commands/DropIndexCommand.java         |   2 -
 .../catalog/descriptors/CatalogIndexStatus.java    |   7 +
 .../descriptors/CatalogSchemaDescriptor.java       |  19 +-
 ...CatalogManagerDescriptorCausalityTokenTest.java |  16 +-
 .../catalog/CatalogManagerRecoveryTest.java        |   4 +-
 .../internal/catalog/CatalogManagerSelfTest.java   |  93 +++---
 .../commands/AbstractCommandValidationTest.java    |  58 +++-
 .../AlterTableDropColumnCommandValidationTest.java |  26 +-
 .../catalog/commands/CatalogUtilsTest.java         |  70 ++---
 .../CreateAbstractIndexCommandValidationTest.java  |  17 ++
 .../commands/DropZoneCommandValidationTest.java    |   5 +-
 .../commands/RenameTableCommandValidationTest.java |   5 +-
 .../ignite/client/handler/FakeCatalogService.java  |   2 +-
 .../ignite/internal/index/ItBuildIndexTest.java    | 195 +-----------
 .../index/ItDropIndexMultipleNodesTest.java        | 336 +++++++++++++++++++++
 .../internal/index/ItDropIndexOneNodeTest.java     | 112 +++++++
 .../ignite/internal/index/IndexChooserTest.java    |  56 ++--
 .../ignite/internal/index/IndexManagerTest.java    |   4 +-
 .../internal/index/TestIndexManagementUtils.java   |   4 -
 .../internal/ClusterPerClassIntegrationTest.java   |   2 +-
 .../storage/AbstractMvTableStorageTest.java        |   8 +-
 .../index/AbstractHashIndexStorageTest.java        |   2 +-
 .../index/AbstractSortedIndexStorageTest.java      |   2 +-
 .../internal/table/distributed/TableUtilsTest.java |  16 +-
 .../FullStateTransferIndexChooserTest.java         |  79 +++--
 .../ignite/internal/table/TableTestUtils.java      |  10 +-
 31 files changed, 782 insertions(+), 427 deletions(-)

diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
index 64e4313429..a123dc1267 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
@@ -213,8 +213,8 @@ public class CatalogManagerImpl extends AbstractEventProducer<CatalogEvent, Cata
     }
 
     @Override
-    public @Nullable CatalogIndexDescriptor index(String indexName, long timestamp) {
-        return catalogAt(timestamp).schema(DEFAULT_SCHEMA_NAME).index(indexName);
+    public @Nullable CatalogIndexDescriptor aliveIndex(String indexName, long timestamp) {
+        return catalogAt(timestamp).schema(DEFAULT_SCHEMA_NAME).aliveIndex(indexName);
     }
 
     @Override
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogParamsValidationUtils.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogParamsValidationUtils.java
index 4887ea3d5d..351d2ed3c9 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogParamsValidationUtils.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogParamsValidationUtils.java
@@ -104,7 +104,7 @@ public class CatalogParamsValidationUtils {
      * @throws CatalogValidationException If relation with specified name exists in given schema.
      */
     public static void ensureNoTableIndexOrSysViewExistsWithGivenName(CatalogSchemaDescriptor schema, String name) {
-        if (schema.index(name) != null) {
+        if (schema.aliveIndex(name) != null) {
             throw new IndexExistsValidationException(format("Index with name '{}.{}' already exists", schema.name(), name));
         }
 
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogService.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogService.java
index 00bcf66727..d2c167c863 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogService.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogService.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus;
 import org.apache.ignite.internal.catalog.descriptors.CatalogSchemaDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
@@ -60,7 +61,13 @@ public interface CatalogService extends EventProducer<CatalogEvent, CatalogEvent
 
     Collection<CatalogTableDescriptor> tables(int catalogVersion);
 
-    @Nullable CatalogIndexDescriptor index(String indexName, long timestamp);
+    /**
+     * Returns an <em>alive</em> index with the given name, that is an index that has not been dropped yet at a given point in time.
+     *
+     * <p>This effectively means that the index must be present in the Catalog and not in the {@link CatalogIndexStatus#STOPPING}
+     * state.
+     */
+    @Nullable CatalogIndexDescriptor aliveIndex(String indexName, long timestamp);
 
     @Nullable CatalogIndexDescriptor index(int indexId, long timestamp);
 
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommand.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommand.java
index cde22fc06f..728c6b9953 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommand.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommand.java
@@ -24,10 +24,10 @@ import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
 import static org.apache.ignite.internal.util.CollectionUtils.copyOrNull;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 
-import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.ignite.internal.catalog.Catalog;
 import org.apache.ignite.internal.catalog.CatalogCommand;
 import org.apache.ignite.internal.catalog.CatalogValidationException;
@@ -78,16 +78,12 @@ public class AlterTableDropColumnCommand extends AbstractTableCommand {
 
         CatalogTableDescriptor table = tableOrThrow(schema, tableName);
 
-        Set<String> indexedColumns = Arrays.stream(schema.indexes())
-                .filter(index -> index.tableId() == table.id())
-                .flatMap(index -> index instanceof CatalogHashIndexDescriptor
-                        ? ((CatalogHashIndexDescriptor) index).columns().stream()
-                        : ((CatalogSortedIndexDescriptor) index).columns().stream().map(CatalogIndexColumnDescriptor::name))
+        Set<String> indexedColumns = aliveIndexesForTable(catalog, table.id())
+                .flatMap(AlterTableDropColumnCommand::indexColumnNames)
                 .collect(Collectors.toSet());
 
         //To validate always in the same order let's sort given columns
-        List<String> sortedColumns = columns.stream().sorted().collect(Collectors.toUnmodifiableList());
-        for (String columnName : sortedColumns) {
+        columns.stream().sorted().forEach(columnName -> {
             if (table.column(columnName) == null) {
                 throw new CatalogValidationException(format(
                         "Column with name '{}' not found in table '{}.{}'", columnName, schemaName, tableName));
@@ -98,24 +94,38 @@ public class AlterTableDropColumnCommand extends AbstractTableCommand {
             }
 
             if (indexedColumns.contains(columnName)) {
-                List<String> indexesNames = Arrays.stream(schema.indexes())
-                        .filter(index -> index.tableId() == table.id())
-                        .filter(index -> index instanceof CatalogHashIndexDescriptor
-                                ? ((CatalogHashIndexDescriptor) index).columns().contains(columnName)
-                                : ((CatalogSortedIndexDescriptor) index).columns().stream().map(CatalogIndexColumnDescriptor::name)
-                                        .anyMatch(column -> column.equals(columnName))
-                ).map(CatalogIndexDescriptor::name).collect(Collectors.toList());
+                List<String> indexesNames = aliveIndexesForTable(catalog, table.id())
+                        .filter(index -> indexColumnNames(index).anyMatch(columnName::equals))
+                        .map(CatalogIndexDescriptor::name)
+                        .collect(Collectors.toList());
 
                 throw new CatalogValidationException(format(
                         "Deleting column '{}' used by index(es) {}, it is not allowed", columnName, indexesNames));
             }
-        }
+        });
 
         return List.of(
                 new DropColumnsEntry(table.id(), columns, schemaName)
         );
     }
 
+    private static Stream<CatalogIndexDescriptor> aliveIndexesForTable(Catalog catalog, int tableId) {
+        return catalog.indexes(tableId).stream().filter(index -> index.status().isAlive());
+    }
+
+    private static Stream<String> indexColumnNames(CatalogIndexDescriptor index) {
+        switch (index.indexType()) {
+            case HASH:
+                return ((CatalogHashIndexDescriptor) index).columns().stream();
+
+            case SORTED:
+                return ((CatalogSortedIndexDescriptor) index).columns().stream().map(CatalogIndexColumnDescriptor::name);
+
+            default:
+                throw new AssertionError(index.indexType().toString());
+        }
+    }
+
     private static void validate(Set<String> columns) {
         if (nullOrEmpty(columns)) {
             throw new CatalogValidationException("Columns not specified");
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
index 016843cd8c..0740a7070d 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
@@ -469,7 +469,7 @@ public class CatalogUtils {
      * @throws IndexNotFoundValidationException If index does not exist.
      */
     public static CatalogIndexDescriptor indexOrThrow(CatalogSchemaDescriptor schema, String name) throws IndexNotFoundValidationException {
-        CatalogIndexDescriptor index = schema.index(name);
+        CatalogIndexDescriptor index = schema.aliveIndex(name);
 
         if (index == null) {
             throw new IndexNotFoundValidationException(format("Index with name '{}.{}' not found", schema.name(), name));
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DropIndexCommand.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DropIndexCommand.java
index 9e9c52045e..ba20fb563b 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DropIndexCommand.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/DropIndexCommand.java
@@ -82,8 +82,6 @@ public class DropIndexCommand extends AbstractIndexCommand {
                 return List.of(new RemoveIndexEntry(index.id()));
             case AVAILABLE:
                 return List.of(new DropIndexEntry(index.id(), index.tableId()));
-            case STOPPING:
-                throw new CatalogValidationException("Dropping an already dropped index is not allowed");
             default:
                 throw new IllegalStateException("Unknown index status: " + index.status());
         }
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogIndexStatus.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogIndexStatus.java
index c19cfc0899..187886cccc 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogIndexStatus.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogIndexStatus.java
@@ -73,6 +73,13 @@ public enum CatalogIndexStatus {
         return id;
     }
 
+    /**
+     * Returns {@code true} if this index is alive, i.e. not in the {@link #STOPPING} state.
+     */
+    public boolean isAlive() {
+        return this != STOPPING;
+    }
+
     static {
         for (CatalogIndexStatus status : values()) {
             assert VALS[status.id] == null : "Found duplicate id " + status.id;
diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSchemaDescriptor.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSchemaDescriptor.java
index 744cd2a2b1..dc51e679c3 100644
--- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSchemaDescriptor.java
+++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSchemaDescriptor.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.catalog.descriptors;
 
+import static java.util.stream.Collectors.toUnmodifiableMap;
 import static org.apache.ignite.internal.catalog.storage.serialization.CatalogSerializationUtils.readArray;
 import static org.apache.ignite.internal.catalog.storage.serialization.CatalogSerializationUtils.writeArray;
 
@@ -25,7 +26,6 @@ import java.util.Arrays;
 import java.util.Map;
 import java.util.Objects;
 import java.util.function.Function;
-import java.util.stream.Collectors;
 import org.apache.ignite.internal.catalog.storage.serialization.CatalogObjectSerializer;
 import org.apache.ignite.internal.catalog.storage.serialization.CatalogSerializationUtils;
 import org.apache.ignite.internal.tostring.IgniteToStringExclude;
@@ -86,7 +86,13 @@ public class CatalogSchemaDescriptor extends CatalogObjectDescriptor {
         return tablesMap.get(name);
     }
 
-    public @Nullable CatalogIndexDescriptor index(String name) {
+    /**
+     * Returns an <em>alive</em> index with the given name, that is an index that has not been dropped yet.
+     *
+     * <p>This effectively means that the index must be present in the schema and not in the {@link CatalogIndexStatus#STOPPING}
+     * state.
+     */
+    public @Nullable CatalogIndexDescriptor aliveIndex(String name) {
         return indexesMap.get(name);
     }
 
@@ -95,10 +101,11 @@ public class CatalogSchemaDescriptor extends CatalogObjectDescriptor {
     }
 
     private void rebuildMaps() {
-        tablesMap = Arrays.stream(tables).collect(Collectors.toUnmodifiableMap(CatalogObjectDescriptor::name, Function.identity()));
-        indexesMap = Arrays.stream(indexes).collect(Collectors.toUnmodifiableMap(CatalogObjectDescriptor::name, Function.identity()));
-        systemViewsMap = Arrays.stream(systemViews)
-                .collect(Collectors.toUnmodifiableMap(CatalogObjectDescriptor::name, Function.identity()));
+        tablesMap = Arrays.stream(tables).collect(toUnmodifiableMap(CatalogObjectDescriptor::name, Function.identity()));
+        indexesMap = Arrays.stream(indexes)
+                .filter(index -> index.status().isAlive())
+                .collect(toUnmodifiableMap(CatalogObjectDescriptor::name, Function.identity()));
+        systemViewsMap = Arrays.stream(systemViews).collect(toUnmodifiableMap(CatalogObjectDescriptor::name, Function.identity()));
     }
 
     @Override
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerDescriptorCausalityTokenTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerDescriptorCausalityTokenTest.java
index 2289dbee6d..9ee31d4711 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerDescriptorCausalityTokenTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerDescriptorCausalityTokenTest.java
@@ -279,8 +279,8 @@ public class CatalogManagerDescriptorCausalityTokenTest extends BaseCatalogManag
         CatalogSchemaDescriptor schema = manager.schema(1);
 
         assertNotNull(schema);
-        assertNull(schema.index(INDEX_NAME));
-        assertNull(manager.index(INDEX_NAME, 123L));
+        assertNull(schema.aliveIndex(INDEX_NAME));
+        assertNull(manager.aliveIndex(INDEX_NAME, 123L));
 
         long schemaCausalityToken = schema.updateToken();
 
@@ -289,10 +289,10 @@ public class CatalogManagerDescriptorCausalityTokenTest extends BaseCatalogManag
         // Validate actual catalog.
         schema = manager.schema(2);
 
-        CatalogHashIndexDescriptor index = (CatalogHashIndexDescriptor) schema.index(INDEX_NAME);
+        CatalogHashIndexDescriptor index = (CatalogHashIndexDescriptor) schema.aliveIndex(INDEX_NAME);
 
         assertNotNull(schema);
-        assertSame(index, manager.index(INDEX_NAME, clock.nowLong()));
+        assertSame(index, manager.aliveIndex(INDEX_NAME, clock.nowLong()));
         assertSame(index, manager.index(index.id(), clock.nowLong()));
         assertTrue(schema.updateToken() > schemaCausalityToken);
 
@@ -319,8 +319,8 @@ public class CatalogManagerDescriptorCausalityTokenTest extends BaseCatalogManag
         CatalogSchemaDescriptor schema = manager.schema(1);
 
         assertNotNull(schema);
-        assertNull(schema.index(INDEX_NAME));
-        assertNull(manager.index(INDEX_NAME, 123L));
+        assertNull(schema.aliveIndex(INDEX_NAME));
+        assertNull(manager.aliveIndex(INDEX_NAME, 123L));
 
         long schemaCausalityToken = schema.updateToken();
         assertTrue(schemaCausalityToken > INITIAL_CAUSALITY_TOKEN);
@@ -328,10 +328,10 @@ public class CatalogManagerDescriptorCausalityTokenTest extends BaseCatalogManag
         // Validate actual catalog.
         schema = manager.schema(2);
 
-        CatalogSortedIndexDescriptor index = (CatalogSortedIndexDescriptor) schema.index(INDEX_NAME);
+        CatalogSortedIndexDescriptor index = (CatalogSortedIndexDescriptor) schema.aliveIndex(INDEX_NAME);
 
         assertNotNull(schema);
-        assertSame(index, manager.index(INDEX_NAME, clock.nowLong()));
+        assertSame(index, manager.aliveIndex(INDEX_NAME, clock.nowLong()));
         assertSame(index, manager.index(index.id(), clock.nowLong()));
         assertTrue(schema.updateToken() > schemaCausalityToken);
 
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerRecoveryTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerRecoveryTest.java
index 86be7eabf2..bbbf6917c9 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerRecoveryTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerRecoveryTest.java
@@ -183,7 +183,7 @@ public class CatalogManagerRecoveryTest extends BaseIgniteAbstractTest {
 
         int expCreationCatalogVersion = catalogManager.latestCatalogVersion();
 
-        int indexId = catalogManager.index(INDEX_NAME, clock.nowLong()).id();
+        int indexId = catalogManager.aliveIndex(INDEX_NAME, clock.nowLong()).id();
 
         assertThat(catalogManager.execute(startBuildingIndexCommand(indexId)), willCompleteSuccessfully());
         assertThat(catalogManager.execute(simpleTable(TABLE_NAME + 1)), willCompleteSuccessfully());
@@ -192,7 +192,7 @@ public class CatalogManagerRecoveryTest extends BaseIgniteAbstractTest {
 
         createAndStartComponents();
 
-        assertEquals(expCreationCatalogVersion, catalogManager.index(INDEX_NAME, clock.nowLong()).txWaitCatalogVersion());
+        assertEquals(expCreationCatalogVersion, catalogManager.aliveIndex(INDEX_NAME, clock.nowLong()).txWaitCatalogVersion());
     }
 
     private void createAndStartComponents() {
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
index 0d37c2ec30..91cc5947fa 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
@@ -258,12 +258,12 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
 
         assertNull(schema.table(TABLE_NAME));
         assertNull(manager.table(TABLE_NAME, 123L));
-        assertNull(manager.index(pkIndexName(TABLE_NAME), 123L));
+        assertNull(manager.aliveIndex(pkIndexName(TABLE_NAME), 123L));
 
         // Validate actual catalog
         schema = manager.schema(SCHEMA_NAME, 1);
         CatalogTableDescriptor table = schema.table(TABLE_NAME);
-        CatalogHashIndexDescriptor pkIndex = (CatalogHashIndexDescriptor) schema.index(pkIndexName(TABLE_NAME));
+        CatalogHashIndexDescriptor pkIndex = (CatalogHashIndexDescriptor) schema.aliveIndex(pkIndexName(TABLE_NAME));
 
         assertNotNull(schema);
         assertEquals(SCHEMA_NAME, schema.name());
@@ -272,7 +272,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertSame(table, manager.table(TABLE_NAME, clock.nowLong()));
         assertSame(table, manager.table(table.id(), clock.nowLong()));
 
-        assertSame(pkIndex, manager.index(pkIndexName(TABLE_NAME), clock.nowLong()));
+        assertSame(pkIndex, manager.aliveIndex(pkIndexName(TABLE_NAME), clock.nowLong()));
         assertSame(pkIndex, manager.index(pkIndex.id(), clock.nowLong()));
 
         // Validate newly created table
@@ -298,9 +298,9 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         // Validate actual catalog has both tables.
         schema = manager.schema(2);
         table = schema.table(TABLE_NAME);
-        pkIndex = (CatalogHashIndexDescriptor) schema.index(pkIndexName(TABLE_NAME));
+        pkIndex = (CatalogHashIndexDescriptor) schema.aliveIndex(pkIndexName(TABLE_NAME));
         CatalogTableDescriptor table2 = schema.table(TABLE_NAME_2);
-        CatalogHashIndexDescriptor pkIndex2 = (CatalogHashIndexDescriptor) schema.index(pkIndexName(TABLE_NAME_2));
+        CatalogHashIndexDescriptor pkIndex2 = (CatalogHashIndexDescriptor) schema.aliveIndex(pkIndexName(TABLE_NAME_2));
 
         assertNotNull(schema);
         assertEquals(SCHEMA_NAME, schema.name());
@@ -309,13 +309,13 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertSame(table, manager.table(TABLE_NAME, clock.nowLong()));
         assertSame(table, manager.table(table.id(), clock.nowLong()));
 
-        assertSame(pkIndex, manager.index(pkIndexName(TABLE_NAME), clock.nowLong()));
+        assertSame(pkIndex, manager.aliveIndex(pkIndexName(TABLE_NAME), clock.nowLong()));
         assertSame(pkIndex, manager.index(pkIndex.id(), clock.nowLong()));
 
         assertSame(table2, manager.table(TABLE_NAME_2, clock.nowLong()));
         assertSame(table2, manager.table(table2.id(), clock.nowLong()));
 
-        assertSame(pkIndex2, manager.index(pkIndexName(TABLE_NAME_2), clock.nowLong()));
+        assertSame(pkIndex2, manager.aliveIndex(pkIndexName(TABLE_NAME_2), clock.nowLong()));
         assertSame(pkIndex2, manager.index(pkIndex2.id(), clock.nowLong()));
 
         assertNotSame(table, table2);
@@ -344,8 +344,8 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         CatalogSchemaDescriptor schema = manager.schema(2);
         CatalogTableDescriptor table1 = schema.table(TABLE_NAME);
         CatalogTableDescriptor table2 = schema.table(TABLE_NAME_2);
-        CatalogIndexDescriptor pkIndex1 = schema.index(pkIndexName(TABLE_NAME));
-        CatalogIndexDescriptor pkIndex2 = schema.index(pkIndexName(TABLE_NAME_2));
+        CatalogIndexDescriptor pkIndex1 = schema.aliveIndex(pkIndexName(TABLE_NAME));
+        CatalogIndexDescriptor pkIndex2 = schema.aliveIndex(pkIndexName(TABLE_NAME_2));
 
         assertNotEquals(table1.id(), table2.id());
         assertNotEquals(pkIndex1.id(), pkIndex2.id());
@@ -357,13 +357,13 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertSame(table1, manager.table(TABLE_NAME, beforeDropTimestamp));
         assertSame(table1, manager.table(table1.id(), beforeDropTimestamp));
 
-        assertSame(pkIndex1, manager.index(pkIndexName(TABLE_NAME), beforeDropTimestamp));
+        assertSame(pkIndex1, manager.aliveIndex(pkIndexName(TABLE_NAME), beforeDropTimestamp));
         assertSame(pkIndex1, manager.index(pkIndex1.id(), beforeDropTimestamp));
 
         assertSame(table2, manager.table(TABLE_NAME_2, beforeDropTimestamp));
         assertSame(table2, manager.table(table2.id(), beforeDropTimestamp));
 
-        assertSame(pkIndex2, manager.index(pkIndexName(TABLE_NAME_2), beforeDropTimestamp));
+        assertSame(pkIndex2, manager.aliveIndex(pkIndexName(TABLE_NAME_2), beforeDropTimestamp));
         assertSame(pkIndex2, manager.index(pkIndex2.id(), beforeDropTimestamp));
 
         // Validate actual catalog
@@ -377,14 +377,14 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertNull(manager.table(TABLE_NAME, clock.nowLong()));
         assertNull(manager.table(table1.id(), clock.nowLong()));
 
-        assertThat(schema.index(pkIndexName(TABLE_NAME)), is(nullValue()));
-        assertThat(manager.index(pkIndexName(TABLE_NAME), clock.nowLong()), is(nullValue()));
+        assertThat(schema.aliveIndex(pkIndexName(TABLE_NAME)), is(nullValue()));
+        assertThat(manager.aliveIndex(pkIndexName(TABLE_NAME), clock.nowLong()), is(nullValue()));
         assertThat(manager.index(pkIndex1.id(), clock.nowLong()), is(nullValue()));
 
         assertSame(table2, manager.table(TABLE_NAME_2, clock.nowLong()));
         assertSame(table2, manager.table(table2.id(), clock.nowLong()));
 
-        assertSame(pkIndex2, manager.index(pkIndexName(TABLE_NAME_2), clock.nowLong()));
+        assertSame(pkIndex2, manager.aliveIndex(pkIndexName(TABLE_NAME_2), clock.nowLong()));
         assertSame(pkIndex2, manager.index(pkIndex2.id(), clock.nowLong()));
 
         // Validate schema wasn't changed.
@@ -979,7 +979,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         // Validate catalog version from the past.
         CatalogSchemaDescriptor schema = manager.schema(beforeDropVersion);
         CatalogTableDescriptor table = schema.table(TABLE_NAME);
-        CatalogIndexDescriptor index = schema.index(INDEX_NAME);
+        CatalogIndexDescriptor index = schema.aliveIndex(INDEX_NAME);
 
         assertNotNull(schema);
         assertEquals(SCHEMA_NAME, schema.name());
@@ -988,7 +988,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertSame(table, manager.table(TABLE_NAME, beforeDropTimestamp));
         assertSame(table, manager.table(table.id(), beforeDropTimestamp));
 
-        assertSame(index, manager.index(INDEX_NAME, beforeDropTimestamp));
+        assertSame(index, manager.aliveIndex(INDEX_NAME, beforeDropTimestamp));
         assertSame(index, manager.index(index.id(), beforeDropTimestamp));
 
         // Validate actual catalog
@@ -1002,8 +1002,8 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertNull(manager.table(TABLE_NAME, clock.nowLong()));
         assertNull(manager.table(table.id(), clock.nowLong()));
 
-        assertThat(schema.index(INDEX_NAME), is(nullValue()));
-        assertThat(manager.index(INDEX_NAME, clock.nowLong()), is(nullValue()));
+        assertThat(schema.aliveIndex(INDEX_NAME), is(nullValue()));
+        assertThat(manager.aliveIndex(INDEX_NAME, clock.nowLong()), is(nullValue()));
         assertThat(manager.index(index.id(), clock.nowLong()), is(nullValue()));
     }
 
@@ -1017,16 +1017,16 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         CatalogSchemaDescriptor schema = manager.schema(1);
 
         assertNotNull(schema);
-        assertNull(schema.index(INDEX_NAME));
-        assertNull(manager.index(INDEX_NAME, 123L));
+        assertNull(schema.aliveIndex(INDEX_NAME));
+        assertNull(manager.aliveIndex(INDEX_NAME, 123L));
 
         // Validate actual catalog
         schema = manager.schema(2);
 
-        CatalogHashIndexDescriptor index = (CatalogHashIndexDescriptor) schema.index(INDEX_NAME);
+        CatalogHashIndexDescriptor index = (CatalogHashIndexDescriptor) schema.aliveIndex(INDEX_NAME);
 
         assertNotNull(schema);
-        assertSame(index, manager.index(INDEX_NAME, clock.nowLong()));
+        assertSame(index, manager.aliveIndex(INDEX_NAME, clock.nowLong()));
         assertSame(index, manager.index(index.id(), clock.nowLong()));
 
         // Validate newly created hash index
@@ -1055,17 +1055,17 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         CatalogSchemaDescriptor schema = manager.schema(1);
 
         assertNotNull(schema);
-        assertNull(schema.index(INDEX_NAME));
-        assertNull(manager.index(INDEX_NAME, 123L));
+        assertNull(schema.aliveIndex(INDEX_NAME));
+        assertNull(manager.aliveIndex(INDEX_NAME, 123L));
         assertNull(manager.index(4, 123L));
 
         // Validate actual catalog
         schema = manager.schema(2);
 
-        CatalogSortedIndexDescriptor index = (CatalogSortedIndexDescriptor) schema.index(INDEX_NAME);
+        CatalogSortedIndexDescriptor index = (CatalogSortedIndexDescriptor) schema.aliveIndex(INDEX_NAME);
 
         assertNotNull(schema);
-        assertSame(index, manager.index(INDEX_NAME, clock.nowLong()));
+        assertSame(index, manager.aliveIndex(INDEX_NAME, clock.nowLong()));
         assertSame(index, manager.index(index.id(), clock.nowLong()));
 
         // Validate newly created sorted index
@@ -1228,10 +1228,12 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         assertThat(manager.execute(createIndexCmd), willCompleteSuccessfully());
         verify(eventListener).notify(any(CreateIndexEventParameters.class));
 
-        startBuildingIndex(indexId(INDEX_NAME));
+        int indexId = indexId(INDEX_NAME);
+
+        startBuildingIndex(indexId);
         verify(eventListener).notify(any(StartBuildingIndexEventParameters.class));
 
-        makeIndexAvailable(indexId(INDEX_NAME));
+        makeIndexAvailable(indexId);
         verify(eventListener).notify(any(MakeIndexAvailableEventParameters.class));
 
         verifyNoMoreInteractions(eventListener);
@@ -1242,7 +1244,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         verify(eventListener).notify(any(StoppingIndexEventParameters.class));
 
         // Remove index.
-        removeIndex(indexId(INDEX_NAME));
+        removeIndex(indexId);
         verify(eventListener).notify(any(RemoveIndexEventParameters.class));
 
         verifyNoMoreInteractions(eventListener);
@@ -1624,13 +1626,13 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
 
         assertThat(manager.execute(createHashIndexCommand(INDEX_NAME, List.of("VAL"))), willBe(nullValue()));
 
-        int indexId = manager.index(INDEX_NAME, clock.nowLong()).id();
+        int indexId = manager.aliveIndex(INDEX_NAME, clock.nowLong()).id();
 
         startBuildingIndex(indexId);
         makeIndexAvailable(indexId);
 
         int tableId = manager.table(TABLE_NAME, clock.nowLong()).id();
-        int pkIndexId = manager.index(pkIndexName(TABLE_NAME), clock.nowLong()).id();
+        int pkIndexId = manager.aliveIndex(pkIndexName(TABLE_NAME), clock.nowLong()).id();
 
         assertNotEquals(tableId, indexId);
 
@@ -1672,7 +1674,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         createSomeIndex(TABLE_NAME, INDEX_NAME);
 
         int catalogVersion = manager.latestCatalogVersion();
-        CatalogIndexDescriptor index1 = manager.index(INDEX_NAME, clock.nowLong());
+        CatalogIndexDescriptor index1 = manager.aliveIndex(INDEX_NAME, clock.nowLong());
         assertNotNull(index1);
 
         int indexId1 = index1.id();
@@ -1682,12 +1684,12 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         // Drop index.
         dropIndex(INDEX_NAME);
         removeIndex(indexId1);
-        assertNull(manager.index(INDEX_NAME, clock.nowLong()));
+        assertNull(manager.aliveIndex(INDEX_NAME, clock.nowLong()));
 
         // Re-create index with same name.
         createSomeSortedIndex(TABLE_NAME, INDEX_NAME);
 
-        CatalogIndexDescriptor index2 = manager.index(INDEX_NAME, clock.nowLong());
+        CatalogIndexDescriptor index2 = manager.aliveIndex(INDEX_NAME, clock.nowLong());
         assertNotNull(index2);
         assertThat(index2.indexType(), equalTo(CatalogIndexDescriptorType.SORTED));
 
@@ -1911,12 +1913,15 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
     void droppingAnAvailableIndexMovesItToStoppingState() {
         createSomeTable(TABLE_NAME);
         createSomeIndex(TABLE_NAME, INDEX_NAME);
-        startBuildingIndex(indexId(INDEX_NAME));
-        makeIndexAvailable(indexId(INDEX_NAME));
+
+        int indexId = indexId(INDEX_NAME);
+
+        startBuildingIndex(indexId);
+        makeIndexAvailable(indexId);
 
         dropIndex(INDEX_NAME);
 
-        CatalogIndexDescriptor index = index(manager.latestCatalogVersion(), INDEX_NAME);
+        CatalogIndexDescriptor index = manager.index(indexId, manager.latestCatalogVersion());
 
         assertThat(index, is(notNullValue()));
         assertThat(index.status(), is(STOPPING));
@@ -1950,13 +1955,15 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
         createSomeTable(TABLE_NAME);
         createSomeIndex(TABLE_NAME, INDEX_NAME);
 
-        rollIndexStatusTo(STOPPING, indexId(INDEX_NAME));
+        int indexId = indexId(INDEX_NAME);
+
+        rollIndexStatusTo(STOPPING, indexId);
 
-        assertThat(index(manager.latestCatalogVersion(), INDEX_NAME).status(), is(STOPPING));
+        assertThat(manager.index(indexId, manager.latestCatalogVersion()).status(), is(STOPPING));
 
-        removeIndex(indexId(INDEX_NAME));
+        removeIndex(indexId);
 
-        CatalogIndexDescriptor index = index(manager.latestCatalogVersion(), INDEX_NAME);
+        CatalogIndexDescriptor index = manager.index(indexId, manager.latestCatalogVersion());
 
         assertThat(index, is(nullValue()));
     }
@@ -2587,7 +2594,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
     }
 
     private @Nullable CatalogIndexDescriptor index(int catalogVersion, String indexName) {
-        return manager.schema(catalogVersion).index(indexName);
+        return manager.schema(catalogVersion).aliveIndex(indexName);
     }
 
     private int tableId(String tableName) {
@@ -2599,7 +2606,7 @@ public class CatalogManagerSelfTest extends BaseCatalogManagerTest {
     }
 
     private int indexId(String indexName) {
-        CatalogIndexDescriptor index = manager.index(indexName, clock.nowLong());
+        CatalogIndexDescriptor index = manager.aliveIndex(indexName, clock.nowLong());
 
         assertNotNull(index, indexName);
 
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AbstractCommandValidationTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AbstractCommandValidationTest.java
index 1b2dec3f3a..27d19c820c 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AbstractCommandValidationTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AbstractCommandValidationTest.java
@@ -23,14 +23,18 @@ import static org.apache.ignite.internal.catalog.commands.CatalogUtils.DEFAULT_P
 import static org.apache.ignite.internal.catalog.commands.CatalogUtils.DEFAULT_SCALE;
 import static org.apache.ignite.internal.catalog.commands.CatalogUtils.SYSTEM_SCHEMAS;
 import static org.apache.ignite.sql.ColumnType.INT32;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
 
 import java.util.List;
+import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.function.Consumer;
 import java.util.stream.Stream;
 import org.apache.ignite.internal.catalog.Catalog;
 import org.apache.ignite.internal.catalog.CatalogCommand;
 import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus;
 import org.apache.ignite.internal.catalog.descriptors.CatalogSchemaDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogSystemViewDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
@@ -96,15 +100,16 @@ abstract class AbstractCommandValidationTest extends BaseIgniteAbstractTest {
 
     static Catalog catalogWithZones(String zone1, String zone2) {
         return catalog(
-                List.of(createZoneCommand(zone1), createZoneCommand(zone2))
+                createZoneCommand(zone1),
+                createZoneCommand(zone2)
         );
     }
 
     static Catalog catalogWithIndex(String name) {
-        return catalog(List.of(
+        return catalog(
                 createTableCommand(TABLE_NAME),
                 createIndexCommand(TABLE_NAME, name)
-        ));
+        );
     }
 
     static CatalogCommand createIndexCommand(String tableName, String indexName) {
@@ -139,13 +144,7 @@ abstract class AbstractCommandValidationTest extends BaseIgniteAbstractTest {
                 .build();
     }
 
-    static Catalog catalog(CatalogCommand commandToApply) {
-        return catalog(List.of(commandToApply));
-    }
-
-    static Catalog catalog(List<CatalogCommand> commandsToApply) {
-        Catalog catalog = emptyCatalog();
-
+    static Catalog applyCommandsToCatalog(Catalog catalog, CatalogCommand... commandsToApply) {
         for (CatalogCommand command : commandsToApply) {
             for (UpdateEntry updates : command.get(catalog)) {
                 catalog = updates.applyUpdate(catalog, INITIAL_CAUSALITY_TOKEN);
@@ -155,6 +154,10 @@ abstract class AbstractCommandValidationTest extends BaseIgniteAbstractTest {
         return catalog;
     }
 
+    static Catalog catalog(CatalogCommand... commandsToApply) {
+        return applyCommandsToCatalog(emptyCatalog(), commandsToApply);
+    }
+
     static Catalog catalog(
             int version,
             CatalogTableDescriptor[] tables,
@@ -193,4 +196,39 @@ abstract class AbstractCommandValidationTest extends BaseIgniteAbstractTest {
     static CatalogTableColumnDescriptor tableColumn(String columnName) {
         return new CatalogTableColumnDescriptor(columnName, INT32, false, DEFAULT_PRECISION, DEFAULT_SCALE, DEFAULT_LENGTH, null);
     }
+
+    /**
+     * Transitions a given index from {@link CatalogIndexStatus#REGISTERED} to {@link CatalogIndexStatus#STOPPING} state.
+     *
+     * @throws NoSuchElementException if the given index does not exist.
+     */
+    static Catalog transitionIndexToStoppingState(Catalog catalog, String indexName) {
+        int indexId = findIndex(catalog, indexName).id();
+
+        CatalogCommand startIndexBuildingCommand = StartBuildingIndexCommand.builder()
+                .indexId(indexId)
+                .build();
+
+        CatalogCommand makeIndexAvailableCommand = MakeIndexAvailableCommand.builder()
+                .indexId(indexId)
+                .build();
+
+        CatalogCommand dropIndexCommand = DropIndexCommand.builder()
+                .schemaName(SCHEMA_NAME)
+                .indexName(indexName)
+                .build();
+
+        catalog = applyCommandsToCatalog(catalog, startIndexBuildingCommand, makeIndexAvailableCommand, dropIndexCommand);
+
+        assertThat(findIndex(catalog, indexName).status(), is(CatalogIndexStatus.STOPPING));
+
+        return catalog;
+    }
+
+    private static CatalogIndexDescriptor findIndex(Catalog catalog, String indexName) {
+        return catalog.indexes().stream()
+                .filter(index -> index.name().equals(indexName))
+                .findAny()
+                .orElseThrow();
+    }
 }
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommandValidationTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommandValidationTest.java
index 91a9927611..c386a51e26 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommandValidationTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/AlterTableDropColumnCommandValidationTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.catalog.commands;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrows;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
 import static org.apache.ignite.sql.ColumnType.INT32;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.util.HashSet;
@@ -31,6 +32,7 @@ import org.apache.ignite.internal.catalog.CatalogCommand;
 import org.apache.ignite.internal.catalog.CatalogValidationException;
 import org.apache.ignite.internal.catalog.descriptors.CatalogHashIndexDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogObjectDescriptor;
+import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.MethodSource;
@@ -212,13 +214,12 @@ public class AlterTableDropColumnCommandValidationTest extends AbstractCommandVa
 
     @Test
     void rightExceptionIsThrownIfSameColumnNameBelongsToIndexesForDifferentTables() {
-        Catalog catalog =  catalog(List.of(
+        Catalog catalog = catalog(
                 createTableCommand(TABLE_NAME),
                 createIndexCommand(TABLE_NAME, "TEST_IDX"),
                 createTableCommand(TABLE_NAME + "_1"),
                 createIndexCommand(TABLE_NAME + "_1", "TEST_IDX" + "_1")
-        ));
-
+        );
 
         Set<String> indexes = catalog.indexes().stream()
                 .filter(index -> ((CatalogHashIndexDescriptor) index).columns().contains("VAL"))
@@ -253,4 +254,23 @@ public class AlterTableDropColumnCommandValidationTest extends AbstractCommandVa
                 "Operations with reserved schemas are not allowed"
         );
     }
+
+    @Test
+    void noExceptionIsThrownIfColumnBelongsToStoppingIndex() {
+        String indexName = "TEST_IDX";
+
+        Catalog catalog = catalogWithIndex(indexName);
+
+        CatalogCommand dropColumnCommand = AlterTableDropColumnCommand.builder()
+                .schemaName(SCHEMA_NAME)
+                .tableName(TABLE_NAME)
+                .columns(Set.of("VAL"))
+                .build();
+
+        Assertions.assertThrows(CatalogValidationException.class, () -> applyCommandsToCatalog(catalog, dropColumnCommand));
+
+        Catalog newCatalog = transitionIndexToStoppingState(catalog, indexName);
+
+        assertDoesNotThrow(() -> applyCommandsToCatalog(newCatalog, dropColumnCommand));
+    }
 }
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CatalogUtilsTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CatalogUtilsTest.java
index 28550dc094..ac1183d69a 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CatalogUtilsTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CatalogUtilsTest.java
@@ -139,13 +139,13 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
 
         createTable(TABLE_NAME);
         createIndex(TABLE_NAME, indexName0);
-        createIndex(TABLE_NAME, indexName1);
-        createIndex(TABLE_NAME, indexName2);
+        int indexId1 = createIndex(TABLE_NAME, indexName1);
+        int indexId2 = createIndex(TABLE_NAME, indexName2);
 
-        startBuildingIndex(indexName1);
-        makeIndexAvailable(indexName1);
+        startBuildingIndex(indexId1);
+        makeIndexAvailable(indexId1);
 
-        startBuildingIndex(indexName2);
+        startBuildingIndex(indexId2);
 
         int latestCatalogVersion = catalogManager.latestCatalogVersion();
         int earliestCatalogVersion = catalogManager.earliestCatalogVersion();
@@ -181,13 +181,13 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
 
         createTable(TABLE_NAME);
         createIndex(TABLE_NAME, indexName0);
-        createIndex(TABLE_NAME, indexName1);
-        createIndex(TABLE_NAME, indexName2);
+        int indexId1 = createIndex(TABLE_NAME, indexName1);
+        int indexId2 = createIndex(TABLE_NAME, indexName2);
 
-        startBuildingIndex(indexName1);
-        makeIndexAvailable(indexName1);
+        startBuildingIndex(indexId1);
+        makeIndexAvailable(indexId1);
 
-        startBuildingIndex(indexName2);
+        startBuildingIndex(indexId2);
 
         int catalogVersionBeforeDropIndex0 = catalogManager.latestCatalogVersion();
 
@@ -199,7 +199,7 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
 
         int catalogVersionBeforeRemoveIndex1 = catalogManager.latestCatalogVersion();
 
-        removeIndex(indexName1);
+        removeIndex(indexId1);
 
         int latestCatalogVersion = catalogManager.latestCatalogVersion();
         int earliestCatalogVersion = catalogManager.earliestCatalogVersion();
@@ -250,23 +250,23 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
         String indexName3 = INDEX_NAME + 3;
 
         createTable(TABLE_NAME);
-        createIndex(TABLE_NAME, indexName0);
-        createIndex(TABLE_NAME, indexName1);
-        createIndex(TABLE_NAME, indexName2);
+        int indexId0 = createIndex(TABLE_NAME, indexName0);
+        int indexId1 = createIndex(TABLE_NAME, indexName1);
+        int indexId2 = createIndex(TABLE_NAME, indexName2);
         createIndex(TABLE_NAME, indexName3);
 
-        startBuildingIndex(indexName0);
-        startBuildingIndex(indexName1);
-        startBuildingIndex(indexName2);
+        startBuildingIndex(indexId0);
+        startBuildingIndex(indexId1);
+        startBuildingIndex(indexId2);
 
-        makeIndexAvailable(indexName0);
-        makeIndexAvailable(indexName2);
+        makeIndexAvailable(indexId0);
+        makeIndexAvailable(indexId2);
 
         dropIndex(indexName0);
 
         int catalogVersionBeforeRemoveIndex0 = catalogManager.latestCatalogVersion();
 
-        removeIndex(indexName0);
+        removeIndex(indexId0);
 
         int catalogVersionBeforeDropIndex3 = catalogManager.latestCatalogVersion();
 
@@ -353,7 +353,7 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
 
         assertThat(schema, is(notNullValue()));
 
-        var fooIndex = (CatalogHashIndexDescriptor) catalogManager.index("foo", clock.nowLong());
+        var fooIndex = (CatalogHashIndexDescriptor) catalogManager.aliveIndex("foo", clock.nowLong());
 
         assertThat(fooIndex, is(notNullValue()));
 
@@ -415,7 +415,7 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
         assertThat(catalogManager.execute(catalogCommand), willCompleteSuccessfully());
     }
 
-    private void createIndex(String tableName, String indexName) {
+    private int createIndex(String tableName, String indexName) {
         CatalogCommand catalogCommand = CreateHashIndexCommand.builder()
                 .schemaName(DEFAULT_SCHEMA_NAME)
                 .tableName(tableName)
@@ -425,22 +425,18 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
                 .build();
 
         assertThat(catalogManager.execute(catalogCommand), willCompleteSuccessfully());
-    }
 
-    private void startBuildingIndex(String indexName) {
-        CatalogIndexDescriptor index = index(catalogManager, catalogManager.latestCatalogVersion(), indexName);
+        return catalogManager.aliveIndex(indexName, clock.nowLong()).id();
+    }
 
-        CatalogCommand catalogCommand = StartBuildingIndexCommand.builder().indexId(index.id()).build();
+    private void startBuildingIndex(int indexId) {
+        CatalogCommand catalogCommand = StartBuildingIndexCommand.builder().indexId(indexId).build();
 
         assertThat(catalogManager.execute(catalogCommand), willCompleteSuccessfully());
     }
 
-    private void makeIndexAvailable(String indexName) {
-        CatalogIndexDescriptor index = index(catalogManager, catalogManager.latestCatalogVersion(), indexName);
-
-        CatalogCommand catalogCommand = MakeIndexAvailableCommand.builder()
-                .indexId(index.id())
-                .build();
+    private void makeIndexAvailable(int indexId) {
+        CatalogCommand catalogCommand = MakeIndexAvailableCommand.builder().indexId(indexId).build();
 
         assertThat(catalogManager.execute(catalogCommand), willCompleteSuccessfully());
     }
@@ -454,14 +450,8 @@ public class CatalogUtilsTest extends BaseIgniteAbstractTest {
         assertThat(catalogManager.execute(catalogCommand), willCompleteSuccessfully());
     }
 
-    private void removeIndex(String indexName) {
-        CatalogIndexDescriptor indexDescriptor = catalogManager.index(indexName, HybridTimestamp.MAX_VALUE.longValue());
-
-        assertThat(indexDescriptor, is(notNullValue()));
-
-        CatalogCommand catalogCommand = RemoveIndexCommand.builder()
-                .indexId(indexDescriptor.id())
-                .build();
+    private void removeIndex(int indexId) {
+        CatalogCommand catalogCommand = RemoveIndexCommand.builder().indexId(indexId).build();
 
         assertThat(catalogManager.execute(catalogCommand), willCompleteSuccessfully());
     }
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateAbstractIndexCommandValidationTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateAbstractIndexCommandValidationTest.java
index eb80690bfb..af561d4522 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateAbstractIndexCommandValidationTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateAbstractIndexCommandValidationTest.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.catalog.commands;
 
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
 import static org.apache.ignite.sql.ColumnType.INT32;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -219,4 +221,19 @@ public abstract class CreateAbstractIndexCommandValidationTest extends AbstractC
                 "Unique index must include all colocation columns"
         );
     }
+
+    @Test
+    void noExceptionIsThrownIfStoppingIndexWithGivenNameAlreadyExists() {
+        String indexName = "IDX";
+
+        Catalog catalog = catalogWithIndex(indexName);
+
+        CatalogCommand createIndexCommand = prefilledBuilder().indexName(indexName).build();
+
+        assertThrows(CatalogValidationException.class, () -> applyCommandsToCatalog(catalog, createIndexCommand));
+
+        Catalog newCatalog = transitionIndexToStoppingState(catalog, indexName);
+
+        assertDoesNotThrow(() -> applyCommandsToCatalog(newCatalog, createIndexCommand));
+    }
 }
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/DropZoneCommandValidationTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/DropZoneCommandValidationTest.java
index 23db9b4643..59edacd369 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/DropZoneCommandValidationTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/DropZoneCommandValidationTest.java
@@ -21,7 +21,6 @@ import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_ZONE_NAM
 import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrows;
 
-import java.util.List;
 import org.apache.ignite.internal.catalog.Catalog;
 import org.apache.ignite.internal.catalog.CatalogCommand;
 import org.apache.ignite.internal.catalog.CatalogValidationException;
@@ -61,10 +60,10 @@ public class DropZoneCommandValidationTest extends AbstractCommandValidationTest
     void rejectToDropZoneWithTable() {
         String tableName = "table1";
 
-        Catalog catalog = catalog(List.of(
+        Catalog catalog = catalog(
                 createZoneCommand(ZONE_NAME),
                 createTableCommand(ZONE_NAME, tableName)
-        ));
+        );
 
         assertThrows(
                 DistributionZoneCantBeDroppedValidationException.class,
diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/RenameTableCommandValidationTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/RenameTableCommandValidationTest.java
index ec68b0bc3c..891a8f2d76 100644
--- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/RenameTableCommandValidationTest.java
+++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/RenameTableCommandValidationTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.catalog.commands;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrows;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
 
-import java.util.List;
 import org.apache.ignite.internal.catalog.Catalog;
 import org.apache.ignite.internal.catalog.CatalogCommand;
 import org.apache.ignite.internal.catalog.CatalogValidationException;
@@ -113,10 +112,10 @@ public class RenameTableCommandValidationTest extends AbstractCommandValidationT
 
     @Test
     void exceptionIsThrownIfTableWithNewNameExists() {
-        Catalog catalog = catalog(List.of(
+        Catalog catalog = catalog(
                 createTableCommand("TEST"),
                 createTableCommand("TEST2")
-        ));
+        );
 
         CatalogCommand command = RenameTableCommand.builder()
                 .schemaName(SCHEMA_NAME)
diff --git a/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/FakeCatalogService.java b/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/FakeCatalogService.java
index 5301f19c77..2e74e02311 100644
--- a/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/FakeCatalogService.java
+++ b/modules/client-handler/src/testFixtures/java/org/apache/ignite/client/handler/FakeCatalogService.java
@@ -70,7 +70,7 @@ public class FakeCatalogService implements CatalogService {
     }
 
     @Override
-    public CatalogIndexDescriptor index(String indexName, long timestamp) {
+    public CatalogIndexDescriptor aliveIndex(String indexName, long timestamp) {
         return null;
     }
 
diff --git a/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItBuildIndexTest.java b/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItBuildIndexTest.java
index 11e8d9ca5d..58220a4e66 100644
--- a/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItBuildIndexTest.java
+++ b/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItBuildIndexTest.java
@@ -17,23 +17,17 @@
 
 package org.apache.ignite.internal.index;
 
-import static java.util.concurrent.CompletableFuture.runAsync;
 import static java.util.stream.Collectors.joining;
 import static org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus.AVAILABLE;
 import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
 import static org.apache.ignite.internal.raft.util.OptimizedMarshaller.NO_POOL;
 import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIndexScan;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willTimeoutFast;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
-import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast;
-import static org.apache.ignite.internal.util.CompletableFutures.falseCompletedFuture;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.hasSize;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -45,8 +39,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.function.BiPredicate;
 import java.util.stream.Stream;
@@ -54,11 +46,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.internal.app.IgniteImpl;
 import org.apache.ignite.internal.catalog.CatalogManager;
 import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
-import org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus;
 import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
-import org.apache.ignite.internal.catalog.events.CatalogEvent;
-import org.apache.ignite.internal.catalog.events.RemoveIndexEventParameters;
-import org.apache.ignite.internal.catalog.events.StartBuildingIndexEventParameters;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.network.NetworkMessage;
 import org.apache.ignite.internal.network.serialization.MessageSerializationRegistry;
@@ -72,12 +60,9 @@ import org.apache.ignite.internal.table.InternalTable;
 import org.apache.ignite.internal.table.TableTestUtils;
 import org.apache.ignite.internal.table.TableViewInternal;
 import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
-import org.apache.ignite.internal.table.distributed.replication.request.BuildIndexReplicaRequest;
 import org.apache.ignite.internal.table.distributed.schema.PartitionCommandsMarshallerImpl;
-import org.apache.ignite.internal.tx.InternalTransaction;
 import org.apache.ignite.raft.jraft.rpc.WriteActionRequest;
 import org.apache.ignite.table.Table;
-import org.apache.ignite.tx.TransactionOptions;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Assertions;
@@ -130,182 +115,6 @@ public class ItBuildIndexTest extends BaseSqlIntegrationTest {
                 .check();
     }
 
-    @Test
-    void testDropIndexDuringTransaction() throws Exception {
-        int partitions = initialNodes();
-
-        int replicas = initialNodes();
-
-        createAndPopulateTable(replicas, partitions);
-
-        createIndex(INDEX_NAME);
-
-        checkIndexBuild(partitions, replicas, INDEX_NAME);
-
-        CompletableFuture<Void> indexRemovedFuture = indexRemovedFuture();
-
-        IgniteImpl node = CLUSTER.aliveNode();
-
-        // Start a transaction. We expect that the index will not be removed until this transaction completes.
-        node.transactions().runInTransaction(tx -> {
-            dropIndex(INDEX_NAME);
-
-            CatalogIndexDescriptor indexDescriptor = getIndexDescriptor(node, INDEX_NAME);
-
-            assertThat(indexDescriptor, is(notNullValue()));
-            assertThat(indexDescriptor.status(), is(CatalogIndexStatus.STOPPING));
-            assertThat(indexRemovedFuture, willTimeoutFast());
-        }, new TransactionOptions().readOnly(false));
-
-        assertThat(indexRemovedFuture, willCompleteSuccessfully());
-    }
-
-    @Test
-    void testWritingIntoStoppingIndex() throws Exception {
-        int partitions = initialNodes();
-
-        int replicas = initialNodes();
-
-        createAndPopulateTable(replicas, partitions);
-
-        createIndex(INDEX_NAME);
-
-        checkIndexBuild(partitions, replicas, INDEX_NAME);
-
-        IgniteImpl node = CLUSTER.aliveNode();
-
-        // Latch for waiting for the RW transaction to start before dropping the index.
-        var startTransactionLatch = new CountDownLatch(1);
-        // Latch for waiting for the index to be dropped, before inserting data in the transaction.
-        var dropIndexLatch = new CountDownLatch(1);
-
-        CompletableFuture<Void> dropIndexFuture = runAsync(() -> {
-            try {
-                startTransactionLatch.await(1, TimeUnit.SECONDS);
-            } catch (InterruptedException e) {
-                throw new CompletionException(e);
-            }
-
-            dropIndex(INDEX_NAME);
-
-            CatalogIndexDescriptor indexDescriptor = getIndexDescriptor(node, INDEX_NAME);
-
-            assertThat(indexDescriptor, is(notNullValue()));
-            assertThat(indexDescriptor.status(), is(CatalogIndexStatus.STOPPING));
-
-            dropIndexLatch.countDown();
-        });
-
-        CompletableFuture<Void> insertDataIntoIndexTransaction = runAsync(() -> {
-            node.transactions().runInTransaction(tx -> {
-                startTransactionLatch.countDown();
-
-                try {
-                    dropIndexLatch.await(1, TimeUnit.SECONDS);
-                } catch (InterruptedException e) {
-                    throw new CompletionException(e);
-                }
-
-                // Insert data into a STOPPING index. We expect it to be inserted.
-                sql(tx, format("INSERT INTO {} VALUES {}", TABLE_NAME, toValuesString(List.of(239, 239))));
-
-                assertQuery((InternalTransaction) tx, format("SELECT * FROM {} WHERE i1 > 10", TABLE_NAME))
-                        .matches(containsIndexScan("PUBLIC", TABLE_NAME, INDEX_NAME))
-                        .returns(239, 239)
-                        .check();
-            }, new TransactionOptions().readOnly(false));
-        });
-
-        assertThat(dropIndexFuture, willCompleteSuccessfully());
-        assertThat(insertDataIntoIndexTransaction, willCompleteSuccessfully());
-    }
-
-    @Test
-    void testDropIndexAfterRegistering() {
-        int partitions = initialNodes();
-
-        int replicas = initialNodes();
-
-        createAndPopulateTable(replicas, partitions);
-
-        CompletableFuture<Void> indexRemovedFuture = indexRemovedFuture();
-
-        CLUSTER.aliveNode().transactions().runInTransaction(tx -> {
-            // Create an index inside a transaction, this will prevent the index from building.
-            try {
-                createIndex(INDEX_NAME);
-
-                dropIndex(INDEX_NAME);
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-        }, new TransactionOptions().readOnly(false));
-
-        assertThat(indexRemovedFuture, willCompleteSuccessfully());
-    }
-
-    @Test
-    void testDropIndexDuringBuilding() throws Exception {
-        int partitions = initialNodes();
-
-        int replicas = initialNodes();
-
-        createAndPopulateTable(replicas, partitions);
-
-        // Block index building messages, this way index will never become AVAILABLE.
-        CLUSTER.runningNodes().forEach(ignite -> ignite.dropMessages((id, message) -> message instanceof BuildIndexReplicaRequest));
-
-        CompletableFuture<Void> indexBuildingFuture = indexBuildingFuture();
-
-        CompletableFuture<Void> indexRemovedFuture = indexRemovedFuture();
-
-        createIndex(INDEX_NAME);
-
-        assertThat(indexBuildingFuture, willCompleteSuccessfully());
-
-        dropIndex(INDEX_NAME);
-
-        assertThat(indexRemovedFuture, willCompleteSuccessfully());
-    }
-
-    private static CompletableFuture<Void> indexBuildingFuture() {
-        IgniteImpl node = CLUSTER.aliveNode();
-
-        var indexBuildingFuture = new CompletableFuture<Void>();
-
-        node.catalogManager().listen(CatalogEvent.INDEX_BUILDING, (StartBuildingIndexEventParameters parameters) -> {
-            CatalogIndexDescriptor indexDescriptor = node.catalogManager().index(parameters.indexId(), parameters.catalogVersion());
-
-            if (indexDescriptor != null && indexDescriptor.name().equals(INDEX_NAME)) {
-                indexBuildingFuture.complete(null);
-            }
-
-            return falseCompletedFuture();
-        });
-
-        return indexBuildingFuture;
-    }
-
-    private static CompletableFuture<Void> indexRemovedFuture() {
-        IgniteImpl node = CLUSTER.aliveNode();
-
-        var indexRemovedFuture = new CompletableFuture<Void>();
-
-        node.catalogManager().listen(CatalogEvent.INDEX_REMOVED, (RemoveIndexEventParameters parameters) -> {
-            node.catalogManager()
-                    .catalog(parameters.catalogVersion() - 1)
-                    .indexes()
-                    .stream()
-                    .filter(index -> index.name().equals(INDEX_NAME))
-                    .findAny()
-                    .ifPresent(index -> indexRemovedFuture.complete(null));
-
-            return falseCompletedFuture();
-        });
-
-        return indexRemovedFuture;
-    }
-
     @Test
     @Disabled("https://issues.apache.org/jira/browse/IGNITE-20525")
     void testChangePrimaryReplicaOnMiddleBuildIndex() throws Exception {
@@ -592,7 +401,7 @@ public class ItBuildIndexTest extends BaseSqlIntegrationTest {
     private static @Nullable CatalogIndexDescriptor getIndexDescriptor(Ignite node, String indexName) {
         IgniteImpl nodeImpl = (IgniteImpl) node;
 
-        return nodeImpl.catalogManager().index(indexName, nodeImpl.clock().nowLong());
+        return nodeImpl.catalogManager().aliveIndex(indexName, nodeImpl.clock().nowLong());
     }
 
     /**
@@ -609,7 +418,7 @@ public class ItBuildIndexTest extends BaseSqlIntegrationTest {
         CatalogManager catalogManager = ignite.catalogManager();
         HybridClock clock = ignite.clock();
 
-        CatalogIndexDescriptor indexDescriptor = catalogManager.index(indexName, clock.nowLong());
+        CatalogIndexDescriptor indexDescriptor = catalogManager.aliveIndex(indexName, clock.nowLong());
 
         return indexDescriptor != null && indexDescriptor.status() == AVAILABLE;
     }
diff --git a/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItDropIndexMultipleNodesTest.java b/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItDropIndexMultipleNodesTest.java
new file mode 100644
index 0000000000..a30708087f
--- /dev/null
+++ b/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItDropIndexMultipleNodesTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.runAsync;
+import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIndexScan;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willTimeoutFast;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiPredicate;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus;
+import org.apache.ignite.internal.catalog.events.CatalogEvent;
+import org.apache.ignite.internal.catalog.events.RemoveIndexEventParameters;
+import org.apache.ignite.internal.catalog.events.StartBuildingIndexEventParameters;
+import org.apache.ignite.internal.event.EventListener;
+import org.apache.ignite.internal.event.EventParameters;
+import org.apache.ignite.internal.sql.BaseSqlIntegrationTest;
+import org.apache.ignite.internal.table.distributed.replication.request.BuildIndexReplicaRequest;
+import org.apache.ignite.internal.tx.InternalTransaction;
+import org.apache.ignite.tx.Transaction;
+import org.apache.ignite.tx.TransactionOptions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test class for scenarios related to dropping of indices, executed on a multiple node cluster.
+ */
+public class ItDropIndexMultipleNodesTest extends BaseSqlIntegrationTest {
+    private static final String TABLE_NAME = "TEST";
+
+    private static final String INDEX_NAME = "TEST_IDX";
+
+    @BeforeEach
+    void createTable() {
+        int partitions = initialNodes();
+
+        int replicas = initialNodes();
+
+        createTable(TABLE_NAME, replicas, partitions);
+    }
+
+    @AfterEach
+    void cleanup() {
+        CLUSTER.runningNodes().forEach(IgniteImpl::stopDroppingMessages);
+
+        dropAllTables();
+    }
+
+    @Test
+    void testActiveRwTransactionPreventsStoppingIndexFromBeingRemoved() {
+        int indexId = createIndex();
+
+        CompletableFuture<Void> indexRemovedFuture = indexRemovedFuture();
+
+        IgniteImpl node = CLUSTER.aliveNode();
+
+        // Start a transaction. We expect that the index will not be removed until this transaction completes.
+        runInRwTransaction(node, tx -> {
+            dropIndex();
+
+            CatalogIndexDescriptor indexDescriptor = node.catalogManager().index(indexId, node.clock().nowLong());
+
+            assertThat(indexDescriptor, is(notNullValue()));
+            assertThat(indexDescriptor.status(), is(CatalogIndexStatus.STOPPING));
+            assertThat(indexRemovedFuture, willTimeoutFast());
+        });
+
+        assertThat(indexRemovedFuture, willCompleteSuccessfully());
+    }
+
+    @Test
+    void testWritingIntoStoppingIndex() {
+        int indexId = createIndex();
+
+        IgniteImpl node = CLUSTER.aliveNode();
+
+        // Latch for waiting for the RW transaction to start before dropping the index.
+        var startTransactionLatch = new CountDownLatch(1);
+        // Latch for waiting for the index to be dropped, before inserting data in the transaction.
+        var dropIndexLatch = new CountDownLatch(1);
+
+        CompletableFuture<Void> dropIndexFuture = runAsync(() -> {
+            try {
+                startTransactionLatch.await(1, TimeUnit.SECONDS);
+            } catch (InterruptedException e) {
+                throw new CompletionException(e);
+            }
+
+            dropIndex();
+
+            CatalogIndexDescriptor indexDescriptor = node.catalogManager().index(indexId, node.clock().nowLong());
+
+            assertThat(indexDescriptor, is(notNullValue()));
+            assertThat(indexDescriptor.status(), is(CatalogIndexStatus.STOPPING));
+
+            dropIndexLatch.countDown();
+        });
+
+        CompletableFuture<Void> insertDataIntoIndexTransaction = runAsync(() -> {
+            runInRwTransaction(node, tx -> {
+                startTransactionLatch.countDown();
+
+                try {
+                    dropIndexLatch.await(1, TimeUnit.SECONDS);
+                } catch (InterruptedException e) {
+                    throw new CompletionException(e);
+                }
+
+                // Insert data into a STOPPING index. We expect it to be inserted.
+                insertPeople(tx, TABLE_NAME, new Person(239, "foo", 0));
+
+                assertQuery((InternalTransaction) tx, String.format("SELECT id FROM %s WHERE NAME > 'a'", TABLE_NAME))
+                        .matches(containsIndexScan("PUBLIC", TABLE_NAME, INDEX_NAME))
+                        .returns(239)
+                        .check();
+            });
+        });
+
+        assertThat(dropIndexFuture, willCompleteSuccessfully());
+        assertThat(insertDataIntoIndexTransaction, willCompleteSuccessfully());
+    }
+
+    /**
+     * Tests the following scenario.
+     *
+     * <ol>
+     *     <li>Transaction A is started that is expected to observe an index in the {@link CatalogIndexStatus#AVAILABLE} state;</li>
+     *     <li>The index is dropped;</li>
+     *     <li>Transaction B is started that is expected to observe the index in the {@link CatalogIndexStatus#STOPPING} state;</li>
+     *     <li>Transaction B inserts data into the table and, therefore, into the index;</li>
+     *     <li>Transaction A performs a scan that utilizes the index over the table and is expected to see the data written by
+     *     Transaction B.</li>
+     * </ol>
+     */
+    @Test
+    void testWritingIntoStoppingIndexInDifferentTransactions() {
+        int indexId = createIndex();
+
+        IgniteImpl node = CLUSTER.aliveNode();
+
+        // Latch that will be released when a transaction expected to observe the index in the AVAILABLE state is started.
+        var startAvailableTransactionLatch = new CountDownLatch(1);
+        // Latch that will be released when a transaction expected to observe the index in the STOPPING state is started.
+        var insertDataTransactionLatch = new CountDownLatch(1);
+        // Latch for waiting for the index to be dropped.
+        var dropIndexLatch = new CountDownLatch(1);
+
+        CompletableFuture<Void> dropIndexFuture = runAsync(() -> {
+            // Wait for a transaction to start before dropping the index. This way, the dropped index will be stuck in the
+            // STOPPING state until that transaction finishes.
+            try {
+                startAvailableTransactionLatch.await(1, TimeUnit.SECONDS);
+            } catch (InterruptedException e) {
+                throw new CompletionException(e);
+            }
+
+            dropIndex();
+
+            CatalogIndexDescriptor indexDescriptor = node.catalogManager().index(indexId, node.clock().nowLong());
+
+            assertThat(indexDescriptor, is(notNullValue()));
+            assertThat(indexDescriptor.status(), is(CatalogIndexStatus.STOPPING));
+
+            dropIndexLatch.countDown();
+        });
+
+        CompletableFuture<Void> insertDataIntoIndexTransaction = runAsync(() -> {
+            // Wait for the index to be dropped, so that the transaction will see the index in the STOPPING state.
+            try {
+                dropIndexLatch.await(1, TimeUnit.SECONDS);
+            } catch (InterruptedException e) {
+                throw new CompletionException(e);
+            }
+
+            // Insert data into a STOPPING index. We expect it to be inserted.
+            runInRwTransaction(node, tx -> insertPeople(tx, TABLE_NAME, new Person(239, "foo", 0)));
+
+            insertDataTransactionLatch.countDown();
+        });
+
+        CompletableFuture<Void> readDataFromIndexTransaction = runAsync(() -> runInRwTransaction(node, tx -> {
+            startAvailableTransactionLatch.countDown();
+
+            // Wait for the transaction that will insert data into the stopping index.
+            try {
+                insertDataTransactionLatch.await(1, TimeUnit.SECONDS);
+            } catch (InterruptedException e) {
+                throw new CompletionException(e);
+            }
+
+            assertQuery((InternalTransaction) tx, String.format("SELECT id FROM %s WHERE NAME > 'a'", TABLE_NAME))
+                    .matches(containsIndexScan("PUBLIC", TABLE_NAME, INDEX_NAME))
+                    .returns(239)
+                    .check();
+        }));
+
+        assertThat(dropIndexFuture, willCompleteSuccessfully());
+        assertThat(insertDataIntoIndexTransaction, willCompleteSuccessfully());
+        assertThat(readDataFromIndexTransaction, willCompleteSuccessfully());
+    }
+
+    @Test
+    void testDropIndexAfterRegistering() {
+        // We are going to block index building.
+        setAwaitIndexAvailability(false);
+
+        populateTable();
+
+        CompletableFuture<Void> indexRemovedFuture = indexRemovedFuture();
+
+        runInRwTransaction(CLUSTER.aliveNode(), tx -> {
+            // Create an index inside a transaction, this will prevent the index from building.
+            try {
+                createIndex();
+
+                dropIndex();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        });
+
+        assertThat(indexRemovedFuture, willCompleteSuccessfully());
+    }
+
+    @Test
+    void testDropIndexDuringBuilding() {
+        // We are going to block index building.
+        setAwaitIndexAvailability(false);
+
+        populateTable();
+
+        // Block index building messages, this way index will never become AVAILABLE.
+        CLUSTER.runningNodes().forEach(ignite -> ignite.dropMessages((id, message) -> message instanceof BuildIndexReplicaRequest));
+
+        CompletableFuture<Void> indexBuildingFuture = indexBuildingFuture();
+
+        CompletableFuture<Void> indexRemovedFuture = indexRemovedFuture();
+
+        createIndex();
+
+        assertThat(indexBuildingFuture, willCompleteSuccessfully());
+
+        dropIndex();
+
+        assertThat(indexRemovedFuture, willCompleteSuccessfully());
+    }
+
+    private static int createIndex() {
+        createIndex(TABLE_NAME, INDEX_NAME, "name");
+
+        IgniteImpl node = CLUSTER.aliveNode();
+
+        return node.catalogManager().aliveIndex(INDEX_NAME, node.clock().nowLong()).id();
+    }
+
+    private static void dropIndex() {
+        dropIndex(INDEX_NAME);
+    }
+
+    private static void runInRwTransaction(IgniteImpl node, Consumer<Transaction> action) {
+        node.transactions().runInTransaction(action, new TransactionOptions().readOnly(false));
+    }
+
+    private static void populateTable() {
+        int idx = 0;
+
+        insertData(TABLE_NAME, List.of("ID", "NAME", "SALARY"), new Object[][]{
+                {idx++, "Igor", 10.0d},
+                {idx++, null, 15.0d},
+                {idx++, "Ilya", 15.0d},
+                {idx++, "Roma", 10.0d},
+                {idx, "Roma", 10.0d}
+        });
+    }
+
+    private static CompletableFuture<Void> indexBuildingFuture() {
+        return indexEventFuture(CatalogEvent.INDEX_BUILDING, (StartBuildingIndexEventParameters parameters, CatalogService catalog) -> {
+            CatalogIndexDescriptor indexDescriptor = catalog.index(parameters.indexId(), parameters.catalogVersion());
+
+            return indexDescriptor != null && indexDescriptor.name().equals(INDEX_NAME);
+        });
+    }
+
+    private static CompletableFuture<Void> indexRemovedFuture() {
+        return indexEventFuture(CatalogEvent.INDEX_REMOVED, (RemoveIndexEventParameters parameters, CatalogService catalog) ->
+                catalog.catalog(parameters.catalogVersion() - 1)
+                        .indexes()
+                        .stream()
+                        .anyMatch(index -> index.name().equals(INDEX_NAME))
+        );
+    }
+
+    private static <P extends EventParameters> CompletableFuture<Void> indexEventFuture(
+            CatalogEvent event, BiPredicate<P, CatalogService> action
+    ) {
+        CatalogService catalog = CLUSTER.aliveNode().catalogManager();
+
+        var result = new CompletableFuture<Void>();
+
+        catalog.listen(event, EventListener.fromConsumer(parameters -> {
+            if (action.test((P) parameters, catalog)) {
+                result.complete(null);
+            }
+        }));
+
+        return result;
+    }
+}
diff --git a/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItDropIndexOneNodeTest.java b/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItDropIndexOneNodeTest.java
new file mode 100644
index 0000000000..3339357328
--- /dev/null
+++ b/modules/index/src/integrationTest/java/org/apache/ignite/internal/index/ItDropIndexOneNodeTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.index;
+
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+
+import java.util.function.Consumer;
+import org.apache.ignite.internal.ClusterPerClassIntegrationTest;
+import org.apache.ignite.internal.catalog.CatalogValidationException;
+import org.apache.ignite.tx.Transaction;
+import org.apache.ignite.tx.TransactionOptions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test class for scenarios related to dropping of indices, executed on a single node cluster.
+ */
+public class ItDropIndexOneNodeTest extends ClusterPerClassIntegrationTest {
+    private static final String TABLE_NAME = "TEST";
+
+    private static final String INDEX_NAME = "TEST_IDX";
+
+    private static final String COLUMN_NAME = "name";
+
+    @Override
+    protected int initialNodes() {
+        return 1;
+    }
+
+    @BeforeEach
+    void createTable() {
+        createTable(TABLE_NAME, 1, 1);
+
+        createIndex(TABLE_NAME, INDEX_NAME, COLUMN_NAME);
+    }
+
+    @AfterEach
+    void cleanup() {
+        dropAllTables();
+    }
+
+    @Test
+    void testCreateIndexAfterDrop() {
+        dropIndex(INDEX_NAME);
+
+        assertDoesNotThrow(() -> createIndex(TABLE_NAME, INDEX_NAME, COLUMN_NAME));
+    }
+
+    @Test
+    void testCreateIndexAfterDropWhileTransactionInProgress() {
+        runInRwTransaction(tx -> {
+            dropIndex(INDEX_NAME);
+
+            // The new index will not become available, since we are inside a transaction that has been started before this index was
+            // created.
+            setAwaitIndexAvailability(false);
+
+            assertDoesNotThrow(() -> createIndex(TABLE_NAME, INDEX_NAME, COLUMN_NAME));
+        });
+    }
+
+    @Test
+    void testDoubleCreateIndex() {
+        assertThrowsWithCause(() -> createIndex(TABLE_NAME, INDEX_NAME, COLUMN_NAME), CatalogValidationException.class);
+    }
+
+    @Test
+    void testDropIndexColumn() {
+        dropIndex(INDEX_NAME);
+
+        assertDoesNotThrow(ItDropIndexOneNodeTest::dropIndexedColumn);
+    }
+
+    @Test
+    void testDropIndexColumnWhileTransactionInProgress() {
+        runInRwTransaction(tx -> {
+            dropIndex(INDEX_NAME);
+
+            assertDoesNotThrow(ItDropIndexOneNodeTest::dropIndexedColumn);
+        });
+    }
+
+    @Test
+    void testDropIndexColumnFails() {
+        assertThrowsWithCause(ItDropIndexOneNodeTest::dropIndexedColumn, CatalogValidationException.class);
+    }
+
+    private static void runInRwTransaction(Consumer<Transaction> action) {
+        CLUSTER.aliveNode().transactions().runInTransaction(action, new TransactionOptions().readOnly(false));
+    }
+
+    private static void dropIndexedColumn() {
+        sql(String.format("ALTER TABLE %s DROP COLUMN %s", TABLE_NAME, COLUMN_NAME));
+    }
+}
diff --git a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexChooserTest.java b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexChooserTest.java
index c1b47ef863..23da6a9001 100644
--- a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexChooserTest.java
+++ b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexChooserTest.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.catalog.commands.StartBuildingIndexCommand;
 import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.table.TableTestUtils;
 import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.junit.jupiter.api.AfterEach;
@@ -113,8 +114,8 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
     @ParameterizedTest(name = "withRecovery = {0}")
     @ValueSource(booleans = {false, true})
     void testChooseForRwTxOperationAfterStartBuildingIndex(boolean withRecovery) {
-        createIndex(INDEX_NAME);
-        startBuildingIndex(INDEX_NAME);
+        int indexId = createIndex(INDEX_NAME);
+        startBuildingIndex(indexId);
 
         int catalogVersion = catalogManager.latestCatalogVersion();
 
@@ -131,9 +132,9 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
     @ParameterizedTest(name = "withRecovery = {0}")
     @ValueSource(booleans = {false, true})
     void testChooseForRwTxOperationAfterMakeIndexAvailable(boolean withRecovery) {
-        createIndex(INDEX_NAME);
-        startBuildingIndex(INDEX_NAME);
-        makeIndexAvailable(INDEX_NAME);
+        int indexId = createIndex(INDEX_NAME);
+        startBuildingIndex(indexId);
+        makeIndexAvailable(indexId);
 
         int catalogVersion = catalogManager.latestCatalogVersion();
 
@@ -168,8 +169,8 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
     @ParameterizedTest(name = "withRecovery = {0}")
     @ValueSource(booleans = {false, true})
     void testChooseForRwTxOperationAfterDropBuildingIndex(boolean withRecovery) {
-        createIndex(INDEX_NAME);
-        startBuildingIndex(INDEX_NAME);
+        int indexId = createIndex(INDEX_NAME);
+        startBuildingIndex(indexId);
         dropIndex(INDEX_NAME);
 
         int catalogVersion = catalogManager.latestCatalogVersion();
@@ -187,9 +188,9 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
     @ParameterizedTest(name = "withRecovery = {0}")
     @ValueSource(booleans = {false, true})
     void testChooseForRwTxOperationAfterDropAvailableIndex(boolean withRecovery) {
-        createIndex(INDEX_NAME);
-        startBuildingIndex(INDEX_NAME);
-        makeIndexAvailable(INDEX_NAME);
+        int indexId = createIndex(INDEX_NAME);
+        startBuildingIndex(indexId);
+        makeIndexAvailable(indexId);
         dropIndex(INDEX_NAME);
 
         int catalogVersion = catalogManager.latestCatalogVersion();
@@ -207,14 +208,14 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
     @ParameterizedTest(name = "withRecovery = {0}")
     @ValueSource(booleans = {false, true})
     void testChooseForRwTxOperationAfterRemoveStoppedIndex(boolean withRecovery) {
-        createIndex(INDEX_NAME);
-        startBuildingIndex(INDEX_NAME);
-        makeIndexAvailable(INDEX_NAME);
+        int indexId = createIndex(INDEX_NAME);
+        startBuildingIndex(indexId);
+        makeIndexAvailable(indexId);
         dropIndex(INDEX_NAME);
 
         int catalogVersionAfterDropIndex = catalogManager.latestCatalogVersion();
 
-        removeIndex(INDEX_NAME);
+        removeIndex(indexId);
 
         int catalogVersion = catalogManager.latestCatalogVersion();
 
@@ -263,13 +264,16 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
                 toStartBuildingIndexCommand(indexName5)
         );
 
+        int indexId1 = indexId(catalogManager, indexName1, clock);
+        int indexId4 = indexId(catalogManager, indexName4, clock);
+
         // after execute: I0(A) I1(A) I3(B) I4(S)
         executeCatalogCommands(toDropIndexCommand(indexName4), toDropIndexCommand(indexName5));
 
         int catalogVersionBeforeRemoveIndex4 = catalogManager.latestCatalogVersion();
 
         // after execute: I0(A) I1(A) I3(B)
-        executeCatalogCommands(toRemoveIndexCommand(indexName4));
+        executeCatalogCommands(toRemoveIndexCommand(indexId4));
 
         // after execute: I0(A) I1(S) I3(B)
         executeCatalogCommands(toDropIndexCommand(indexName1));
@@ -277,7 +281,7 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
         int catalogVersionBeforeRemoveIndex1 = catalogManager.latestCatalogVersion();
 
         // after execute: I0(A) I3(B)
-        executeCatalogCommands(toRemoveIndexCommand(indexName1));
+        executeCatalogCommands(toRemoveIndexCommand(indexId1));
 
         // after execute: I0(A) I3(B) I6(R)
         executeCatalogCommands(toCreateHashIndexCommand(indexName6));
@@ -301,19 +305,17 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
         );
     }
 
-    private void createIndex(String indexName) {
+    private int createIndex(String indexName) {
         TestIndexManagementUtils.createIndex(catalogManager, TABLE_NAME, indexName, COLUMN_NAME);
-    }
 
-    private void startBuildingIndex(String indexName) {
-        int indexId = indexId(catalogManager, indexName, clock);
+        return indexId(catalogManager, indexName, clock);
+    }
 
+    private void startBuildingIndex(int indexId) {
         TestIndexManagementUtils.startBuildingIndex(catalogManager, indexId);
     }
 
-    private void makeIndexAvailable(String indexName) {
-        int indexId = indexId(catalogManager, indexName, clock);
-
+    private void makeIndexAvailable(int indexId) {
         TestIndexManagementUtils.makeIndexAvailable(catalogManager, indexId);
     }
 
@@ -321,8 +323,8 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
         TestIndexManagementUtils.dropIndex(catalogManager, indexName);
     }
 
-    private void removeIndex(String indexName) {
-        TestIndexManagementUtils.removeIndex(catalogManager, indexName);
+    private void removeIndex(int indexId) {
+        TableTestUtils.removeIndex(catalogManager, indexId);
     }
 
     private List<CatalogIndexDescriptor> chooseForRwTxOperation(int catalogVersion) {
@@ -381,9 +383,9 @@ public class IndexChooserTest extends BaseIgniteAbstractTest {
                 .build();
     }
 
-    private CatalogCommand toRemoveIndexCommand(String indexName) {
+    private static CatalogCommand toRemoveIndexCommand(int indexId) {
         return RemoveIndexCommand.builder()
-                .indexId(indexId(catalogManager, indexName, clock))
+                .indexId(indexId)
                 .build();
     }
 }
diff --git a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
index 65322897ff..10f8c94ebd 100644
--- a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
+++ b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
@@ -221,7 +221,7 @@ public class IndexManagerTest extends BaseIgniteAbstractTest {
     void testDestroyIndex() throws Exception {
         createIndex(TABLE_NAME, INDEX_NAME);
 
-        CatalogIndexDescriptor indexDescriptor = catalogManager.index(INDEX_NAME, catalogManager.latestCatalogVersion());
+        CatalogIndexDescriptor indexDescriptor = catalogManager.aliveIndex(INDEX_NAME, catalogManager.latestCatalogVersion());
         int indexId = indexDescriptor.id();
         int tableId = indexDescriptor.tableId();
 
@@ -239,7 +239,7 @@ public class IndexManagerTest extends BaseIgniteAbstractTest {
     void testIndexDestroyedWithTable() throws Exception {
         createIndex(TABLE_NAME, INDEX_NAME);
 
-        CatalogIndexDescriptor indexDescriptor = catalogManager.index(INDEX_NAME, catalogManager.latestCatalogVersion());
+        CatalogIndexDescriptor indexDescriptor = catalogManager.aliveIndex(INDEX_NAME, catalogManager.latestCatalogVersion());
         int indexId = indexDescriptor.id();
         int tableId = indexDescriptor.tableId();
 
diff --git a/modules/index/src/test/java/org/apache/ignite/internal/index/TestIndexManagementUtils.java b/modules/index/src/test/java/org/apache/ignite/internal/index/TestIndexManagementUtils.java
index a4698c2624..66e8b3dbf1 100644
--- a/modules/index/src/test/java/org/apache/ignite/internal/index/TestIndexManagementUtils.java
+++ b/modules/index/src/test/java/org/apache/ignite/internal/index/TestIndexManagementUtils.java
@@ -91,10 +91,6 @@ class TestIndexManagementUtils {
         TableTestUtils.dropIndex(catalogManager, DEFAULT_SCHEMA_NAME, indexName);
     }
 
-    static void removeIndex(CatalogManager catalogManager, String indexName) {
-        TableTestUtils.removeIndex(catalogManager, indexName);
-    }
-
     static int indexId(CatalogService catalogService, String indexName, HybridClock clock) {
         return TableTestUtils.getIndexIdStrict(catalogService, indexName, clock.nowLong());
     }
diff --git a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
index 15f3b15784..32233e6e49 100644
--- a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
+++ b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
@@ -511,7 +511,7 @@ public abstract class ClusterPerClassIntegrationTest extends IgniteIntegrationTe
         CatalogManager catalogManager = ignite.catalogManager();
         HybridClock clock = ignite.clock();
 
-        CatalogIndexDescriptor indexDescriptor = catalogManager.index(indexName, clock.nowLong());
+        CatalogIndexDescriptor indexDescriptor = catalogManager.aliveIndex(indexName, clock.nowLong());
 
         return indexDescriptor != null && indexDescriptor.status() == AVAILABLE;
     }
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
index 85f3068b41..7a5b193692 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
@@ -129,8 +129,8 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
 
         CatalogTableDescriptor catalogTableDescriptor = catalogService.table(TABLE_NAME, clock.nowLong());
 
-        CatalogIndexDescriptor catalogSortedIndexDescriptor = catalogService.index(SORTED_INDEX_NAME, clock.nowLong());
-        CatalogIndexDescriptor catalogHashIndexDescriptor = catalogService.index(HASH_INDEX_NAME, clock.nowLong());
+        CatalogIndexDescriptor catalogSortedIndexDescriptor = catalogService.aliveIndex(SORTED_INDEX_NAME, clock.nowLong());
+        CatalogIndexDescriptor catalogHashIndexDescriptor = catalogService.aliveIndex(HASH_INDEX_NAME, clock.nowLong());
 
         sortedIdx = new StorageSortedIndexDescriptor(catalogTableDescriptor, (CatalogSortedIndexDescriptor) catalogSortedIndexDescriptor);
         hashIdx = new StorageHashIndexDescriptor(catalogTableDescriptor, (CatalogHashIndexDescriptor) catalogHashIndexDescriptor);
@@ -806,8 +806,8 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
         );
 
         when(catalogService.table(eq(TABLE_NAME), anyLong())).thenReturn(tableDescriptor);
-        when(catalogService.index(eq(SORTED_INDEX_NAME), anyLong())).thenReturn(sortedIndex);
-        when(catalogService.index(eq(HASH_INDEX_NAME), anyLong())).thenReturn(hashIndex);
+        when(catalogService.aliveIndex(eq(SORTED_INDEX_NAME), anyLong())).thenReturn(sortedIndex);
+        when(catalogService.aliveIndex(eq(HASH_INDEX_NAME), anyLong())).thenReturn(hashIndex);
 
         when(catalogService.table(eq(tableId), anyInt())).thenReturn(tableDescriptor);
         when(catalogService.index(eq(sortedIndexId), anyInt())).thenReturn(sortedIndex);
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
index 02ca308c39..8d01b5d4be 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
@@ -55,7 +55,7 @@ public abstract class AbstractHashIndexStorageTest extends AbstractIndexStorageT
                 Stream.of(columnTypes).map(AbstractIndexStorageTest::columnName).collect(toList())
         );
 
-        when(catalogService.index(eq(name), anyLong())).thenReturn(catalogHashIndexDescriptor);
+        when(catalogService.aliveIndex(eq(name), anyLong())).thenReturn(catalogHashIndexDescriptor);
         when(catalogService.index(eq(catalogHashIndexDescriptor.id()), anyInt())).thenReturn(catalogHashIndexDescriptor);
 
         return tableStorage.getOrCreateHashIndex(
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
index afca52cccd..34f00f9236 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
@@ -125,7 +125,7 @@ public abstract class AbstractSortedIndexStorageTest extends AbstractIndexStorag
                 List.of(columns)
         );
 
-        when(catalogService.index(eq(catalogSortedIndexDescriptor.name()), anyLong())).thenReturn(catalogSortedIndexDescriptor);
+        when(catalogService.aliveIndex(eq(catalogSortedIndexDescriptor.name()), anyLong())).thenReturn(catalogSortedIndexDescriptor);
         when(catalogService.index(eq(catalogSortedIndexDescriptor.id()), anyInt())).thenReturn(catalogSortedIndexDescriptor);
 
         return tableStorage.getOrCreateSortedIndex(
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableUtilsTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableUtilsTest.java
index aa9d1f3ba4..f4075a89d4 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableUtilsTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableUtilsTest.java
@@ -69,6 +69,12 @@ public class TableUtilsTest extends IgniteAbstractTest {
                 createSimpleHashIndex(catalogManager, TABLE_NAME, indexName);
             }
 
+            int indexId0 = indexId(catalogManager, indexName0);
+            int indexId1 = indexId(catalogManager, indexName1);
+            int indexId2 = indexId(catalogManager, indexName2);
+            int indexId3 = indexId(catalogManager, indexName3);
+            int indexId4 = indexId(catalogManager, indexName4);
+
             for (String indexName : List.of(indexName1, indexName2, indexName3, indexName4)) {
                 startBuildingIndex(catalogManager, indexId(catalogManager, indexName));
             }
@@ -81,7 +87,7 @@ public class TableUtilsTest extends IgniteAbstractTest {
                 dropIndex(catalogManager, DEFAULT_SCHEMA_NAME, indexName);
             }
 
-            removeIndex(catalogManager, indexName4);
+            removeIndex(catalogManager, indexId4);
 
             CatalogManager spy = spy(catalogManager);
 
@@ -93,10 +99,10 @@ public class TableUtilsTest extends IgniteAbstractTest {
                     indexIdsAtRwTxBeginTs(spy, transactionId(beginTs, 1), tableId),
                     contains(
                             indexId(catalogManager, pkIndexName(TABLE_NAME)),
-                            indexId(catalogManager, indexName0),
-                            indexId(catalogManager, indexName1),
-                            indexId(catalogManager, indexName2),
-                            indexId(catalogManager, indexName3)
+                            indexId0,
+                            indexId1,
+                            indexId2,
+                            indexId3
                     )
             );
 
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/FullStateTransferIndexChooserTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/FullStateTransferIndexChooserTest.java
index 7e82ed370b..e7e92d9f2f 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/FullStateTransferIndexChooserTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/FullStateTransferIndexChooserTest.java
@@ -99,20 +99,16 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
         int pkIndexId = indexId(PK_INDEX_NAME);
         assertThat(chooseForAddWriteLatest(), contains(pkIndexId));
 
-        createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
-        int registeredIndexId = indexId(REGISTERED_INDEX_NAME);
+        int registeredIndexId = createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
         assertThat(chooseForAddWriteLatest(), contains(pkIndexId, registeredIndexId));
 
-        createSimpleBuildingIndex(BUILDING_INDEX_NAME);
-        int buildingIndexId = indexId(BUILDING_INDEX_NAME);
+        int buildingIndexId = createSimpleBuildingIndex(BUILDING_INDEX_NAME);
         assertThat(chooseForAddWriteLatest(), contains(pkIndexId, registeredIndexId, buildingIndexId));
 
-        createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
-        int availableIndexId = indexId(AVAILABLE_INDEX_NAME);
+        int availableIndexId = createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
         assertThat(chooseForAddWriteLatest(), contains(pkIndexId, registeredIndexId, buildingIndexId, availableIndexId));
 
-        createSimpleStoppingIndex(STOPPING_INDEX_NAME);
-        int stoppingIndexId = indexId(STOPPING_INDEX_NAME);
+        int stoppingIndexId = createSimpleStoppingIndex(STOPPING_INDEX_NAME);
         assertThat(chooseForAddWriteLatest(), contains(pkIndexId, registeredIndexId, buildingIndexId, availableIndexId, stoppingIndexId));
     }
 
@@ -120,10 +116,9 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
     void chooseForAddWriteWithSecondaryAndWithoutReadOnlyAndRegisteredIndexes() {
         HybridTimestamp beginTsBeforeCreateRegisteredIndex = clock.now();
 
-        createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
+        int registeredIndexId = createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
 
         int pkIndexId = indexId(PK_INDEX_NAME);
-        int registeredIndexId = indexId(REGISTERED_INDEX_NAME);
 
         assertThat(chooseForAddWriteLatest(beginTsBeforeCreateRegisteredIndex), contains(pkIndexId));
 
@@ -145,16 +140,13 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
         createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
         assertThat(chooseForAddWriteCommittedLatest(), contains(pkIndexId));
 
-        createSimpleBuildingIndex(BUILDING_INDEX_NAME);
-        int buildingIndexId = indexId(BUILDING_INDEX_NAME);
+        int buildingIndexId = createSimpleBuildingIndex(BUILDING_INDEX_NAME);
         assertThat(chooseForAddWriteCommittedLatest(), contains(pkIndexId, buildingIndexId));
 
-        createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
-        int availableIndexId = indexId(AVAILABLE_INDEX_NAME);
+        int availableIndexId = createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
         assertThat(chooseForAddWriteCommittedLatest(), contains(pkIndexId, buildingIndexId, availableIndexId));
 
-        createSimpleStoppingIndex(STOPPING_INDEX_NAME);
-        int stoppingIndexId = indexId(STOPPING_INDEX_NAME);
+        int stoppingIndexId = createSimpleStoppingIndex(STOPPING_INDEX_NAME);
         assertThat(chooseForAddWriteCommittedLatest(), contains(pkIndexId, buildingIndexId, availableIndexId, stoppingIndexId));
     }
 
@@ -166,16 +158,15 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
 
         HybridTimestamp commitTsBeforeStoppingIndex = clock.now();
 
-        createSimpleStoppingIndex(READ_ONLY_INDEX_NAME);
+        int readOnlyIndexId = createSimpleStoppingIndex(READ_ONLY_INDEX_NAME);
 
         HybridTimestamp commitTsOnStoppingIndex = latestCatalogVersionActivationTs();
 
         int pkIndexId = indexId(PK_INDEX_NAME);
-        int readOnlyIndexId = indexId(READ_ONLY_INDEX_NAME);
 
         dropIndex(REGISTERED_INDEX_NAME);
         dropIndex(BUILDING_INDEX_NAME);
-        removeIndex(READ_ONLY_INDEX_NAME);
+        removeIndex(readOnlyIndexId);
 
         if (recovery) {
             recoverIndexChooser();
@@ -194,16 +185,15 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
 
         HybridTimestamp beginTsBeforeStoppingIndex = clock.now();
 
-        createSimpleStoppingIndex(READ_ONLY_INDEX_NAME);
+        int readOnlyIndexId = createSimpleStoppingIndex(READ_ONLY_INDEX_NAME);
 
         HybridTimestamp beginTsOnStoppingIndex = latestCatalogVersionActivationTs();
 
         int pkIndexId = indexId(PK_INDEX_NAME);
-        int readOnlyIndexId = indexId(READ_ONLY_INDEX_NAME);
 
         dropIndex(REGISTERED_INDEX_NAME);
         dropIndex(BUILDING_INDEX_NAME);
-        removeIndex(READ_ONLY_INDEX_NAME);
+        removeIndex(readOnlyIndexId);
 
         if (recovery) {
             recoverIndexChooser();
@@ -221,12 +211,10 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
 
         createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
         createSimpleBuildingIndex(BUILDING_INDEX_NAME);
-        createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
-        createSimpleStoppingIndex(STOPPING_INDEX_NAME);
+        int availableIndexId = createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
+        int stoppingIndexId = createSimpleStoppingIndex(STOPPING_INDEX_NAME);
 
         int pkIndexId = indexId(PK_INDEX_NAME);
-        int availableIndexId = indexId(AVAILABLE_INDEX_NAME);
-        int stoppingIndexId = indexId(STOPPING_INDEX_NAME);
 
         int tableId = tableId(TABLE_NAME);
 
@@ -258,12 +246,10 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
 
         createSimpleRegisteredIndex(REGISTERED_INDEX_NAME);
         createSimpleBuildingIndex(BUILDING_INDEX_NAME);
-        createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
-        createSimpleStoppingIndex(STOPPING_INDEX_NAME);
+        int availableIndexId = createSimpleAvailableIndex(AVAILABLE_INDEX_NAME);
+        int stoppingIndexId = createSimpleStoppingIndex(STOPPING_INDEX_NAME);
 
         int pkIndexId = indexId(PK_INDEX_NAME);
-        int availableIndexId = indexId(AVAILABLE_INDEX_NAME);
-        int stoppingIndexId = indexId(STOPPING_INDEX_NAME);
 
         int tableId = tableId(TABLE_NAME);
 
@@ -312,27 +298,38 @@ public class FullStateTransferIndexChooserTest extends BaseIgniteAbstractTest {
         return chooseForAddWriteLatest(HybridTimestamp.MAX_VALUE);
     }
 
-    private void createSimpleRegisteredIndex(String indexName) {
+    private int createSimpleRegisteredIndex(String indexName) {
         createSimpleHashIndex(catalogManager, TABLE_NAME, indexName);
+
+        return indexId(indexName);
     }
 
-    private void createSimpleBuildingIndex(String indexName) {
-        createSimpleHashIndex(catalogManager, TABLE_NAME, indexName);
-        startBuildingIndex(catalogManager, indexId(indexName));
+    private int createSimpleBuildingIndex(String indexName) {
+        int indexId = createSimpleRegisteredIndex(indexName);
+
+        startBuildingIndex(catalogManager, indexId);
+
+        return indexId;
     }
 
-    private void createSimpleAvailableIndex(String indexName) {
-        createSimpleBuildingIndex(indexName);
-        makeIndexAvailable(catalogManager, indexId(indexName));
+    private int createSimpleAvailableIndex(String indexName) {
+        int indexId = createSimpleBuildingIndex(indexName);
+
+        makeIndexAvailable(catalogManager, indexId);
+
+        return indexId;
     }
 
-    private void createSimpleStoppingIndex(String indexName) {
-        createSimpleAvailableIndex(indexName);
+    private int createSimpleStoppingIndex(String indexName) {
+        int indexId = createSimpleAvailableIndex(indexName);
+
         dropIndex(indexName);
+
+        return indexId;
     }
 
-    private void removeIndex(String indexName) {
-        TableTestUtils.removeIndex(catalogManager, indexName);
+    private void removeIndex(int indexId) {
+        TableTestUtils.removeIndex(catalogManager, indexId);
     }
 
     private void dropIndex(String indexName) {
diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java
index dc4588069d..0047b57220 100644
--- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java
+++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java
@@ -118,11 +118,9 @@ public class TableTestUtils {
      * Removes index from the catalog.
      *
      * @param catalogManager Catalog manager.
-     * @param indexName Index name.
+     * @param indexId Index ID.
      */
-    public static void removeIndex(CatalogManager catalogManager, String indexName) {
-        int indexId = getIndexIdStrict(catalogManager, indexName, Long.MAX_VALUE);
-
+    public static void removeIndex(CatalogManager catalogManager, int indexId) {
         assertThat(
                 catalogManager.execute(RemoveIndexCommand.builder().indexId(indexId).build()),
                 willCompleteSuccessfully()
@@ -259,7 +257,7 @@ public class TableTestUtils {
      * @param timestamp Timestamp.
      */
     public static @Nullable CatalogIndexDescriptor getIndex(CatalogService catalogService, String indexName, long timestamp) {
-        return catalogService.index(indexName, timestamp);
+        return catalogService.aliveIndex(indexName, timestamp);
     }
 
     /**
@@ -271,7 +269,7 @@ public class TableTestUtils {
      * @throws AssertionError If table descriptor is absent.
      */
     public static CatalogIndexDescriptor getIndexStrict(CatalogService catalogService, String indexName, long timestamp) {
-        CatalogIndexDescriptor index = catalogService.index(indexName, timestamp);
+        CatalogIndexDescriptor index = catalogService.aliveIndex(indexName, timestamp);
 
         assertNotNull(index, "indexName=" + indexName + ", timestamp=" + timestamp);