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 2021/11/01 15:13:19 UTC

[ignite-3] branch main updated: IGNITE-15414 Schema validation refactoring with configuration validators (#360)

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 97c5197  IGNITE-15414 Schema validation refactoring with configuration validators (#360)
97c5197 is described below

commit 97c5197f0840e358b659aa5d18b158ad4f94b048
Author: Andrew V. Mashenkov <AM...@users.noreply.github.com>
AuthorDate: Mon Nov 1 18:13:16 2021 +0300

    IGNITE-15414 Schema validation refactoring with configuration validators (#360)
---
 .../schemas/table/ColumnConfigurationSchema.java   |   3 +
 .../table/ColumnTypeConfigurationSchema.java       |   9 ++
 ...urationSchema.java => ColumnTypeValidator.java} |  31 ++--
 .../ignite/schema/definition/ColumnType.java       |  25 ++--
 .../apache/ignite/internal/manager/Producer.java   |   2 +-
 .../runner/app/AbstractSchemaChangeTest.java       | 108 ++++++--------
 .../runner/app/ITDynamicTableCreationTest.java     | 147 +++++++++++++++++--
 .../org/apache/ignite/internal/app/IgniteImpl.java |  13 +-
 .../apache/ignite/internal/schema/NativeTypes.java |  79 -----------
 .../apache/ignite/internal/schema/SchemaUtils.java |  20 +--
 .../configuration/ColumnTypeValidatorImpl.java     |  61 ++++++++
 .../SchemaConfigurationConverter.java              |   2 +-
 .../configuration/SchemaDescriptorConverter.java   |   2 +-
 .../ignite/internal/schema/NativeTypeTest.java     |   6 +-
 .../internal/table/distributed/TableManager.java   | 157 +++++++++++----------
 15 files changed, 381 insertions(+), 284 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
index ee38059..7292087 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
@@ -20,6 +20,7 @@ package org.apache.ignite.configuration.schemas.table;
 import org.apache.ignite.configuration.annotation.Config;
 import org.apache.ignite.configuration.annotation.ConfigValue;
 import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Immutable;
 
 /**
  * Configuration for single column in SQL table.
@@ -32,10 +33,12 @@ public class ColumnConfigurationSchema {
 
     /** Column type. */
     @ConfigValue
+    @ColumnTypeValidator
     public ColumnTypeConfigurationSchema type;
 
     /** Nullable flag. */
     @Value
+    @Immutable
     public boolean nullable;
 
     /** Default value. */
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
index 16813bb..e9ec647 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
@@ -19,6 +19,8 @@ package org.apache.ignite.configuration.schemas.table;
 
 import org.apache.ignite.configuration.annotation.Config;
 import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Immutable;
+import org.apache.ignite.configuration.validation.Min;
 
 /**
  * Configuration for SQL table column type.
@@ -27,17 +29,24 @@ import org.apache.ignite.configuration.annotation.Value;
 public class ColumnTypeConfigurationSchema {
     /** Type name. */
     @Value
+    @Immutable
     public String type;
 
     /** Length. */
     @Value(hasDefault = true)
+    @Immutable
+    @Min(0)
     public int length = 0;
 
     /** Precision. */
     @Value(hasDefault = true)
+    @Immutable
+    @Min(0)
     public int precision = 0;
 
     /** Scale. */
     @Value(hasDefault = true)
+    @Immutable
+    @Min(0)
     public int scale = 0;
 }
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeValidator.java
similarity index 63%
copy from modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
copy to modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeValidator.java
index 16813bb..c479274 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeValidator.java
@@ -17,27 +17,20 @@
 
 package org.apache.ignite.configuration.schemas.table;
 
-import org.apache.ignite.configuration.annotation.Config;
-import org.apache.ignite.configuration.annotation.Value;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.PARAMETER;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
 
 /**
- * Configuration for SQL table column type.
+ * Annotation to validate whole table configuration.
+ *
+ * Activate SchemaTableValidatorImpl in configuration engine for {@link TablesConfigurationSchema#tables}.
  */
-@Config
-public class ColumnTypeConfigurationSchema {
-    /** Type name. */
-    @Value
-    public String type;
-
-    /** Length. */
-    @Value(hasDefault = true)
-    public int length = 0;
-
-    /** Precision. */
-    @Value(hasDefault = true)
-    public int precision = 0;
+@Target({ FIELD, PARAMETER })
+@Retention(RUNTIME)
+public @interface ColumnTypeValidator {
 
-    /** Scale. */
-    @Value(hasDefault = true)
-    public int scale = 0;
 }
diff --git a/modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java b/modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java
index eff23e4..853dee1 100644
--- a/modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java
+++ b/modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java
@@ -60,15 +60,6 @@ public class ColumnType {
     /** Timezone-free three-part value representing a year, month, and day. */
     public static final ColumnType DATE = new ColumnType(ColumnTypeSpec.DATE);
 
-    /** String varlen type of unlimited length. */
-    private static final VarLenColumnType UNLIMITED_STRING = stringOf(0);
-
-    /** Blob varlen type of unlimited length. */
-    private static final VarLenColumnType UNLIMITED_BLOB = blobOf(0);
-
-    /** Number type with unlimited precision. */
-    public static final NumberColumnType UNLIMITED_NUMBER = new NumberColumnType(ColumnTypeSpec.NUMBER, NumberColumnType.UNLIMITED_PRECISION);
-
     /**
      * Returns bit mask type.
      *
@@ -85,7 +76,7 @@ public class ColumnType {
      * @return String type.
      */
     public static VarLenColumnType string() {
-        return UNLIMITED_STRING;
+        return VarLenColumnType.UNLIMITED_STRING;
     }
 
     /**
@@ -105,7 +96,7 @@ public class ColumnType {
      * @see #blobOf(int)
      */
     public static VarLenColumnType blobOf() {
-        return UNLIMITED_BLOB;
+        return VarLenColumnType.UNLIMITED_BLOB;
     }
 
     /**
@@ -139,7 +130,7 @@ public class ColumnType {
      * @see #numberOf(int)
      */
     public static NumberColumnType numberOf() {
-        return UNLIMITED_NUMBER;
+        return NumberColumnType.UNLIMITED_NUMBER;
     }
 
     /**
@@ -269,6 +260,12 @@ public class ColumnType {
      * Column type of variable length.
      */
     public static class VarLenColumnType extends ColumnType {
+        /** String varlen type of unlimited length. */
+        private static final VarLenColumnType UNLIMITED_STRING = new VarLenColumnType(ColumnTypeSpec.STRING, 0);
+
+        /** Blob varlen type of unlimited length. */
+        private static final VarLenColumnType UNLIMITED_BLOB = new VarLenColumnType(ColumnTypeSpec.BLOB, 0);
+
         /** Max length. */
         private final int length;
 
@@ -389,8 +386,8 @@ public class ColumnType {
      * Number column type.
      */
     public static class NumberColumnType extends ColumnType {
-        /** Undefined precision. */
-        private static final int UNLIMITED_PRECISION = 0;
+        /** Number type with unlimited precision. */
+        public static final NumberColumnType UNLIMITED_NUMBER = new NumberColumnType(ColumnTypeSpec.NUMBER, 0);
 
         /** Max precision of value. If -1, column has no precision restrictions. */
         private final int precision;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/manager/Producer.java b/modules/core/src/main/java/org/apache/ignite/internal/manager/Producer.java
index 5faba5f..3fea0af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/manager/Producer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/manager/Producer.java
@@ -62,7 +62,7 @@ public abstract class Producer<T extends Event, P extends EventParameters> {
      */
     public void removeListener(T evt, EventListener<P> closure, @Nullable IgniteInternalCheckedException cause) {
         if (listeners.computeIfAbsent(evt, evtKey -> new ConcurrentLinkedQueue<>()).remove(closure))
-            closure.remove(cause == null ? new ListenerRemovedException() : new ListenerRemovedException(cause));
+            closure.remove(cause == null ? new ListenerRemovedException() : cause.getCause() == null ? cause : cause.getCause());
     }
 
     /**
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java
index 4949376..cde325a 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/AbstractSchemaChangeTest.java
@@ -22,11 +22,13 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Consumer;
 import java.util.function.Supplier;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.ITUtils;
-import org.apache.ignite.internal.schema.InvalidTypeException;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -39,7 +41,6 @@ import org.jetbrains.annotations.NotNull;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -127,85 +128,44 @@ abstract class AbstractSchemaChangeTest {
     /**
      * Check unsupported column type change.
      */
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-15056")
     @Test
     public void testChangeColumnType() {
         List<Ignite> grid = startGrid();
 
         createTable(grid);
 
-        Assertions.assertThrows(InvalidTypeException.class, () -> {
-            grid.get(0).tables().alterTable(TABLE,
-                tblChanger -> tblChanger.changeColumns(cols -> {
-                    final String colKey = tblChanger.columns().namedListKeys().stream()
-                        .filter(c -> "valInt".equals(tblChanger.columns().get(c).name()))
-                        .findFirst()
-                        .orElseThrow(() -> {
-                            throw new IllegalStateException("Column not found.");
-                        });
+        assertColumnChangeFailed(grid, "valStr", c -> c.changeType(t -> t.changeType("UNKNOWN_TYPE")));
 
-                    tblChanger.changeColumns(listChanger ->
-                        listChanger.createOrUpdate(colKey, colChanger -> colChanger.changeType(c -> c.changeType("STRING")))
-                    );
-                })
-            );
-        });
-    }
+        assertColumnChangeFailed(grid, "valInt", colChanger -> colChanger.changeType(t -> t.changeType(ColumnType.blobOf().typeSpec().name())));
 
-    /**
-     * Check unsupported column nullability change.
-     */
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-15056")
-    @Test
-    public void testMakeColumnNonNullable() {
-        List<Ignite> grid = startGrid();
+        assertColumnChangeFailed(grid, "valInt", colChanger -> colChanger.changeType(t -> t.changePrecision(10)));
+        assertColumnChangeFailed(grid, "valInt", colChanger -> colChanger.changeType(t -> t.changeScale(10)));
+        assertColumnChangeFailed(grid, "valInt", colChanger -> colChanger.changeType(t -> t.changeLength(1)));
 
-        createTable(grid);
+        assertColumnChangeFailed(grid, "valBigInt", colChanger -> colChanger.changeType(t -> t.changePrecision(-1)));
+        assertColumnChangeFailed(grid, "valBigInt", colChanger -> colChanger.changeType(t -> t.changePrecision(10)));
+        assertColumnChangeFailed(grid, "valBigInt", colChanger -> colChanger.changeType(t -> t.changeScale(2)));
+        assertColumnChangeFailed(grid, "valBigInt", colChanger -> colChanger.changeType(t -> t.changeLength(10)));
 
-        Assertions.assertThrows(InvalidTypeException.class, () -> {
-            grid.get(0).tables().alterTable(TABLE,
-                tblChanger -> tblChanger.changeColumns(cols -> {
-                    final String colKey = tblChanger.columns().namedListKeys().stream()
-                        .filter(c -> "valInt".equals(tblChanger.columns().get(c).name()))
-                        .findFirst()
-                        .orElseThrow(() -> {
-                            throw new IllegalStateException("Column not found.");
-                        });
-
-                    tblChanger.changeColumns(listChanger ->
-                        listChanger.createOrUpdate(colKey, colChanger -> colChanger.changeNullable(false))
-                    );
-                })
-            );
-        });
+        assertColumnChangeFailed(grid, "valDecimal", colChanger -> colChanger.changeType(c -> c.changePrecision(-1)));
+        assertColumnChangeFailed(grid, "valDecimal", colChanger -> colChanger.changeType(c -> c.changePrecision(0)));
+        assertColumnChangeFailed(grid, "valDecimal", colChanger -> colChanger.changeType(c -> c.changeScale(-2)));
+        assertColumnChangeFailed(grid, "valDecimal", colChanger -> colChanger.changeType(c -> c.changePrecision(10)));
+        assertColumnChangeFailed(grid, "valDecimal", colChanger -> colChanger.changeType(c -> c.changeScale(2)));
+        assertColumnChangeFailed(grid, "valDecimal", colChanger -> colChanger.changeType(c -> c.changeLength(10)));
     }
 
     /**
      * Check unsupported nullability change.
      */
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-15056")
     @Test
-    public void testMakeColumnsNullable() {
+    public void testChangeColumnsNullability() {
         List<Ignite> grid = startGrid();
 
         createTable(grid);
 
-        Assertions.assertThrows(InvalidTypeException.class, () -> {
-            grid.get(0).tables().alterTable(TABLE,
-                tblChanger -> tblChanger.changeColumns(cols -> {
-                    final String colKey = tblChanger.columns().namedListKeys().stream()
-                        .filter(c -> "valStr".equals(tblChanger.columns().get(c).name()))
-                        .findFirst()
-                        .orElseThrow(() -> {
-                            throw new IllegalStateException("Column not found.");
-                        });
-
-                    tblChanger.changeColumns(listChanger ->
-                        listChanger.createOrUpdate(colKey, colChanger -> colChanger.changeNullable(true))
-                    );
-                })
-            );
-        });
+        assertColumnChangeFailed(grid, "valStr", colChanger -> colChanger.changeNullable(true));
+        assertColumnChangeFailed(grid, "valInt", colChanger -> colChanger.changeNullable(false));
     }
 
     /**
@@ -227,6 +187,9 @@ abstract class AbstractSchemaChangeTest {
         TableDefinition schTbl1 = SchemaBuilders.tableBuilder("PUBLIC", "tbl1").columns(
             SchemaBuilders.column("key", ColumnType.INT64).asNonNull().build(),
             SchemaBuilders.column("valInt", ColumnType.INT32).asNullable().build(),
+            SchemaBuilders.column("valBlob", ColumnType.blobOf()).asNullable().build(),
+            SchemaBuilders.column("valDecimal", ColumnType.decimalOf()).asNullable().build(),
+            SchemaBuilders.column("valBigInt", ColumnType.numberOf()).asNullable().build(),
             SchemaBuilders.column("valStr", ColumnType.string()).withDefaultValueExpression("default").build()
         ).withPrimaryKey("key").build();
 
@@ -313,6 +276,29 @@ abstract class AbstractSchemaChangeTest {
     }
 
     /**
+     * Ensure configuration validation failed.
+     *
+     * @param grid Grid.
+     * @param colName Column to change.
+     * @param colChanger Column configuration changer.
+     */
+    private void assertColumnChangeFailed(List<Ignite> grid, String colName, Consumer<ColumnChange> colChanger) {
+        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
+            grid.get(0).tables().alterTable(TABLE,
+                tblChanger -> tblChanger.changeColumns(cols -> {
+                    final String colKey = tblChanger.columns().namedListKeys().stream()
+                        .filter(c -> colName.equals(tblChanger.columns().get(c).name()))
+                        .findFirst()
+                        .orElseGet(() -> Assertions.fail("Column not found."));
+
+                    tblChanger.changeColumns(listChanger -> listChanger.createOrUpdate(colKey, colChanger)
+                    );
+                })
+            );
+        });
+    }
+
+    /**
      * @param expectedType Expected cause type.
      * @param executable Executable that throws exception.
      */
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java
index d4f3d1b..81aad09 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ITDynamicTableCreationTest.java
@@ -23,9 +23,12 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
-
+import java.util.concurrent.CompletionException;
+import java.util.function.Consumer;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.ITUtils;
 import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
 import org.apache.ignite.internal.testframework.WorkDirectory;
@@ -38,8 +41,11 @@ import org.apache.ignite.table.KeyValueView;
 import org.apache.ignite.table.RecordView;
 import org.apache.ignite.table.Table;
 import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.NotNull;
 import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -55,7 +61,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
 @ExtendWith(WorkDirectoryExtension.class)
 class ITDynamicTableCreationTest {
     /** Network ports of the test nodes. */
-    private static final int[] PORTS = { 3344, 3345, 3346 };
+    private static final int[] PORTS = {3344, 3345, 3346};
 
     /** Nodes bootstrap configuration. */
     private final Map<String, String> nodesBootstrapCfg = new LinkedHashMap<>();
@@ -121,16 +127,25 @@ class ITDynamicTableCreationTest {
     }
 
     /**
-     * Check dynamic table creation.
+     * @return Grid nodes.
      */
-    @Test
-    void testDynamicSimpleTableCreation() {
+    @NotNull protected List<Ignite> startGrid() {
         nodesBootstrapCfg.forEach((nodeName, configStr) ->
             clusterNodes.add(IgnitionManager.start(nodeName, configStr, workDir.resolve(nodeName)))
         );
 
         assertEquals(3, clusterNodes.size());
 
+        return clusterNodes;
+    }
+
+    /**
+     * Check dynamic table creation.
+     */
+    @Test
+    void testDynamicSimpleTableCreation() {
+        startGrid();
+
         // Create table on node 0.
         TableDefinition schTbl1 = SchemaBuilders.tableBuilder("PUBLIC", "tbl1").columns(
             SchemaBuilders.column("key", ColumnType.INT64).asNonNull().build(),
@@ -178,11 +193,7 @@ class ITDynamicTableCreationTest {
      */
     @Test
     void testDynamicTableCreation() {
-        nodesBootstrapCfg.forEach((nodeName, configStr) ->
-            clusterNodes.add(IgnitionManager.start(nodeName, configStr, workDir.resolve(nodeName)))
-        );
-
-        assertEquals(3, clusterNodes.size());
+        startGrid();
 
         // Create table on node 0.
         TableDefinition scmTbl1 = SchemaBuilders.tableBuilder("PUBLIC", "tbl1").columns(
@@ -253,4 +264,120 @@ class ITDynamicTableCreationTest {
         assertEquals(7373, (Integer)kvView2.get(keyTuple2).value("valInt"));
         assertNull(kvView2.get(keyTuple2).value("valNull"));
     }
+
+    /**
+     * Check unsupported column type change.
+     */
+    @Test
+    public void testChangeColumnType() {
+        List<Ignite> grid = startGrid();
+
+        assertTableCreationFailed(grid, c -> c.changeType(t -> t.changeType("UNKNOWN_TYPE")));
+
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("STRING").changeLength(-1)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("BYTES").changeLength(-1)));
+
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("INT32").changePrecision(-1)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("INT32").changeScale(-1)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("BYTES").changeLength(-1)));
+
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("NUMBER").changePrecision(-1)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("NUMBER").changeScale(-2)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("BYTES").changeLength(-1)));
+
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(c -> c.changeType("DECIMAL").changePrecision(-1)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(c -> c.changeType("DECIMAL").changePrecision(0)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(c -> c.changeType("DECIMAL").changeScale(-2)));
+        assertTableCreationFailed(grid, colChanger -> colChanger.changeType(t -> t.changeType("BYTES").changeLength(-1)));
+    }
+
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-15747")
+    @Test
+    void testMissedPK() {
+        List<Ignite> grid = startGrid();
+
+        // Missed PK.
+        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
+            try {
+                grid.get(0).tables().createTable(
+                    "PUBLIC.tbl1",
+                    tblChanger -> tblChanger.changeName("PUBLIC.tbl1")
+                        .changeColumns(cols -> {
+                            cols.create("0", col -> col.changeName("key").changeType(t -> t.changeType("INT64")).changeNullable(false));
+                            cols.create("1", col -> col.changeName("val").changeNullable(true).changeType(t -> t.changeType("INT32")));
+                        })
+                        .changeReplicas(1)
+                        .changePartitions(10)
+                );
+            } catch (CompletionException ex) {
+                throw ex.getCause();
+            }
+        });
+
+        //Missed affinity cols.
+        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
+            try {
+                grid.get(0).tables().createTable(
+                    "PUBLIC.tbl1",
+                    tblChanger -> tblChanger.changeName("PUBLIC.tbl1")
+                        .changeColumns(cols -> {
+                            cols.create("0", col -> col.changeName("key").changeType(t -> t.changeType("INT64")).changeNullable(false));
+                            cols.create("1", col -> col.changeName("val").changeNullable(true).changeType(t -> t.changeType("INT32")));
+                        })
+                        .changePrimaryKey(pk -> pk.changeColumns("key"))
+                        .changeReplicas(1)
+                        .changePartitions(10)
+                );
+            } catch (CompletionException ex) {
+                throw ex.getCause();
+            }
+        });
+
+        //Missed key cols.
+        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
+            try {
+                grid.get(0).tables().createTable(
+                    "PUBLIC.tbl1",
+                    tblChanger -> tblChanger.changeName("PUBLIC.tbl1")
+                        .changeColumns(cols -> {
+                            cols.create("0", col -> col.changeName("key").changeType(t -> t.changeType("INT64")).changeNullable(false));
+                            cols.create("1", col -> col.changeName("val").changeNullable(true).changeType(t -> t.changeType("INT32")));
+                        })
+                        .changePrimaryKey(pk -> pk.changeAffinityColumns("key"))
+                        .changeReplicas(1)
+                        .changePartitions(10)
+                );
+            } catch (CompletionException ex) {
+                throw ex.getCause();
+            }
+        });
+    }
+
+    /**
+     * Ensure configuration validation failed.
+     *
+     * @param grid Grid.
+     * @param colChanger Column configuration changer.
+     */
+    private void assertTableCreationFailed(List<Ignite> grid, Consumer<ColumnChange> colChanger) {
+        Assertions.assertThrows(ConfigurationValidationException.class, () -> {
+            try {
+                grid.get(0).tables().createTable(
+                    "PUBLIC.tbl1",
+                    tblChanger -> tblChanger.changeName("PUBLIC.tbl1")
+                        .changeColumns(cols -> {
+                            cols.create("0", col -> col.changeName("key").changeType(t -> t.changeType("INT64")).changeNullable(false));
+
+                            cols.create("1", col -> colChanger.accept(col.changeName("val").changeNullable(true)));
+                        })
+                        .changePrimaryKey(pk -> pk.changeColumns("key").changeAffinityColumns("key"))
+                        .changeReplicas(1)
+                        .changePartitions(10)
+                );
+            }
+            catch (CompletionException ex) {
+                throw ex.getCause();
+            }
+        });
+    }
 }
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index ced4cce..ec3f101 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -22,7 +22,6 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.ListIterator;
@@ -39,6 +38,7 @@ import org.apache.ignite.configuration.schemas.rest.RestConfiguration;
 import org.apache.ignite.configuration.schemas.runner.ClusterConfiguration;
 import org.apache.ignite.configuration.schemas.runner.NodeConfiguration;
 import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeValidator;
 import org.apache.ignite.configuration.schemas.table.TableValidator;
 import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
 import org.apache.ignite.internal.baseline.BaselineManager;
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.metastorage.server.persistence.RocksDBKeyValue
 import org.apache.ignite.internal.processors.query.calcite.QueryProcessor;
 import org.apache.ignite.internal.processors.query.calcite.SqlQueryProcessor;
 import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.schema.configuration.ColumnTypeValidatorImpl;
 import org.apache.ignite.internal.schema.configuration.TableValidatorImpl;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.vault.VaultManager;
@@ -148,7 +149,7 @@ public class IgniteImpl implements Ignite {
         vaultMgr = createVault(workDir);
 
         nodeCfgMgr = new ConfigurationManager(
-            Arrays.asList(
+            List.of(
                 NetworkConfiguration.KEY,
                 NodeConfiguration.KEY,
                 RestConfiguration.KEY,
@@ -178,14 +179,16 @@ public class IgniteImpl implements Ignite {
             new RocksDBKeyValueStorage(workDir.resolve(METASTORAGE_DB_PATH))
         );
 
-        // TODO: IGNITE-15414 Schema validation refactoring with configuration validators.
         clusterCfgMgr = new ConfigurationManager(
-            Arrays.asList(
+            List.of(
                 ClusterConfiguration.KEY,
                 TablesConfiguration.KEY,
                 DataStorageConfiguration.KEY
             ),
-            Map.of(TableValidator.class, Set.of(TableValidatorImpl.INSTANCE)),
+            Map.of(
+                TableValidator.class, Set.of(TableValidatorImpl.INSTANCE),
+                ColumnTypeValidator.class, Set.of(ColumnTypeValidatorImpl.INSTANCE)
+            ),
             new DistributedConfigurationStorage(metaStorageMgr, vaultMgr),
             Collections.singletonList(ExtendedTableConfigurationSchema.class),
             List.of()
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java
index 9a2f8a2..f9f5ff2 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java
@@ -238,83 +238,4 @@ public class NativeTypes {
                 return null;
         }
     }
-
-    /**
-     * Maps column type to native type.
-     *
-     * @param type Column type.
-     * @return Native type.
-     */
-    public static NativeType from(ColumnType type) {
-        switch (type.typeSpec()) {
-            case INT8:
-                return INT8;
-
-            case INT16:
-                return INT16;
-
-            case INT32:
-                return INT32;
-
-            case INT64:
-                return INT64;
-
-            case UINT8:
-            case UINT16:
-            case UINT32:
-            case UINT64:
-                throw new UnsupportedOperationException("Unsigned types are not supported yet.");
-
-            case FLOAT:
-                return FLOAT;
-
-            case DOUBLE:
-                return DOUBLE;
-
-            case DECIMAL: {
-                ColumnType.DecimalColumnType numType = (ColumnType.DecimalColumnType)type;
-
-                return new DecimalNativeType(numType.precision(), numType.scale());
-            }
-            case UUID:
-                return UUID;
-
-            case DATE:
-                return DATE;
-
-            case TIME:
-                return time(((ColumnType.TemporalColumnType)type).precision());
-
-            case DATETIME:
-                return datetime(((ColumnType.TemporalColumnType)type).precision());
-
-            case TIMESTAMP:
-                return timestamp(((ColumnType.TemporalColumnType)type).precision());
-
-            case BITMASK:
-                return new BitmaskNativeType(((ColumnType.VarLenColumnType)type).length());
-
-            case STRING:
-                return new VarlenNativeType(
-                    NativeTypeSpec.STRING,
-                    ((ColumnType.VarLenColumnType)type).length() > 0 ?
-                        ((ColumnType.VarLenColumnType)type).length() : Integer.MAX_VALUE
-                );
-
-            case BLOB:
-                return new VarlenNativeType(
-                    NativeTypeSpec.BYTES,
-                    ((ColumnType.VarLenColumnType)type).length() > 0 ?
-                        ((ColumnType.VarLenColumnType)type).length() : Integer.MAX_VALUE
-                );
-
-            case NUMBER: {
-                ColumnType.NumberColumnType numberType = (ColumnType.NumberColumnType)type;
-
-                return new NumberNativeType(numberType.precision());
-            }
-            default:
-                throw new InvalidTypeException("Unexpected type " + type);
-        }
-    }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaUtils.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaUtils.java
index 9572062..52b3f46 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaUtils.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaUtils.java
@@ -79,24 +79,6 @@ public class SchemaUtils {
                 final Column newCol = newDesc.column(newColView.name());
                 final Column oldCol = oldDesc.column(oldColView.name());
 
-                // TODO: IGNITE-15414 Assertion just in case, proper validation should be implemented with the help of
-                // TODO: configuration validators.
-                assert newCol.type().equals(oldCol.type()) :
-                    LoggerMessageHelper.format(
-                        "Column types doesn't match [column={}, oldType={}, newType={}",
-                        oldCol.name(),
-                        oldCol.type(),
-                        newCol.type()
-                    );
-
-                assert newCol.nullable() == oldCol.nullable() :
-                    LoggerMessageHelper.format(
-                        "Column nullable properties doesn't match [column={}, oldNullable={}, newNullable={}",
-                        oldCol.name(),
-                        oldCol.nullable(),
-                        newCol.nullable()
-                    );
-
                 if (newCol.schemaIndex() == oldCol.schemaIndex())
                     continue;
 
@@ -113,7 +95,7 @@ public class SchemaUtils {
             .filter(c -> oldDesc.isKeyColumn(c.schemaIndex()))
             .findAny();
 
-        // TODO: IGNITE-15414 Assertion just in case, proper validation should be implemented with the help of
+        // TODO: IGNITE-15774 Assertion just in case, proper validation should be implemented with the help of
         // TODO: configuration validators.
         assert !droppedKeyCol.isPresent() :
             LoggerMessageHelper.format(
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ColumnTypeValidatorImpl.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ColumnTypeValidatorImpl.java
new file mode 100644
index 0000000..27157e0
--- /dev/null
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ColumnTypeValidatorImpl.java
@@ -0,0 +1,61 @@
+/*
+ * 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.schema.configuration;
+
+import java.util.Objects;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeValidator;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+
+/**
+ * Column definition validator implementation validates column changes.
+ */
+public class ColumnTypeValidatorImpl implements Validator<ColumnTypeValidator, ColumnTypeView> {
+    /** Static instance. */
+    public static final ColumnTypeValidatorImpl INSTANCE = new ColumnTypeValidatorImpl();
+
+    /** {@inheritDoc} */
+    @Override public void validate(ColumnTypeValidator annotation, ValidationContext<ColumnTypeView> ctx) {
+        ColumnTypeView newType = ctx.getNewValue();
+        ColumnTypeView oldType = ctx.getOldValue();
+
+        try {
+            SchemaConfigurationConverter.convert(newType);
+        } catch (IllegalArgumentException ex) {
+            ctx.addIssue(new ValidationIssue(ctx.currentKey() + ": " + ex.getMessage()));
+
+            return;
+        }
+
+        if (oldType == null)
+            return; // Nothing to do.
+
+        if (!Objects.deepEquals(newType.type(), oldType.type()) ||
+                newType.precision() != oldType.precision() ||
+                newType.scale() != oldType.scale() ||
+                newType.length() != oldType.length())
+            ctx.addIssue(new ValidationIssue("Unsupported column type change: " + ctx.currentKey()));
+
+    }
+
+    /** Private constructor. */
+    private ColumnTypeValidatorImpl() {
+    }
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
index 0f85414..ee2f2d8 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
@@ -332,7 +332,7 @@ public class SchemaConfigurationConverter {
                     return ColumnType.decimalOf(prec, scale);
 
                 case "NUMBER":
-                    return ColumnType.numberOf(colTypeView.precision());
+                    return colTypeView.precision() == 0 ? ColumnType.numberOf() : ColumnType.numberOf(colTypeView.precision());
 
                 case "TIME":
                     return ColumnType.time(colTypeView.precision());
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
index 9f6dcbb..59215c0 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
@@ -58,7 +58,7 @@ public class SchemaDescriptorConverter {
      * @param colType ColumnType.
      * @return NativeType.
      */
-    private static NativeType convert(ColumnType colType) {
+    public static NativeType convert(ColumnType colType) {
         assert colType != null;
 
         ColumnType.ColumnTypeSpec type = colType.typeSpec();
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java
index 67c7916..b71149e 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.schema;
 
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
 import org.apache.ignite.schema.definition.ColumnType;
 import org.junit.jupiter.api.Test;
 
@@ -34,7 +35,6 @@ import static org.apache.ignite.internal.schema.NativeTypes.bitmaskOf;
 import static org.apache.ignite.internal.schema.NativeTypes.blobOf;
 import static org.apache.ignite.internal.schema.NativeTypes.datetime;
 import static org.apache.ignite.internal.schema.NativeTypes.decimalOf;
-import static org.apache.ignite.internal.schema.NativeTypes.from;
 import static org.apache.ignite.internal.schema.NativeTypes.numberOf;
 import static org.apache.ignite.internal.schema.NativeTypes.stringOf;
 import static org.apache.ignite.internal.schema.NativeTypes.time;
@@ -184,4 +184,8 @@ public class NativeTypeTest {
             assertEquals(timestamp(i), from(ColumnType.timestamp(i)));
         }
     }
+
+    private NativeType from(ColumnType colType) {
+        return SchemaDescriptorConverter.convert(colType);
+    }
 }
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 a7dded5..fc2da6a 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
@@ -46,6 +46,7 @@ import org.apache.ignite.configuration.schemas.table.TableChange;
 import org.apache.ignite.configuration.schemas.table.TableConfiguration;
 import org.apache.ignite.configuration.schemas.table.TableView;
 import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.affinity.AffinityUtils;
 import org.apache.ignite.internal.baseline.BaselineManager;
 import org.apache.ignite.internal.configuration.schema.ExtendedTableChange;
@@ -584,14 +585,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public Table createTable(String name, Consumer<TableChange> tableInitChange) {
-        if (!busyLock.enterBusy())
-            throw new IgniteException(new NodeStoppingException());
-        try {
-            return createTableAsync(name, tableInitChange).join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        return join(createTableAsync(name, tableInitChange));
     }
 
     /** {@inheritDoc} */
@@ -608,14 +602,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public Table createTableIfNotExists(String name, Consumer<TableChange> tableInitChange) {
-        if (!busyLock.enterBusy())
-            throw new IgniteException(new NodeStoppingException());
-        try {
-            return createTableIfNotExistsAsync(name, tableInitChange).join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        return join(createTableIfNotExistsAsync(name, tableInitChange));
     }
 
     /** {@inheritDoc} */
@@ -703,14 +690,22 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                                         changeSchemas(
                                         schemasCh -> schemasCh.create(
                                             String.valueOf(INITIAL_SCHEMA_VERSION),
-                                            schemaCh -> schemaCh.changeSchema(
-                                                SchemaSerializerImpl.INSTANCE.serialize(
-                                                    SchemaUtils.prepareSchemaDescriptor(
+                                            schemaCh -> {
+                                                SchemaDescriptor schemaDesc;
+
+                                                //TODO IGNITE-15747 Remove try-catch and force configuration validation here
+                                                // to ensure a valid configuration passed to prepareSchemaDescriptor() method.
+                                                try {
+                                                    schemaDesc = SchemaUtils.prepareSchemaDescriptor(
                                                         ((ExtendedTableView)ch).schemas().size(),
                                                         ch
-                                                    )
-                                                )
-                                            )
+                                                    );
+                                                } catch (IllegalArgumentException ex) {
+                                                    throw new ConfigurationValidationException(ex.getMessage());
+                                                }
+
+                                                schemaCh.changeSchema(SchemaSerializerImpl.INSTANCE.serialize(schemaDesc));
+                                            }
                                         )
                                     );
                             }
@@ -731,14 +726,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public void alterTable(String name, Consumer<TableChange> tableChange) {
-        if (!busyLock.enterBusy())
-            throw new IgniteException(new NodeStoppingException());
-        try {
-            alterTableAsync(name, tableChange).join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        join(alterTableAsync(name, tableChange));
     }
 
     /** {@inheritDoc} */
@@ -805,17 +793,33 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                                         schemaCh -> {
                                             ExtendedTableView currTableView = (ExtendedTableView)tablesCfg.tables().get(name).value();
 
-                                            SchemaDescriptor descriptor = SchemaUtils.prepareSchemaDescriptor(
-                                                ((ExtendedTableView)tblCh).schemas().size(),
-                                                tblCh
-                                            );
-
-                                            descriptor.columnMapping(SchemaUtils.columnMapper(
-                                                tablesById.get(tblId).schemaView().schema(currTableView.schemas().size()),
-                                                currTableView,
-                                                descriptor,
-                                                tblCh
-                                            ));
+                                            SchemaDescriptor descriptor;
+
+                                            //TODO IGNITE-15747 Remove try-catch and force configuration validation here
+                                            // to ensure a valid configuration passed to prepareSchemaDescriptor() method.
+                                            try {
+                                                descriptor = SchemaUtils.prepareSchemaDescriptor(
+                                                    ((ExtendedTableView)tblCh).schemas().size(),
+                                                    tblCh
+                                                );
+
+                                                descriptor.columnMapping(SchemaUtils.columnMapper(
+                                                    tablesById.get(tblId).schemaView().schema(currTableView.schemas().size()),
+                                                    currTableView,
+                                                    descriptor,
+                                                    tblCh
+                                                ));
+                                            }
+                                            catch (IllegalArgumentException ex) {
+                                                // Convert unexpected exceptions here,
+                                                // because validation actually happens later,
+                                                // when bulk configuration update is applied.
+                                                ConfigurationValidationException e = new ConfigurationValidationException(ex.getMessage());
+
+                                                e.addSuppressed(ex);
+
+                                                throw e;
+                                            }
 
                                             schemaCh.changeSchema(SchemaSerializerImpl.INSTANCE.serialize(descriptor));
                                         }
@@ -837,14 +841,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public void dropTable(String name) {
-        if (!busyLock.enterBusy())
-            throw new IgniteException(new NodeStoppingException());
-        try {
-            dropTableAsync(name).join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        join(dropTableAsync(name));
     }
 
     /** {@inheritDoc} */
@@ -914,14 +911,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public List<Table> tables() {
-        if (!busyLock.enterBusy())
-            throw new IgniteException(new NodeStoppingException());
-        try {
-            return tablesAsync().join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        return join(tablesAsync());
     }
 
     /** {@inheritDoc} */
@@ -1022,14 +1012,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public Table table(String name) {
-        if (!busyLock.enterBusy())
-            throw new IgniteException(new NodeStoppingException());
-        try {
-            return tableAsync(name).join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        return join(tableAsync(name));
     }
 
     /** {@inheritDoc} */
@@ -1046,14 +1029,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override public TableImpl table(IgniteUuid id) throws NodeStoppingException {
-        if (!busyLock.enterBusy())
-            throw new NodeStoppingException();
-        try {
-            return tableAsync(id).join();
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
+        return join(tableAsync(id));
     }
 
     /** {@inheritDoc} */
@@ -1195,6 +1171,41 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     }
 
     /**
+     * Waits for future result and return, or
+     * unwraps {@link CompletionException} to {@link IgniteException} if failed.
+     *
+     * @param future Completable future.
+     * @return Future result.
+     */
+    private <T> T join(CompletableFuture<T> future) {
+        if (!busyLock.enterBusy())
+            throw new IgniteException(new NodeStoppingException());
+
+        try {
+            return future.join();
+        }
+        catch (CompletionException ex) {
+            throw convertThrowable(ex.getCause());
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Convert to public throwable.
+     *
+     * @param th Throwable.
+     * @return Public throwable.
+     */
+    private RuntimeException convertThrowable(Throwable th) {
+        if (th instanceof RuntimeException)
+            return (RuntimeException)th;
+
+        return new IgniteException(th);
+    }
+
+    /**
      * Sets the nodes as baseline for all tables created by the manager.
      *
      * @param nodes New baseline nodes.