You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2022/09/22 13:47:49 UTC

[ignite-3] branch main updated: IGNITE-17562 Sql. Implement cascade drop for indexes (#1110)

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

amashenkov 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 976020390b IGNITE-17562 Sql. Implement cascade drop for indexes (#1110)
976020390b is described below

commit 976020390b51f4f20d7847a601561ceaeb5fe1e2
Author: Evgeniy Stanilovskiy <st...@gmail.com>
AuthorDate: Thu Sep 22 16:47:41 2022 +0300

    IGNITE-17562 Sql. Implement cascade drop for indexes (#1110)
---
 .../apache/ignite/internal/index/IndexManager.java |  70 ++++-----
 .../ignite/internal/index/IndexManagerTest.java    | 163 +++++++++------------
 .../internal/sql/api/ItSqlAsynchronousApiTest.java |  14 +-
 .../internal/sql/api/ItSqlSynchronousApiTest.java  |  14 +-
 .../internal/sql/engine/ItIndexSpoolTest.java      |   6 -
 .../configuration/IndexValidatorImplTest.java      |   4 +
 .../internal/table/distributed/TableManager.java   |  27 ++--
 7 files changed, 139 insertions(+), 159 deletions(-)

diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
index 6359a38627..9236d172d4 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
@@ -34,12 +34,13 @@ import org.apache.ignite.configuration.notifications.ConfigurationNotificationEv
 import org.apache.ignite.configuration.schemas.table.HashIndexView;
 import org.apache.ignite.configuration.schemas.table.IndexColumnView;
 import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
 import org.apache.ignite.configuration.schemas.table.TableIndexChange;
 import org.apache.ignite.configuration.schemas.table.TableIndexConfiguration;
 import org.apache.ignite.configuration.schemas.table.TableIndexView;
 import org.apache.ignite.configuration.schemas.table.TableView;
 import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
-import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
+import org.apache.ignite.internal.configuration.schema.ExtendedTableConfiguration;
 import org.apache.ignite.internal.index.event.IndexEvent;
 import org.apache.ignite.internal.index.event.IndexEventParameters;
 import org.apache.ignite.internal.logger.IgniteLogger;
@@ -148,20 +149,22 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
             tablesCfg.indexes().change(indexListChange -> {
                 idxExist.set(false);
 
-                UUID tableId;
-
-                try {
-                    tableId = ConfigurationUtil.internalId(tablesCfg.tables().value(), canonicalName);
-                } catch (IllegalArgumentException e) {
-                    throw new TableNotFoundException(canonicalName);
-                }
-
                 if (indexListChange.get(canonicalIndexName) != null) {
                     idxExist.set(true);
 
                     throw new IndexAlreadyExistsException(canonicalIndexName);
                 }
 
+                TableConfiguration tableCfg = tablesCfg.tables().get(canonicalName);
+
+                if (tableCfg == null) {
+                    throw new TableNotFoundException(canonicalName);
+                }
+
+                ExtendedTableConfiguration exTableCfg = ((ExtendedTableConfiguration) tableCfg);
+
+                final UUID tableId = exTableCfg.id().value();
+
                 Consumer<TableIndexChange> chg = indexChange.andThen(c -> c.changeTableId(tableId));
 
                 indexListChange.create(canonicalIndexName, chg);
@@ -175,7 +178,7 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
                     } else {
                         future.completeExceptionally(th);
                     }
-                } else if (!future.isDone()) {
+                } else {
                     TableIndexConfiguration idxCfg = tablesCfg.indexes().get(canonicalIndexName);
 
                     if (idxCfg != null && idxCfg.value() != null) {
@@ -225,42 +228,43 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
         try {
             validateName(indexName);
 
-            String canonicalName = parseCanonicalName(canonicalName(schemaName, indexName));
-
-            TableIndexConfiguration idxCfg = tablesCfg.indexes().get(canonicalName);
-
-            if (idxCfg == null) {
-                return  failIfNotExists
-                        ? CompletableFuture.failedFuture(new IndexNotFoundException(canonicalName))
-                        : CompletableFuture.completedFuture(false);
-            }
+            final String canonicalName = parseCanonicalName(canonicalName(schemaName, indexName));
 
             final CompletableFuture<Boolean> future = new CompletableFuture<>();
 
-            final UUID tableId = idxCfg.tableId().value();
+            // Check index existence flag, avoid usage of hasCause + IndexAlreadyExistsException.
+            AtomicBoolean idxOrTblNotExist = new AtomicBoolean(false);
 
             tablesCfg.indexes().change(indexListChange -> {
-                TableView tableView = getByInternalId(tablesCfg.tables().value(), tableId);
+                idxOrTblNotExist.set(false);
 
-                if (tableView == null) {
-                    if (failIfNotExists) {
-                        throw new TableNotFoundException(canonicalName);
-                    } else {
-                        future.complete(true);
-                    }
-                }
+                TableIndexView idxView = indexListChange.get(canonicalName);
+
+                if (idxView == null) {
+                    idxOrTblNotExist.set(true);
 
-                if (indexListChange.get(canonicalName) == null) {
                     throw new IndexNotFoundException(canonicalName);
-                } else if (tableView != null) {
-                    indexListChange.delete(canonicalName);
                 }
+
+                TableView tableView = getByInternalId(tablesCfg.tables().value(), idxView.tableId());
+
+                if (tableView == null) {
+                    idxOrTblNotExist.set(true);
+
+                    throw new TableNotFoundException(canonicalName);
+                }
+
+                indexListChange.delete(canonicalName);
             }).whenComplete((ignored, th) -> {
                 if (th != null) {
                     LOG.info("Unable to drop index [schema={}, index={}]", th, schemaName, indexName);
 
-                    future.completeExceptionally(th);
-                } else if (!future.isDone()) {
+                    if (!failIfNotExists && idxOrTblNotExist.get()) {
+                        future.complete(false);
+                    } else {
+                        future.completeExceptionally(th);
+                    }
+                } else {
                     LOG.info("Index dropped [schema={}, index={}]", schemaName, indexName);
 
                     future.complete(true);
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 56fa46ef97..a7e69308aa 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
@@ -39,6 +39,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.configuration.NamedConfigurationTree;
 import org.apache.ignite.configuration.schemas.store.UnknownDataStorageConfigurationSchema;
@@ -60,6 +61,8 @@ import org.apache.ignite.configuration.schemas.table.UnlimitedBudgetConfiguratio
 import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.configuration.NamedListConfiguration;
+import org.apache.ignite.internal.configuration.schema.ExtendedTableChange;
+import org.apache.ignite.internal.configuration.schema.ExtendedTableConfigurationSchema;
 import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import org.apache.ignite.internal.configuration.tree.ConverterToMapVisitor;
@@ -73,8 +76,8 @@ import org.apache.ignite.lang.ErrorGroups.Table;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IndexNotFoundException;
 import org.apache.ignite.lang.TableNotFoundException;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
@@ -84,18 +87,27 @@ import org.junit.jupiter.api.extension.ExtendWith;
 @ExtendWith(ConfigurationExtension.class)
 public class IndexManagerTest {
     /** Configuration registry with one table for each test. */
-    private ConfigurationRegistry confRegistry;
+    private static ConfigurationRegistry confRegistry;
+
+    /** Tables configuration. */
+    private static TablesConfiguration tablesConfig;
+
+    /** Index manager. */
+    private static IndexManager indexManager;
+
+    /** Per test unique index name.  */
+    private static AtomicInteger index = new AtomicInteger();
 
     /**
-     * Prepare configuration registry for test.
+     * Common components initialization.
      */
-    @BeforeEach
-    public void createRegistry() {
+    @BeforeAll
+    public static void setUp() throws Exception {
         confRegistry = new ConfigurationRegistry(
                 List.of(TablesConfiguration.KEY),
                 Map.of(IndexValidator.class, Set.of(IndexValidatorImpl.INSTANCE)),
                 new TestConfigurationStorage(DISTRIBUTED),
-                List.of(),
+                List.of(ExtendedTableConfigurationSchema.class),
                 List.of(
                         HashIndexConfigurationSchema.class,
                         SortedIndexConfigurationSchema.class,
@@ -110,27 +122,34 @@ public class IndexManagerTest {
         );
 
         confRegistry.start();
-    }
 
-    @AfterEach
-    void tearDown() throws Exception {
-        confRegistry.stop();
-    }
-
-    @Test
-    void configurationChangedWhenCreateIsInvoked() throws Exception {
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
+        tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
 
-        var indexManager = new IndexManager(tablesConfig);
+        indexManager = new IndexManager(tablesConfig);
         indexManager.start();
 
         tablesConfig.tables().change(tableChange -> tableChange.create("SNAME.TNAME", chg -> {
-            chg.changeColumns(cols ->
-                    cols.create("c1", col -> col.changeType(t -> t.changeType("STRING")))
-                            .create("c2", col -> col.changeType(t -> t.changeType("STRING"))));
+            chg.changeColumns(cols -> cols
+                    .create("c1", col -> col.changeType(t -> t.changeType("STRING")))
+                    .create("c2", col -> col.changeType(t -> t.changeType("STRING"))));
 
             chg.changePrimaryKey(pk -> pk.changeColumns("c1").changeColocationColumns("c1"));
+
+            ((ExtendedTableChange) chg).changeAssignments((byte) 1);
         })).get();
+    }
+
+    /**
+     * Sweep all necessary.
+     */
+    @AfterAll
+    public static void tearDown() throws Exception {
+        confRegistry.stop();
+    }
+
+    @Test
+    void configurationChangedWhenCreateIsInvoked() {
+        String indexTitle = "idx" + index.incrementAndGet();
 
         NamedConfigurationTree<TableConfiguration, TableView, TableChange> cfg0 = tablesConfig.tables();
 
@@ -140,7 +159,7 @@ public class IndexManagerTest {
 
         UUID tableId = ids.get(0);
 
-        indexManager.createIndexAsync("sName", "idx", "tName", true, indexChange -> {
+        indexManager.createIndexAsync("sName", indexTitle, "tName", true, indexChange -> {
             SortedIndexChange sortedIndexChange = indexChange.convert(SortedIndexChange.class);
 
             sortedIndexChange.changeColumns(columns -> {
@@ -152,7 +171,7 @@ public class IndexManagerTest {
         }).join();
 
 
-        String awaitIdxName = parseCanonicalName(canonicalName("sName", "idx"));
+        String awaitIdxName = parseCanonicalName(canonicalName("sName", indexTitle));
 
         var expected = List.of(
                 Map.of(
@@ -178,29 +197,17 @@ public class IndexManagerTest {
 
     @Test
     public void createIndexForNonExistingTable() {
-        var canonicalName = parseCanonicalName(canonicalName("sName", "tName"));
-
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
-
-        var indexManager = new IndexManager(tablesConfig);
-        indexManager.start();
-
         CompletionException completionException = assertThrows(
                 CompletionException.class,
-                () -> indexManager.createIndexAsync("sName", "idx", "tName", true, indexChange -> {/* doesn't matter */}).join()
+                () -> indexManager.createIndexAsync("sName", "idx", "tName_notExist", true, indexChange -> {/* doesn't matter */}).join()
         );
 
         assertTrue(IgniteTestUtils.hasCause(completionException, TableNotFoundException.class,
-                "The table does not exist [name=SNAME.TNAME]"));
+                "The table does not exist [name=SNAME.TNAME_NOTEXIST]"));
     }
 
     @Test
     public void createIndexWithEmptyName() {
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
-
-        var indexManager = new IndexManager(tablesConfig);
-        indexManager.start();
-
         CompletionException completionException = assertThrows(
                 CompletionException.class,
                 () -> indexManager.createIndexAsync("sName", "", "tName", true, indexChange -> {/* doesn't matter */}).join()
@@ -214,22 +221,12 @@ public class IndexManagerTest {
     }
 
     @Test
-    public void createIndexWithEmptyColumnList() throws Exception {
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
-
-        var indexManager = new IndexManager(tablesConfig);
-        indexManager.start();
-
-        tablesConfig.tables().change(tableChange -> tableChange.create("SNAME.TNAME", chg -> {
-            chg.changeColumns(cols ->
-                    cols.create("id", col -> col.changeType(t -> t.changeType("STRING"))));
-
-            chg.changePrimaryKey(pk -> pk.changeColumns("id").changeColocationColumns("id"));
-        })).get();
+    public void createIndexWithEmptyColumnList() {
+        String indexTitle = "idx" + index.incrementAndGet();
 
         CompletionException completionException = assertThrows(
                 CompletionException.class,
-                () -> indexManager.createIndexAsync("sName", "idx", "tName", true,
+                () -> indexManager.createIndexAsync("sName", indexTitle, "tName", true,
                         indexChange -> indexChange.convert(HashIndexChange.class).changeColumnNames()
                                 .changeTableId(UUID.randomUUID())).join()
         );
@@ -244,22 +241,12 @@ public class IndexManagerTest {
     }
 
     @Test
-    public void createIndexForNonExistingColumn() throws Exception {
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
-
-        var indexManager = new IndexManager(tablesConfig);
-        indexManager.start();
-
-        tablesConfig.tables().change(tableChange -> tableChange.create("SNAME.TNAME", chg -> {
-            chg.changeColumns(cols ->
-                    cols.create("id", col -> col.changeType(t -> t.changeType("STRING"))));
-
-            chg.changePrimaryKey(pk -> pk.changeColumns("id").changeColocationColumns("id"));
-        })).get();
+    public void createIndexForNonExistingColumn() {
+        String indexTitle = "idx" + index.incrementAndGet();
 
         CompletionException completionException = assertThrows(
                 CompletionException.class,
-                () -> indexManager.createIndexAsync("sName", "idx", "tName", true,
+                () -> indexManager.createIndexAsync("sName", indexTitle, "tName", true,
                         indexChange ->
                                 indexChange.convert(HashIndexChange.class).changeColumnNames("nonExistingColumn")
                                         .changeTableId(UUID.randomUUID())).join()
@@ -275,47 +262,22 @@ public class IndexManagerTest {
     }
 
     @Test
-    public void dropNonExistingIndex() throws Exception {
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
-
-        var indexManager = new IndexManager(tablesConfig);
-        indexManager.start();
-
-        tablesConfig.tables().change(tableChange -> tableChange.create("SNAME.TNAME", chg -> {
-            chg.changeColumns(cols ->
-                    cols.create("id", col -> col.changeType(t -> t.changeType("STRING"))));
-
-            chg.changePrimaryKey(pk -> pk.changeColumns("id").changeColocationColumns("id"));
-        })).get();
-
+    public void dropNonExistingIndex() {
         CompletionException completionException = assertThrows(
                 CompletionException.class,
                 () -> indexManager.dropIndexAsync("sName", "nonExisting", true).join()
         );
 
-        assertThat(completionException.getCause(), instanceOf(IndexNotFoundException.class));
-        assertThat(
-                ((IndexNotFoundException) completionException.getCause()).code(),
-                equalTo(ErrorGroups.Index.INDEX_NOT_FOUND_ERR)
-        );
+        assertTrue(IgniteTestUtils.hasCause(completionException, IndexNotFoundException.class,
+                "Index 'SNAME.NONEXISTING' does not exist"));
     }
 
     @Test
     @SuppressWarnings("ConstantConditions")
-    public void eventIsFiredWhenIndexCreated() throws Exception {
-        var indexName = "SCHEMA.INDEXNAME";
-
-        TablesConfiguration tablesConfig = confRegistry.getConfiguration(TablesConfiguration.KEY);
-
-        var indexManager = new IndexManager(tablesConfig);
-        indexManager.start();
+    public void eventIsFiredWhenIndexCreated() {
+        String indexTitle = "idx" + index.incrementAndGet();
 
-        tablesConfig.tables().change(tableChange -> tableChange.create("SCHEMA.TNAME", chg -> {
-            chg.changeColumns(cols ->
-                    cols.create("id", col -> col.changeType(t -> t.changeType("STRING"))));
-
-            chg.changePrimaryKey(pk -> pk.changeColumns("id").changeColocationColumns("id"));
-        })).get();
+        var indexName = "SNAME." + indexTitle.toUpperCase();
 
         AtomicReference<IndexEventParameters> holder = new AtomicReference<>();
 
@@ -325,6 +287,12 @@ public class IndexManagerTest {
             return CompletableFuture.completedFuture(true);
         });
 
+        indexManager.listen(IndexEvent.DROP, (param, th) -> {
+            holder.set(param);
+
+            return CompletableFuture.completedFuture(true);
+        });
+
         indexManager.start();
 
         NamedConfigurationTree<TableConfiguration, TableView, TableChange> cfg0 = tablesConfig.tables();
@@ -335,11 +303,11 @@ public class IndexManagerTest {
 
         UUID tableId = ids.get(0);
 
-        indexManager.createIndexAsync("SCHEMA", "indexName", "tName", true, indexChange -> {
+        indexManager.createIndexAsync("SNAME", indexTitle, "tName", true, indexChange -> {
             SortedIndexChange sortedIndexChange = indexChange.convert(SortedIndexChange.class);
 
             sortedIndexChange.changeColumns(columns -> {
-                columns.create("id", columnChange -> columnChange.changeAsc(true));
+                columns.create("c2", columnChange -> columnChange.changeAsc(true));
             });
 
             sortedIndexChange.changeTableId(tableId);
@@ -355,6 +323,11 @@ public class IndexManagerTest {
         assertThat(holder.get().index().id(), equalTo(indexId));
         assertThat(holder.get().index().tableId(), equalTo(tableId));
         assertThat(holder.get().index().name(), equalTo(indexName));
+
+        indexManager.dropIndexAsync("SNAME", indexTitle, true).join();
+
+        assertThat(holder.get(), notNullValue());
+        assertThat(holder.get().indexId(), equalTo(indexId));
     }
 
     private static Object toMap(Object obj) {
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java
index fa9f8ba929..04aca8b555 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java
@@ -58,6 +58,7 @@ import org.apache.ignite.lang.ColumnNotFoundException;
 import org.apache.ignite.lang.ErrorGroups.Sql;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.IndexNotFoundException;
 import org.apache.ignite.lang.TableAlreadyExistsException;
 import org.apache.ignite.lang.TableNotFoundException;
 import org.apache.ignite.sql.BatchedArguments;
@@ -158,7 +159,8 @@ public class ItSqlAsynchronousApiTest extends AbstractBasicIntegrationTest {
         checkDdl(false, ses, "CREATE INDEX IF NOT EXISTS TEST_IDX ON TEST(VAL1)");
 
         checkDdl(true, ses, "DROP INDEX TESt_iDX");
-        checkDdl(true, ses, "CREATE INDEX TEST_IDX ON TEST(VAL0)");
+        checkDdl(true, ses, "CREATE INDEX TEST_IDX1 ON TEST(VAL0)");
+        checkDdl(true, ses, "CREATE INDEX TEST_IDX2 ON TEST(VAL0)");
 
         // DROP COLUMNS
         checkDdl(true, ses, "ALTER TABLE TEST DROP COLUMN VAL1");
@@ -179,9 +181,7 @@ public class ItSqlAsynchronousApiTest extends AbstractBasicIntegrationTest {
 
         // DROP TABLE
         checkDdl(false, ses, "DROP TABLE IF EXISTS NOT_EXISTS_TABLE");
-        //TODO: https://issues.apache.org/jira/browse/IGNITE-17562
-        // Remove this, indices must be dropped together with the table.
-        checkDdl(true, ses, "DROP INDEX TEST_IDX");
+
         checkDdl(true, ses, "DROP TABLE TEST");
         checkError(
                 TableNotFoundException.class,
@@ -191,6 +191,12 @@ public class ItSqlAsynchronousApiTest extends AbstractBasicIntegrationTest {
         );
 
         checkDdl(false, ses, "DROP INDEX IF EXISTS TEST_IDX");
+
+        checkError(
+                IndexNotFoundException.class,
+                "Index 'PUBLIC.TEST_IDX' does not exist.", ses,
+                "DROP INDEX TEST_IDX"
+        );
     }
 
     @Test
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java
index 599cb71c56..f7c0b0c580 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.lang.ColumnNotFoundException;
 import org.apache.ignite.lang.ErrorGroups.Sql;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.IndexNotFoundException;
 import org.apache.ignite.lang.TableAlreadyExistsException;
 import org.apache.ignite.lang.TableNotFoundException;
 import org.apache.ignite.sql.BatchedArguments;
@@ -134,7 +135,8 @@ public class ItSqlSynchronousApiTest extends AbstractBasicIntegrationTest {
         checkDdl(false, ses, "CREATE INDEX IF NOT EXISTS TEST_IDX ON TEST(VAL1)");
 
         checkDdl(true, ses, "DROP INDEX TESt_iDX");
-        checkDdl(true, ses, "CREATE INDEX TEST_IDX ON TEST(VAL0)");
+        checkDdl(true, ses, "CREATE INDEX TEST_IDX1 ON TEST(VAL0)");
+        checkDdl(true, ses, "CREATE INDEX TEST_IDX2 ON TEST(VAL0)");
 
         // DROP COLUMNS
         checkDdl(true, ses, "ALTER TABLE TEST DROP COLUMN VAL1");
@@ -155,9 +157,7 @@ public class ItSqlSynchronousApiTest extends AbstractBasicIntegrationTest {
 
         // DROP TABLE
         checkDdl(false, ses, "DROP TABLE IF EXISTS NOT_EXISTS_TABLE");
-        //TODO: https://issues.apache.org/jira/browse/IGNITE-17562
-        // Remove this, indices must be dropped together with the table.
-        checkDdl(true, ses, "DROP INDEX TEST_IDX");
+
         checkDdl(true, ses, "DROP TABLE TEST");
         checkError(
                 TableNotFoundException.class,
@@ -167,6 +167,12 @@ public class ItSqlSynchronousApiTest extends AbstractBasicIntegrationTest {
         );
 
         checkDdl(false, ses, "DROP INDEX IF EXISTS TEST_IDX");
+
+        checkError(
+                IndexNotFoundException.class,
+                "Index 'PUBLIC.TEST_IDX' does not exist.", ses,
+                "DROP INDEX TEST_IDX"
+        );
     }
 
     @Test
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
index 2b3c58b4af..4abac558df 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
@@ -46,12 +46,6 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
             LOG.info("Start cleanUp()");
         }
 
-        //TODO: https://issues.apache.org/jira/browse/IGNITE-17562
-        // Remove this, indices must be dropped together with the table.
-        CLUSTER_NODES.get(0).tables().tables().stream()
-                .map(Table::name)
-                .forEach(name -> sql("DROP INDEX IF EXISTS " + name + "_JID_IDX"));
-
         CLUSTER_NODES.get(0).tables().tables().stream()
                 .map(Table::name)
                 .forEach(CLUSTER_NODES.get(0).tables()::dropTable);
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/IndexValidatorImplTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/IndexValidatorImplTest.java
index a7a5483171..e4dee1c6d3 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/IndexValidatorImplTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/IndexValidatorImplTest.java
@@ -53,6 +53,8 @@ public class IndexValidatorImplTest extends AbstractTableIndexValidatorTest {
 
         UUID tableId = ids.get(0);
 
+        TablesView oldRoot = tablesCfg.value();
+
         tablesCfg.indexes().change(c -> c.create("schema.idx", idxCng -> idxCng.convert(HashIndexChange.class)
                 .changeColumnNames("ID2").changeTableId(tableId))).get();
 
@@ -60,6 +62,8 @@ public class IndexValidatorImplTest extends AbstractTableIndexValidatorTest {
 
         when(ctxIdx.getNewRoot(any())).thenReturn(view);
 
+        when(ctxIdx.getOldRoot(any())).thenReturn(oldRoot);
+
         validate(IndexValidatorImpl.INSTANCE, mock(IndexValidator.class), ctxIdx, null);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index 89033fa3cb..19e947fd14 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -65,12 +65,10 @@ import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import org.apache.ignite.configuration.ConfigurationChangeException;
 import org.apache.ignite.configuration.ConfigurationProperty;
-import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
 import org.apache.ignite.configuration.schemas.table.TableChange;
 import org.apache.ignite.configuration.schemas.table.TableConfiguration;
-import org.apache.ignite.configuration.schemas.table.TableIndexView;
 import org.apache.ignite.configuration.schemas.table.TableView;
 import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
 import org.apache.ignite.configuration.validation.ConfigurationValidationException;
@@ -1252,27 +1250,22 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
             if (tbl == null) {
                 dropTblFut.completeExceptionally(new TableNotFoundException(name));
             } else {
-                tablesCfg.tables()
-                        .change(change -> {
-                            TableView tableCfg = change.get(name);
+                tablesCfg.change(chg ->
+                        chg.changeTables(tblChg -> {
+                            TableView tableCfg = tblChg.get(name);
 
                             if (tableCfg == null) {
                                 throw new TableNotFoundException(name);
                             }
 
-                            NamedListView<TableIndexView> idxView = tablesCfg.indexes().value();
-
-                            boolean idxFound = idxView.namedListKeys().stream()
-                                    .anyMatch(idx -> idxView.get(idx).tableId().equals(tbl.tableId()));
-
-                            //TODO: https://issues.apache.org/jira/browse/IGNITE-17562
-                            // Let's drop orphaned indices instantly.
-                            if (idxFound) {
-                                throw new IgniteException("Can't drop table with indices.");
-                            }
+                            tblChg.delete(name);
+                        }
+                        ).changeIndexes(idxChg -> {
+                            List<String> indicesNames = tablesCfg.indexes().value().namedListKeys();
 
-                            change.delete(name);
-                        })
+                            indicesNames.stream().filter(idx ->
+                                    tablesCfg.indexes().get(idx).tableId().value().equals(tbl.tableId())).forEach(idxChg::delete);
+                        }))
                         .whenComplete((res, t) -> {
                             if (t != null) {
                                 Throwable ex = getRootCause(t);