You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sa...@apache.org on 2023/01/05 08:17:16 UTC

[ignite-3] branch main updated: IGNITE-18470 Exception handling in DistributionZoneManager was reworked. (#1485)

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

sanpwc 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 4f2a80453e IGNITE-18470 Exception handling in DistributionZoneManager was reworked. (#1485)
4f2a80453e is described below

commit 4f2a80453eaf4707984e9b7a1d36f7aa51858176
Author: Sergey Uttsel <ut...@gmail.com>
AuthorDate: Thu Jan 5 11:17:11 2023 +0300

    IGNITE-18470 Exception handling in DistributionZoneManager was reworked. (#1485)
---
 modules/distribution-zones/build.gradle            |   1 +
 modules/distribution-zones/pom.xml                 |   5 +
 .../distributionzones/DistributionZoneManager.java | 318 ++++++++++++++-------
 .../DistributionZoneManagerTest.java               | 229 +++++++++------
 modules/sql-engine/build.gradle                    |   2 +
 .../DdlCommandHandlerExceptionHandlingTest.java    | 187 ++++++++++++
 6 files changed, 550 insertions(+), 192 deletions(-)

diff --git a/modules/distribution-zones/build.gradle b/modules/distribution-zones/build.gradle
index 298c6c8d92..ba2ecaf58a 100644
--- a/modules/distribution-zones/build.gradle
+++ b/modules/distribution-zones/build.gradle
@@ -43,6 +43,7 @@ dependencies {
 
     testImplementation libs.mockito.core
     testImplementation libs.mockito.junit
+    testImplementation libs.hamcrest.core
     testImplementation project(':ignite-raft-api')
     testImplementation project(':ignite-network-api')
     testImplementation project(':ignite-metastorage-api')
diff --git a/modules/distribution-zones/pom.xml b/modules/distribution-zones/pom.xml
index ad853c6aa2..24169f7040 100644
--- a/modules/distribution-zones/pom.xml
+++ b/modules/distribution-zones/pom.xml
@@ -99,6 +99,11 @@
             <scope>test</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.hamcrest</groupId>
+            <artifactId>hamcrest</artifactId>
+            <scope>test</scope>
+        </dependency>
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-metastorage</artifactId>
diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
index 46f4982960..9af30e9ca5 100644
--- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
+++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.distributionzones;
 
 import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
 import static java.util.stream.Collectors.toList;
 import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.deleteDataNodesKeyAndUpdateTriggerKey;
 import static org.apache.ignite.internal.distributionzones.DistributionZonesUtil.triggerKeyCondition;
@@ -36,15 +37,16 @@ import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 import org.apache.ignite.configuration.ConfigurationChangeException;
 import org.apache.ignite.configuration.NamedListChange;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyEventListener;
 import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService;
 import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
@@ -153,21 +155,25 @@ public class DistributionZoneManager implements IgniteComponent {
      * Creates a new distribution zone with the given {@code name} asynchronously.
      *
      * @param distributionZoneCfg Distribution zone configuration.
-     * @return Future representing pending completion of the operation.
-     *      Future can be completed with {@link ConfigurationChangeException} if a zone with the given name already exists
-     *      or {@code distributionZoneCfg} is broken.
-     * @throws NullPointerException if {@code distributionZoneCfg} is {@code null}.
-     * @throws NodeStoppingException If the node is stopping.
+     * @return Future representing pending completion of the operation. Future can be completed with:
+     *      {@link DistributionZoneAlreadyExistsException} if a zone with the given name already exists,
+     *      {@link ConfigurationValidationException} if {@code distributionZoneCfg} is broken,
+     *      {@link IllegalArgumentException} if distribution zone configuration is null,
+     *      {@link NodeStoppingException} if the node is stopping.
      */
     public CompletableFuture<Void> createZone(DistributionZoneConfigurationParameters distributionZoneCfg) {
-        Objects.requireNonNull(distributionZoneCfg, "Distribution zone configuration is null.");
+        if (distributionZoneCfg == null) {
+            return failedFuture(new IllegalArgumentException("Distribution zone configuration is null"));
+        }
 
         if (!busyLock.enterBusy()) {
-            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+            return failedFuture(new NodeStoppingException());
         }
 
         try {
-            return zonesConfiguration.change(zonesChange -> zonesChange.changeDistributionZones(zonesListChange -> {
+            CompletableFuture<Void> fut = new CompletableFuture<>();
+
+            zonesConfiguration.change(zonesChange -> zonesChange.changeDistributionZones(zonesListChange -> {
                 try {
                     zonesListChange.create(distributionZoneCfg.name(), zoneChange -> {
                         if (distributionZoneCfg.dataNodesAutoAdjust() == null) {
@@ -197,7 +203,20 @@ public class DistributionZoneManager implements IgniteComponent {
                 } catch (IllegalArgumentException e) {
                     throw new DistributionZoneAlreadyExistsException(distributionZoneCfg.name(), e);
                 }
-            }));
+            })).whenComplete((res, e) -> {
+                if (e != null) {
+                    fut.completeExceptionally(
+                            unwrapDistributionZoneException(
+                                    e,
+                                    DistributionZoneAlreadyExistsException.class,
+                                    ConfigurationValidationException.class)
+                    );
+                } else {
+                    fut.complete(null);
+                }
+            });
+
+            return fut;
         } finally {
             busyLock.leaveBusy();
         }
@@ -208,22 +227,31 @@ public class DistributionZoneManager implements IgniteComponent {
      *
      * @param name Distribution zone name.
      * @param distributionZoneCfg Distribution zone configuration.
-     * @return Future representing pending completion of the operation.
-     *      Future can be completed with {@link ConfigurationChangeException} if a zone with the given name already exists or
-     *      zone with name for renaming already exists or {@code distributionZoneCfg} is broken.
-     * @throws NullPointerException if {@code name} or {@code distributionZoneCfg} is {@code null}.
-     * @throws NodeStoppingException If the node is stopping.
+     * @return Future representing pending completion of the operation. Future can be completed with:
+     *      {@link DistributionZoneRenameException} if a zone with the given name already exists
+     *      or zone with name for renaming already exists,
+     *      {@link DistributionZoneNotFoundException} if a zone with the given name doesn't exist,
+     *      {@link ConfigurationValidationException} if {@code distributionZoneCfg} is broken,
+     *      {@link IllegalArgumentException} if {@code name} or {@code distributionZoneCfg} is {@code null},
+     *      {@link NodeStoppingException} if the node is stopping.
      */
     public CompletableFuture<Void> alterZone(String name, DistributionZoneConfigurationParameters distributionZoneCfg) {
-        Objects.requireNonNull(name, "Distribution zone name is null.");
-        Objects.requireNonNull(distributionZoneCfg, "Distribution zone configuration is null.");
+        if (name == null || name.isEmpty()) {
+            return failedFuture(new IllegalArgumentException("Distribution zone name is null or empty [name=" + name + ']'));
+        }
+
+        if (distributionZoneCfg == null) {
+            return failedFuture(new IllegalArgumentException("Distribution zone configuration is null"));
+        }
 
         if (!busyLock.enterBusy()) {
-            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+            return failedFuture(new NodeStoppingException());
         }
 
         try {
-            return zonesConfiguration.change(zonesChange -> zonesChange.changeDistributionZones(zonesListChange -> {
+            CompletableFuture<Void> fut = new CompletableFuture<>();
+
+            zonesConfiguration.change(zonesChange -> zonesChange.changeDistributionZones(zonesListChange -> {
                 NamedListChange<DistributionZoneView, DistributionZoneChange> renameChange;
 
                 try {
@@ -257,7 +285,22 @@ public class DistributionZoneManager implements IgniteComponent {
                 } catch (IllegalArgumentException e) {
                     throw new DistributionZoneNotFoundException(distributionZoneCfg.name(), e);
                 }
-            }));
+            }))
+                    .whenComplete((res, e) -> {
+                        if (e != null) {
+                            fut.completeExceptionally(
+                                    unwrapDistributionZoneException(
+                                            e,
+                                            DistributionZoneRenameException.class,
+                                            DistributionZoneNotFoundException.class,
+                                            ConfigurationValidationException.class)
+                            );
+                        } else {
+                            fut.complete(null);
+                        }
+                    });
+
+            return fut;
         } finally {
             busyLock.leaveBusy();
         }
@@ -267,20 +310,24 @@ public class DistributionZoneManager implements IgniteComponent {
      * Drops a distribution zone with the name specified.
      *
      * @param name Distribution zone name.
-     * @return Future representing pending completion of the operation.
-     *      Future can be completed with {@link ConfigurationChangeException} if a zone with the given name doesn't exist.
-     * @throws NullPointerException if {@code name}  is {@code null}.
-     * @throws NodeStoppingException If the node is stopping.
+     * @return Future representing pending completion of the operation. Future can be completed with:
+     *      {@link DistributionZoneNotFoundException} if a zone with the given name doesn't exist,
+     *      {@link IllegalArgumentException} if {@code name} is {@code null},
+     *      {@link NodeStoppingException} if the node is stopping.
      */
     public CompletableFuture<Void> dropZone(String name) {
-        Objects.requireNonNull(name, "Distribution zone name is null.");
+        if (name == null || name.isEmpty()) {
+            return failedFuture(new IllegalArgumentException("Distribution zone name is null or empty [name=" + name + ']'));
+        }
 
         if (!busyLock.enterBusy()) {
-            throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
+            return failedFuture(new NodeStoppingException());
         }
 
         try {
-            return zonesConfiguration.change(zonesChange -> zonesChange.changeDistributionZones(zonesListChange -> {
+            CompletableFuture<Void> fut = new CompletableFuture<>();
+
+            zonesConfiguration.change(zonesChange -> zonesChange.changeDistributionZones(zonesListChange -> {
                 DistributionZoneView view = zonesListChange.get(name);
 
                 if (view == null) {
@@ -288,7 +335,20 @@ public class DistributionZoneManager implements IgniteComponent {
                 }
 
                 zonesListChange.delete(name);
-            }));
+            }))
+                    .whenComplete((res, e) -> {
+                        if (e != null) {
+                            fut.completeExceptionally(
+                                    unwrapDistributionZoneException(
+                                            e,
+                                            DistributionZoneNotFoundException.class)
+                            );
+                        } else {
+                            fut.complete(null);
+                        }
+                    });
+
+            return fut;
         } finally {
             busyLock.leaveBusy();
         }
@@ -509,61 +569,69 @@ public class DistributionZoneManager implements IgniteComponent {
      * {@link DistributionZonesUtil#zonesLogicalTopologyVersionKey()} from meta storage on the start of {@link DistributionZoneManager}.
      */
     private void initMetaStorageKeysOnStart() {
-        logicalTopologyService.logicalTopologyOnLeader().thenAccept(snapshot -> {
-            if (!busyLock.enterBusy()) {
-                throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
-            }
-
-            try {
-                metaStorageManager.get(zonesLogicalTopologyVersionKey()).thenAccept(topVerEntry -> {
-                    if (!busyLock.enterBusy()) {
-                        throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
-                    }
-
-                    try {
-                        long topologyVersionFromCmg = snapshot.version();
-
-                        byte[] topVerFromMetaStorage = topVerEntry.value();
-
-                        if (topVerFromMetaStorage == null || bytesToLong(topVerFromMetaStorage) < topologyVersionFromCmg) {
-                            Set<String> topologyFromCmg = snapshot.nodes().stream().map(ClusterNode::name).collect(Collectors.toSet());
+        if (!busyLock.enterBusy()) {
+            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
 
-                            Condition topologyVersionCondition = topVerFromMetaStorage == null
-                                    ? notExists(zonesLogicalTopologyVersionKey()) :
-                                    value(zonesLogicalTopologyVersionKey()).eq(topVerFromMetaStorage);
+        try {
+            logicalTopologyService.logicalTopologyOnLeader().thenAccept(snapshot -> {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+                }
 
-                            If iff = If.iif(topologyVersionCondition,
-                                    updateLogicalTopologyAndVersion(topologyFromCmg, topologyVersionFromCmg),
-                                    ops().yield(false)
-                            );
+                try {
+                    metaStorageManager.get(zonesLogicalTopologyVersionKey()).thenAccept(topVerEntry -> {
+                        if (!busyLock.enterBusy()) {
+                            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+                        }
 
-                            metaStorageManager.invoke(iff).thenAccept(res -> {
-                                if (res.getAsBoolean()) {
-                                    LOG.debug(
-                                            "Distribution zones' logical topology and version keys were initialised "
-                                                    + "[topology = {}, version = {}]",
-                                            Arrays.toString(topologyFromCmg.toArray()),
-                                            topologyVersionFromCmg
-                                    );
-                                } else {
-                                    LOG.debug(
-                                            "Failed to initialize distribution zones' logical topology "
-                                                    + "and version keys [topology = {}, version = {}]",
-                                            Arrays.toString(topologyFromCmg.toArray()),
-                                            topologyVersionFromCmg
-                                    );
-                                }
-                            });
+                        try {
+                            long topologyVersionFromCmg = snapshot.version();
+
+                            byte[] topVerFromMetaStorage = topVerEntry.value();
+
+                            if (topVerFromMetaStorage == null || bytesToLong(topVerFromMetaStorage) < topologyVersionFromCmg) {
+                                Set<String> topologyFromCmg = snapshot.nodes().stream().map(ClusterNode::name).collect(Collectors.toSet());
+
+                                Condition topologyVersionCondition = topVerFromMetaStorage == null
+                                        ? notExists(zonesLogicalTopologyVersionKey()) :
+                                        value(zonesLogicalTopologyVersionKey()).eq(topVerFromMetaStorage);
+
+                                If iff = If.iif(topologyVersionCondition,
+                                        updateLogicalTopologyAndVersion(topologyFromCmg, topologyVersionFromCmg),
+                                        ops().yield(false)
+                                );
+
+                                metaStorageManager.invoke(iff).thenAccept(res -> {
+                                    if (res.getAsBoolean()) {
+                                        LOG.debug(
+                                                "Distribution zones' logical topology and version keys were initialised "
+                                                        + "[topology = {}, version = {}]",
+                                                Arrays.toString(topologyFromCmg.toArray()),
+                                                topologyVersionFromCmg
+                                        );
+                                    } else {
+                                        LOG.debug(
+                                                "Failed to initialize distribution zones' logical topology "
+                                                        + "and version keys [topology = {}, version = {}]",
+                                                Arrays.toString(topologyFromCmg.toArray()),
+                                                topologyVersionFromCmg
+                                        );
+                                    }
+                                });
+                            }
+                        } finally {
+                            busyLock.leaveBusy();
                         }
-                    } finally {
-                        busyLock.leaveBusy();
-                    }
-                });
+                    });
 
-            } finally {
-                busyLock.leaveBusy();
-            }
-        });
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            });
+        } finally {
+            busyLock.leaveBusy();
+        }
     }
 
     /**
@@ -571,40 +639,49 @@ public class DistributionZoneManager implements IgniteComponent {
      * from {@link DistributionZonesUtil#zonesLogicalTopologyKey()} in vault.
      */
     private void initDataNodesFromVaultManager() {
-        // TODO: Remove this call as part of https://issues.apache.org/jira/browse/IGNITE-18397
-        vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV)
-                .thenApply(appliedRevision -> appliedRevision == null ? 0L : bytesToLong(appliedRevision.value()))
-                .thenAccept(vaultAppliedRevision -> {
-                    if (!busyLock.enterBusy()) {
-                        throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
-                    }
+        if (!busyLock.enterBusy()) {
+            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+        }
 
-                    try {
-                        vaultMgr.get(zonesLogicalTopologyKey())
-                                .thenAccept(vaultEntry -> {
-                                    if (!busyLock.enterBusy()) {
-                                        throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
-                                    }
+        try {
+            // TODO: Remove this call as part of https://issues.apache.org/jira/browse/IGNITE-18397
+            vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV)
+                    .thenApply(appliedRevision -> appliedRevision == null ? 0L : bytesToLong(appliedRevision.value()))
+                    .thenAccept(vaultAppliedRevision -> {
+                        if (!busyLock.enterBusy()) {
+                            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+                        }
+
+                        try {
+                            vaultMgr.get(zonesLogicalTopologyKey())
+                                    .thenAccept(vaultEntry -> {
+                                        if (!busyLock.enterBusy()) {
+                                            throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+                                        }
 
-                                    try {
-                                        if (vaultEntry != null && vaultEntry.value() != null) {
-                                            logicalTopology = ByteUtils.fromBytes(vaultEntry.value());
+                                        try {
+                                            if (vaultEntry != null && vaultEntry.value() != null) {
+                                                logicalTopology = ByteUtils.fromBytes(vaultEntry.value());
 
-                                            zonesConfiguration.distributionZones().value().namedListKeys()
-                                                    .forEach(zoneName -> {
-                                                        int zoneId = zonesConfiguration.distributionZones().get(zoneName).zoneId().value();
+                                                zonesConfiguration.distributionZones().value().namedListKeys()
+                                                        .forEach(zoneName -> {
+                                                            int zoneId = zonesConfiguration.distributionZones().get(zoneName).zoneId()
+                                                                    .value();
 
-                                                        saveDataNodesToMetaStorage(zoneId, vaultEntry.value(), vaultAppliedRevision);
-                                                    });
+                                                            saveDataNodesToMetaStorage(zoneId, vaultEntry.value(), vaultAppliedRevision);
+                                                        });
+                                            }
+                                        } finally {
+                                            busyLock.leaveBusy();
                                         }
-                                    } finally {
-                                        busyLock.leaveBusy();
-                                    }
-                                });
-                    } finally {
-                        busyLock.leaveBusy();
-                    }
-                });
+                                    });
+                        } finally {
+                            busyLock.leaveBusy();
+                        }
+                    });
+        } finally {
+            busyLock.leaveBusy();
+        }
     }
 
     /**
@@ -703,4 +780,31 @@ public class DistributionZoneManager implements IgniteComponent {
             }
         });
     }
+
+    /**
+     * Unwraps distribution zone exception from {@link ConfigurationChangeException} if it is possible.
+     *
+     * @param e Exception.
+     * @param expectedClz Expected exception classes to unwrap.
+     * @return Unwrapped exception if it is expected or original if it is unexpected exception.
+     */
+    private static Throwable unwrapDistributionZoneException(Throwable e, Class<? extends Throwable>... expectedClz) {
+        Throwable ret = unwrapDistributionZoneExceptionRecursively(e, expectedClz);
+
+        return ret != null ? ret : e;
+    }
+
+    private static Throwable unwrapDistributionZoneExceptionRecursively(Throwable e, Class<? extends Throwable>... expectedClz) {
+        if ((e instanceof CompletionException || e instanceof ConfigurationChangeException) && e.getCause() != null) {
+            return unwrapDistributionZoneExceptionRecursively(e.getCause(), expectedClz);
+        }
+
+        for (Class<?> expected : expectedClz) {
+            if (expected.isAssignableFrom(e.getClass())) {
+                return e;
+            }
+        }
+
+        return null;
+    }
 }
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerTest.java
index 6e5e94f715..473296c55f 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerTest.java
@@ -18,18 +18,17 @@
 package org.apache.ignite.internal.distributionzones;
 
 import static org.apache.ignite.configuration.annotation.ConfigurationType.DISTRIBUTED;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.Mockito.mock;
 
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-import org.apache.ignite.configuration.ConfigurationChangeException;
-import org.apache.ignite.internal.cluster.management.topology.LogicalTopologyServiceImpl;
+import org.apache.ignite.configuration.validation.ConfigurationValidationException;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage;
 import org.apache.ignite.internal.distributionzones.DistributionZoneConfigurationParameters.Builder;
@@ -38,9 +37,8 @@ import org.apache.ignite.internal.distributionzones.configuration.DistributionZo
 import org.apache.ignite.internal.distributionzones.exception.DistributionZoneAlreadyExistsException;
 import org.apache.ignite.internal.distributionzones.exception.DistributionZoneNotFoundException;
 import org.apache.ignite.internal.distributionzones.exception.DistributionZoneRenameException;
-import org.apache.ignite.internal.metastorage.MetaStorageManager;
 import org.apache.ignite.internal.testframework.IgniteAbstractTest;
-import org.apache.ignite.internal.vault.VaultManager;
+import org.hamcrest.Matchers;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -72,9 +70,9 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         DistributionZonesConfiguration zonesConfiguration = registry.getConfiguration(DistributionZonesConfiguration.KEY);
         distributionZoneManager = new DistributionZoneManager(
                 zonesConfiguration,
-                mock(MetaStorageManager.class),
-                mock(LogicalTopologyServiceImpl.class),
-                mock(VaultManager.class)
+                null,
+                null,
+                null
         );
     }
 
@@ -93,11 +91,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         DistributionZoneConfiguration zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(ZONE_NAME, zone1.name().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(100, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(ZONE_NAME, zone1.name().value(), "Zone name is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(100, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
     }
 
     @Test
@@ -111,18 +109,18 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         DistributionZoneConfiguration zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(ZONE_NAME, zone1.name().value());
-        assertEquals(100, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(ZONE_NAME, zone1.name().value(), "Zone name is wrong.");
+        assertEquals(100, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
 
         distributionZoneManager.dropZone(ZONE_NAME).get(5, TimeUnit.SECONDS);
 
         zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNull(zone1);
+        assertNull(zone1, "Zone was not dropped.");
     }
 
     @Test
@@ -136,18 +134,18 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         DistributionZoneConfiguration zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(ZONE_NAME, zone1.name().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(200, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(ZONE_NAME, zone1.name().value(), "Zone name is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(200, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
 
         distributionZoneManager.dropZone(ZONE_NAME).get(5, TimeUnit.SECONDS);
 
         zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNull(zone1);
+        assertNull(zone1, "Zone was not dropped.");
     }
 
     @Test
@@ -170,8 +168,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause().getCause() instanceof DistributionZoneAlreadyExistsException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneAlreadyExistsException,
+                "Unexpected type of exception (requires DistributionZoneAlreadyExistsException): " + e
+        );
     }
 
     @Test
@@ -186,8 +187,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause().getCause() instanceof DistributionZoneNotFoundException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneNotFoundException,
+                "Unexpected type of exception (requires DistributionZoneNotFoundException): " + e
+        );
     }
 
     @Test
@@ -200,11 +204,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         DistributionZoneConfiguration zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(ZONE_NAME, zone1.name().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(100, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(ZONE_NAME, zone1.name().value(), "Zone name is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(100, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
 
 
         distributionZoneManager.alterZone(ZONE_NAME, new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
@@ -214,10 +218,10 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(200, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(300, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(200, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(300, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
 
 
         distributionZoneManager.alterZone(ZONE_NAME, new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
@@ -227,10 +231,10 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(400, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(300, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(400, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(300, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
 
 
         distributionZoneManager.alterZone(ZONE_NAME, new DistributionZoneConfigurationParameters.Builder(ZONE_NAME)
@@ -240,10 +244,10 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
         zone1 = registry.getConfiguration(DistributionZonesConfiguration.KEY).distributionZones()
                 .get(ZONE_NAME);
 
-        assertNotNull(zone1);
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(500, zone1.dataNodesAutoAdjust().value());
+        assertNotNull(zone1, "Zone was not created.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone1.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(500, zone1.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
     }
 
     @Test
@@ -264,12 +268,12 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
                 .distributionZones()
                 .get(NEW_ZONE_NAME);
 
-        assertNull(zone1);
-        assertNotNull(zone2);
-        assertEquals(NEW_ZONE_NAME, zone2.name().value());
-        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(100, zone2.dataNodesAutoAdjust().value());
+        assertNull(zone1, "Zone was not renamed.");
+        assertNotNull(zone2, "Zone was not renamed.");
+        assertEquals(NEW_ZONE_NAME, zone2.name().value(), "Zone was not renamed.");
+        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(100, zone2.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
     }
 
     @Test
@@ -290,12 +294,12 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
                 .distributionZones()
                 .get(NEW_ZONE_NAME);
 
-        assertNull(zone1);
-        assertNotNull(zone2);
-        assertEquals(NEW_ZONE_NAME, zone2.name().value());
-        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleUp().value());
-        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleDown().value());
-        assertEquals(400, zone2.dataNodesAutoAdjust().value());
+        assertNull(zone1, "Zone was not renamed.");
+        assertNotNull(zone2, "Zone was not renamed.");
+        assertEquals(NEW_ZONE_NAME, zone2.name().value(), "Zone was not renamed.");
+        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleUp().value(), "dataNodesAutoAdjustScaleUp is wrong.");
+        assertEquals(Integer.MAX_VALUE, zone2.dataNodesAutoAdjustScaleDown().value(), "dataNodesAutoAdjustScaleDown is wrong.");
+        assertEquals(400, zone2.dataNodesAutoAdjust().value(), "dataNodesAutoAdjust is wrong.");
     }
 
     @Test
@@ -312,8 +316,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause().getCause() instanceof DistributionZoneRenameException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneRenameException,
+                "Unexpected type of exception (requires DistributionZoneRenameException): " + e
+        );
     }
 
     @Test
@@ -335,8 +342,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause().getCause() instanceof DistributionZoneRenameException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneRenameException,
+                "Unexpected type of exception (requires DistributionZoneRenameException): " + e
+        );
     }
 
     @Test
@@ -352,8 +362,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause().getCause() instanceof DistributionZoneNotFoundException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneNotFoundException,
+                "Unexpected type of exception (requires DistributionZoneNotFoundException): " + e
+        );
     }
 
     @Test
@@ -369,8 +382,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause() instanceof ConfigurationChangeException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof ConfigurationValidationException,
+                "Unexpected type of exception (requires ConfigurationValidationException): " + e
+        );
     }
 
     @Test
@@ -386,8 +402,11 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause() instanceof ConfigurationChangeException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof ConfigurationValidationException,
+                "Unexpected type of exception (requires ConfigurationValidationException): " + e
+        );
     }
 
     @Test
@@ -403,66 +422,106 @@ class DistributionZoneManagerTest extends IgniteAbstractTest {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e.getCause() instanceof ConfigurationChangeException, e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof ConfigurationValidationException,
+                "Unexpected type of exception (requires ConfigurationValidationException): " + e
+        );
     }
 
     @Test
     public void testCreateZoneWithNullConfiguration() {
         Exception e = null;
 
+        CompletableFuture<Void> fut = distributionZoneManager.createZone(null);
+
         try {
-            distributionZoneManager.createZone(null);
+            fut.get(5, TimeUnit.SECONDS);
         } catch (Exception e0) {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e instanceof NullPointerException, e.toString());
-        assertEquals("Distribution zone configuration is null.", e.getMessage(), e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof IllegalArgumentException,
+                "Unexpected type of exception (requires IllegalArgumentException): " + e
+        );
+        assertEquals(
+                "Distribution zone configuration is null",
+                e.getCause().getMessage(),
+                "Unexpected exception message: " + e.getCause().getMessage()
+        );
     }
 
     @Test
     public void testAlterZoneWithNullName() {
         Exception e = null;
 
+        CompletableFuture<Void> fut = distributionZoneManager.alterZone(null, new Builder(ZONE_NAME).build());
+
         try {
-            distributionZoneManager.alterZone(null, new Builder(ZONE_NAME).build());
+            fut.get(5, TimeUnit.SECONDS);
         } catch (Exception e0) {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e instanceof NullPointerException, e.toString());
-        assertEquals("Distribution zone name is null.", e.getMessage(), e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof IllegalArgumentException,
+                "Unexpected type of exception (requires IllegalArgumentException): " + e
+        );
+        assertThat(
+                "Unexpected exception message: " + e.getCause().getMessage(),
+                e.getCause().getMessage(),
+                Matchers.containsString("Distribution zone name is null")
+        );
     }
 
     @Test
     public void testAlterZoneWithNullConfiguration() {
         Exception e = null;
 
+        CompletableFuture<Void> fut = distributionZoneManager.alterZone(ZONE_NAME, null);
+
         try {
-            distributionZoneManager.alterZone(ZONE_NAME, null);
+            fut.get(5, TimeUnit.SECONDS);
         } catch (Exception e0) {
             e = e0;
         }
 
-        assertTrue(e instanceof NullPointerException, e.toString());
-        assertEquals("Distribution zone configuration is null.", e.getMessage(), e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof IllegalArgumentException,
+                "Unexpected type of exception (requires IllegalArgumentException): " + e
+        );
+        assertEquals(
+                "Distribution zone configuration is null",
+                e.getCause().getMessage(),
+                "Unexpected exception message: " + e.getCause().getMessage()
+        );
     }
 
     @Test
     public void testDropZoneWithNullName() {
         Exception e = null;
 
+        CompletableFuture<Void> fut = distributionZoneManager.dropZone(null);
+
         try {
-            distributionZoneManager.dropZone(null);
+            fut.get(5, TimeUnit.SECONDS);
         } catch (Exception e0) {
             e = e0;
         }
 
-        assertTrue(e != null);
-        assertTrue(e instanceof NullPointerException, e.toString());
-        assertEquals("Distribution zone name is null.", e.getMessage(), e.toString());
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof IllegalArgumentException,
+                "Unexpected type of exception (requires IllegalArgumentException): " + e
+        );
+        assertThat(
+                "Unexpected exception message: " + e.getCause().getMessage(),
+                e.getCause().getMessage(),
+                Matchers.containsString("Distribution zone name is null")
+        );
     }
 }
diff --git a/modules/sql-engine/build.gradle b/modules/sql-engine/build.gradle
index 9a4db30a7f..30999cac1e 100644
--- a/modules/sql-engine/build.gradle
+++ b/modules/sql-engine/build.gradle
@@ -63,6 +63,8 @@ dependencies {
     testImplementation project(':ignite-configuration')
     testImplementation project(':ignite-storage-api')
     testImplementation project(':ignite-storage-rocksdb')
+    testImplementation project(':ignite-cluster-management')
+    testImplementation project(':ignite-vault')
     testImplementation(testFixtures(project(':ignite-core')))
     testImplementation(testFixtures(project(':ignite-configuration')))
     testImplementation(testFixtures(project(':ignite-storage-api')))
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerExceptionHandlingTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerExceptionHandlingTest.java
new file mode 100644
index 0000000000..1678c38467
--- /dev/null
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandlerExceptionHandlingTest.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine.exec.ddl;
+
+import static org.apache.ignite.configuration.annotation.ConfigurationType.DISTRIBUTED;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.mock;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.internal.configuration.ConfigurationRegistry;
+import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage;
+import org.apache.ignite.internal.distributionzones.DistributionZoneConfigurationParameters;
+import org.apache.ignite.internal.distributionzones.DistributionZoneManager;
+import org.apache.ignite.internal.distributionzones.configuration.DistributionZonesConfiguration;
+import org.apache.ignite.internal.distributionzones.exception.DistributionZoneAlreadyExistsException;
+import org.apache.ignite.internal.distributionzones.exception.DistributionZoneNotFoundException;
+import org.apache.ignite.internal.index.IndexManager;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.sql.engine.prepare.ddl.CreateZoneCommand;
+import org.apache.ignite.internal.sql.engine.prepare.ddl.DropZoneCommand;
+import org.apache.ignite.internal.storage.DataStorageManager;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+/**
+ * Tests distribution zone command exception handling.
+ */
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.LENIENT)
+public class DdlCommandHandlerExceptionHandlingTest extends IgniteAbstractTest {
+    @Mock
+    private TableManager tableManager;
+
+    @Mock
+    private IndexManager indexManager;
+
+    @Mock
+    private DataStorageManager dataStorageManager;
+
+    private DdlCommandHandler commandHandler;
+
+    private static final String ZONE_NAME = "zone1";
+
+    private final ConfigurationRegistry registry = new ConfigurationRegistry(
+            List.of(DistributionZonesConfiguration.KEY),
+            Map.of(),
+            new TestConfigurationStorage(DISTRIBUTED),
+            List.of(),
+            List.of()
+    );
+
+    private DistributionZoneManager distributionZoneManager;
+
+    /** Inner initialisation. */
+    @BeforeEach
+    void before() {
+        registry.start();
+
+        registry.initializeDefaults();
+
+        DistributionZonesConfiguration zonesConfiguration = registry.getConfiguration(DistributionZonesConfiguration.KEY);
+
+        distributionZoneManager = new DistributionZoneManager(
+                zonesConfiguration,
+                mock(MetaStorageManager.class),
+                null,
+                null
+        );
+
+        commandHandler = new DdlCommandHandler(distributionZoneManager, tableManager, indexManager, dataStorageManager);
+    }
+
+    @AfterEach
+    public void after() throws Exception {
+        registry.stop();
+    }
+
+    @Test
+    public void testZoneAlreadyExistsOnCreate1() throws ExecutionException, InterruptedException, TimeoutException {
+        CompletableFuture<Boolean> fut = handleCreateZoneCommand(false);
+
+        Exception e = null;
+
+        try {
+            fut.get(5, TimeUnit.SECONDS);
+        } catch (Exception e0) {
+            e = e0;
+        }
+
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneAlreadyExistsException,
+                "Unexpected type of exception (requires DistributionZoneAlreadyExistsException): " + e
+        );
+    }
+
+    @Test
+    public void testZoneAlreadyExistsOnCreate2() throws ExecutionException, InterruptedException, TimeoutException {
+        CompletableFuture<Boolean> fut = handleCreateZoneCommand(true);
+
+        try {
+            fut.get(5, TimeUnit.SECONDS);
+        } catch (Throwable e) {
+            fail("Expected no exception but was: " + e);
+        }
+    }
+
+    @Test
+    public void testZoneNotFoundOnDrop1() {
+        DropZoneCommand cmd = new DropZoneCommand();
+        cmd.zoneName(ZONE_NAME);
+
+        CompletableFuture<Boolean> fut = commandHandler.handle(cmd);
+
+        Exception e = null;
+
+        try {
+            fut.get(5, TimeUnit.SECONDS);
+        } catch (Exception e0) {
+            e = e0;
+        }
+
+        assertTrue(e != null, "Expected exception was not thrown.");
+        assertTrue(
+                e.getCause() instanceof DistributionZoneNotFoundException,
+                "Unexpected type of exception (requires DistributionZoneNotFoundException): " + e
+        );
+    }
+
+    @Test
+    public void testZoneNotFoundOnDrop2() {
+        DropZoneCommand cmd = new DropZoneCommand();
+        cmd.zoneName(ZONE_NAME);
+        cmd.ifExists(true);
+
+        CompletableFuture<Boolean> fut = commandHandler.handle(cmd);
+
+        try {
+            fut.get(5, TimeUnit.SECONDS);
+        } catch (Throwable e) {
+            fail("Expected no exception but was: " + e);
+        }
+    }
+
+    private CompletableFuture<Boolean> handleCreateZoneCommand(boolean ifNotExists)
+            throws ExecutionException, InterruptedException, TimeoutException {
+        distributionZoneManager.createZone(
+                        new DistributionZoneConfigurationParameters.Builder(ZONE_NAME).build()
+                )
+                .get(5, TimeUnit.SECONDS);
+
+        CreateZoneCommand cmd = new CreateZoneCommand();
+        cmd.zoneName(ZONE_NAME);
+        cmd.ifNotExists(ifNotExists);
+
+        return commandHandler.handle(cmd);
+    }
+}