You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2022/08/01 12:04:55 UTC

[ignite-3] branch main updated: IGNITE-14986 Introduced meta storage error codes

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

sk0x50 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 3349f963a IGNITE-14986 Introduced meta storage error codes
3349f963a is described below

commit 3349f963a5f7eca9c7d563a274d708f6bea4044b
Author: Slava Koptilin <sl...@gmail.com>
AuthorDate: Mon Aug 1 15:04:38 2022 +0300

    IGNITE-14986 Introduced meta storage error codes
---
 .../ignite/internal/util/ExceptionUtils.java       | 110 +++++++++++++++++++++
 .../java/org/apache/ignite/lang/ErrorGroup.java    |  16 ++-
 .../java/org/apache/ignite/lang/ErrorGroups.java   |  48 ++++++++-
 .../apache/ignite/lang/IgniteTetraFunction.java    |  48 +++++++++
 .../org/apache/ignite/lang/IgniteTriFunction.java  |  47 +++++++++
 .../apache/ignite/lang/NodeStoppingException.java  |  13 ++-
 .../metastorage/client/CompactedException.java     |  14 ++-
 .../internal/metastorage/client/CursorImpl.java    |  30 +++---
 .../metastorage/client/MetaStorageServiceImpl.java |   4 +-
 .../client/OperationTimeoutException.java          |  14 ++-
 .../metastorage/client/StatementResult.java        |   7 +-
 .../metastorage/common/MetaStorageException.java   | 110 +++++++++++++++++++++
 .../server/persistence/RocksDbKeyValueStorage.java |  43 ++++----
 .../server/persistence/WatchCursor.java            |   5 +-
 .../server/raft/MetaStorageListener.java           |  10 +-
 .../server/SimpleInMemoryKeyValueStorage.java      |   6 +-
 .../internal/metastorage/MetaStorageManager.java   |  15 +--
 17 files changed, 472 insertions(+), 68 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ExceptionUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ExceptionUtils.java
index f0125c5d7..218467f8f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ExceptionUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ExceptionUtils.java
@@ -25,8 +25,13 @@ import java.lang.reflect.Method;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.CompletionException;
 import java.util.concurrent.ExecutionException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteTetraFunction;
+import org.apache.ignite.lang.IgniteTriFunction;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -324,4 +329,109 @@ public final class ExceptionUtils {
 
         return e;
     }
+
+    /**
+     * Creates a new exception, which type is defined by the provided {@code supplier}, with the specified {@code t} as a cause.
+     * In the case when the provided cause {@code t} is an instance of {@link IgniteInternalException}
+     * or {@link IgniteInternalCheckedException}, the original trace identifier and full error code are preserved.
+     * Otherwise, a newly generated trace identifier and {@code defaultCode} are used.
+     *
+     * @param supplier Reference to a exception constructor.
+     * @param defaultCode Error code to be used in the case when the provided cause {@code t} is not an instance of Ignite exception.
+     * @param t Cause to be used.
+     * @param <T> Type of a new exception.
+     * @return New exception with the given cause.
+     */
+    public static <T extends Exception> T withCause(IgniteTriFunction<UUID, Integer, Throwable, T> supplier, int defaultCode, Throwable t) {
+        return withCauseInternal((traceId, code, message, cause) -> supplier.apply(traceId, code, t), defaultCode, t);
+    }
+
+    /**
+     * Creates a new exception, which type is defined by the provided {@code supplier}, with the specified {@code t} as a cause.
+     * In the case when the provided cause {@code t} is an instance of {@link IgniteInternalException}
+     * or {@link IgniteInternalCheckedException}, the original trace identifier and full error code are preserved.
+     * Otherwise, a newly generated trace identifier and {@code defaultCode} are used.
+     *
+     * @param supplier Reference to a exception constructor.
+     * @param defaultCode Error code to be used in the case when the provided cause {@code t} is not an instance of Ignite exception.
+     * @param message Detailed error message.
+     * @param t Cause to be used.
+     * @param <T> Type of a new exception.
+     * @return New exception with the given cause.
+     */
+    public static <T extends Exception> T withCause(
+            IgniteTetraFunction<UUID, Integer, String, Throwable, T> supplier,
+            int defaultCode,
+            String message,
+            Throwable t
+    ) {
+        return withCauseInternal((traceId, code, m, cause) -> supplier.apply(traceId, code, message, t), defaultCode, t);
+    }
+
+    /**
+     * Creates a new exception, which type is defined by the provided {@code supplier}, with the specified {@code t} as a cause
+     * and full error code {@code code}.
+     * In the case when the provided cause {@code t} is an instance of {@link IgniteInternalException}
+     * or {@link IgniteInternalCheckedException}, the original trace identifier preserved.
+     * Otherwise, a newly generated trace identifier is used.
+     *
+     * @param supplier Reference to a exception constructor.
+     * @param code New error code.
+     * @param t Cause to be used.
+     * @param <T> Type of a new exception.
+     * @return New exception with the given cause.
+     */
+    public static <T extends Exception> T withCauseAndCode(IgniteTriFunction<UUID, Integer, Throwable, T> supplier, int code, Throwable t) {
+        return withCauseInternal((traceId, c, message, cause) -> supplier.apply(traceId, code, t), code, t);
+    }
+
+    /**
+     * Creates a new exception, which type is defined by the provided {@code supplier}, with the specified {@code t} as a cause,
+     * full error code {@code code} and error message {@code message}.
+     * In the case when the provided cause {@code t} is an instance of {@link IgniteInternalException}
+     * or {@link IgniteInternalCheckedException}, the original trace identifier preserved.
+     * Otherwise, a newly generated trace identifier is used.
+     *
+     * @param supplier Reference to a exception constructor.
+     * @param code New error code.
+     * @param message Detailed error message.
+     * @param t Cause to be used.
+     * @param <T> Type of a new exception.
+     * @return New exception with the given cause.
+     */
+    public static <T extends Exception> T withCauseAndCode(
+            IgniteTetraFunction<UUID, Integer, String, Throwable, T> supplier,
+            int code,
+            String message,
+            Throwable t
+    ) {
+        return withCauseInternal((traceId, c, m, cause) -> supplier.apply(traceId, code, message, t), code, t);
+    }
+
+    /**
+     * Extracts the trace identifier and full error code from ignite exception and creates a new one based on the provided {@code supplier}.
+     *
+     * @param supplier Supplier to create a concrete exception instance.
+     * @param defaultCode Error code to be used in the case when the provided cause {@code t} is not an instance of Ignite exception.
+     * @param t Cause.
+     * @param <T> Type of a new exception.
+     * @return New
+     */
+    private static <T extends Exception> T withCauseInternal(
+            IgniteTetraFunction<UUID, Integer, String, Throwable, T> supplier,
+            int defaultCode,
+            Throwable t
+    ) {
+        Throwable unwrapped = unwrapCause(t);
+
+        if (unwrapped instanceof IgniteInternalException) {
+            IgniteInternalException iie = (IgniteInternalException) unwrapped;
+            return supplier.apply(iie.traceId(), iie.code(), iie.getMessage(), t);
+        } else if (unwrapped instanceof IgniteInternalCheckedException) {
+            IgniteInternalCheckedException iice = (IgniteInternalCheckedException) unwrapped;
+            return supplier.apply(iice.traceId(), iice.code(), iice.getMessage(), t);
+        }
+
+        return supplier.apply(UUID.randomUUID(), defaultCode, t.getMessage(), t);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroup.java b/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroup.java
index f2c288074..c9a97da9f 100755
--- a/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroup.java
@@ -23,6 +23,8 @@ import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
 import it.unimi.dsi.fastutil.ints.IntSet;
 import java.util.Locale;
 import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 /**
  * This class represents a concept of error group. Error group defines a collection of errors that belong to a single semantic component.
@@ -32,6 +34,10 @@ public class ErrorGroup {
     /** Additional prefix that is used in a human-readable format of ignite errors. */
     public static final String ERR_PREFIX = "IGN-";
 
+    /** Error message pattern. */
+    private static final Pattern EXCEPTION_MESSAGE_PATTERN =
+            Pattern.compile("(.*)(IGN)-([A-Z]+)-(\\d+)\\s(TraceId:)([a-f0-9]{8}(?:-[a-f0-9]{4}){4}[a-f0-9]{8})(\\s?)(.*)");
+
     /** List of all registered error groups. */
     private static final Int2ObjectMap<ErrorGroup> registeredGroups = new Int2ObjectOpenHashMap<>();
 
@@ -195,7 +201,7 @@ public class ErrorGroup {
      * @return New error message with predefined prefix.
      */
     public static String errorMessage(UUID traceId, String groupName, int code, String message) {
-        return ERR_PREFIX + groupName + '-' + extractErrorCode(code) + " Trace ID:" + traceId + ((message != null) ? ' ' + message : "");
+        return ERR_PREFIX + groupName + '-' + extractErrorCode(code) + " TraceId:" + traceId + ((message != null) ? ' ' + message : "");
     }
 
     /**
@@ -222,7 +228,13 @@ public class ErrorGroup {
     public static String errorMessageFromCause(UUID traceId, String groupName, int code, Throwable cause) {
         String c = (cause != null && cause.getMessage() != null) ? cause.getMessage() : null;
 
-        return (c != null && c.startsWith(ERR_PREFIX)) ? c :  errorMessage(traceId, groupName, code, c);
+        if (c != null) {
+            Matcher m = EXCEPTION_MESSAGE_PATTERN.matcher(c);
+
+            c = (m.matches()) ? m.group(8) : c;
+        }
+
+        return errorMessage(traceId, groupName, code, c);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroups.java b/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroups.java
index ab765cbd1..3afd77c9e 100755
--- a/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroups.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/ErrorGroups.java
@@ -27,8 +27,15 @@ public class ErrorGroups {
         /** Unknown error group. */
         public static final ErrorGroup COMMON_ERR_GROUP = ErrorGroup.newGroup("CMN", 1);
 
+        /** Unexpected error. */
+        public static final int UNEXPECTED_ERR = COMMON_ERR_GROUP.registerErrorCode(1);
+
+        /** Node stopping error. */
+        public static final int NODE_STOPPING_ERR = COMMON_ERR_GROUP.registerErrorCode(2);
+
         /** Unknown error. */
-        public static final int UNKNOWN_ERR = COMMON_ERR_GROUP.registerErrorCode(1);
+        @Deprecated
+        public static final int UNKNOWN_ERR = COMMON_ERR_GROUP.registerErrorCode(0xFFFF);
     }
 
     /** Tables error group. */
@@ -108,4 +115,43 @@ public class ErrorGroups {
         /** Cursor is already closed error. */
         public static final int CURSOR_CLOSED_ERR = SQL_ERR_GROUP.registerErrorCode(9);
     }
+
+    /** Meta storage error group. */
+    public static class MetaStorage {
+        /** Meta storage error group. */
+        public static final ErrorGroup META_STORAGE_ERR_GROUP = ErrorGroup.newGroup("META", 5);
+
+        /** Failed to start the underlying key value storage. */
+        public static final int STARTING_STORAGE_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(1);
+
+        /** Failed to restore the underlying key value storage. */
+        public static final int RESTORING_STORAGE_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(2);
+
+        /** Failed to close the underlying key value storage. */
+        public static final int CLOSING_STORAGE_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(3);
+
+        /** Failed to compact the underlying key value storage. */
+        public static final int COMPACTION_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(4);
+
+        /** Failed to perform an operation on the underlying key value storage. */
+        public static final int OP_EXECUTION_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(5);
+
+        /** Failed to perform an operation within a specified time period. Usually in such cases the operation should be retried. */
+        public static final int OP_EXECUTION_TIMEOUT_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(6);
+
+        /** Failed to iterate over the underlying key value storage. */
+        public static final int WATCH_EXECUTION_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(7);
+
+        /** Failed to stop a watcher. */
+        public static final int WATCH_STOPPING_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(8);
+
+        /** Failed to deploy or update a watcher. */
+        public static final int DEPLOYING_WATCH_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(9);
+
+        /** Failed to iterate over meta storage cursor. */
+        public static final int CURSOR_EXECUTION_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(10);
+
+        /** Failed to close a cursor. */
+        public static final int CURSOR_CLOSING_ERR = META_STORAGE_ERR_GROUP.registerErrorCode(11);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteTetraFunction.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteTetraFunction.java
new file mode 100755
index 000000000..8f1b4fb6d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteTetraFunction.java
@@ -0,0 +1,48 @@
+/*
+ * 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.lang;
+
+import java.util.function.Function;
+
+/**
+ * Represents a function that accepts four arguments and produces a result.
+ * This is the four-arity specialization of {@link Function}.
+ *
+ * <p>This is a <a href="package-summary.html">functional interface</a>
+ * whose functional method is {@link #apply(Object, Object, Object, Object)}.
+ *
+ * @param <T> the type of the first argument to the function
+ * @param <U> the type of the second argument to the function
+ * @param <V> the type of the third argument to the function
+ * @param <M> the type of the fourth argument to the function
+ * @param <R> the type of the result of the function
+ *
+ * @see Function
+ */
+@FunctionalInterface
+public interface IgniteTetraFunction<T, U, V, M, R> {
+    /**
+     * Applies this function to the given arguments.
+     *
+     * @param t the first function argument
+     * @param u the second function argument
+     * @param v the third function argument
+     * @return the function result
+     */
+    R apply(T t, U u, V v, M m);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteTriFunction.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteTriFunction.java
new file mode 100755
index 000000000..dd23d37e5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteTriFunction.java
@@ -0,0 +1,47 @@
+/*
+ * 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.lang;
+
+import java.util.function.Function;
+
+/**
+ * Represents a function that accepts three arguments and produces a result.
+ * This is the three-arity specialization of {@link Function}.
+ *
+ * <p>This is a <a href="package-summary.html">functional interface</a>
+ * whose functional method is {@link #apply(Object, Object, Object)}.
+ *
+ * @param <T> the type of the first argument to the function
+ * @param <U> the type of the second argument to the function
+ * @param <V> the type of the third argument to the function
+ * @param <R> the type of the result of the function
+ *
+ * @see Function
+ */
+@FunctionalInterface
+public interface IgniteTriFunction<T, U, V, R> {
+    /**
+     * Applies this function to the given arguments.
+     *
+     * @param t the first function argument
+     * @param u the second function argument
+     * @param v the third function argument
+     * @return the function result
+     */
+    R apply(T t, U u, V v);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/NodeStoppingException.java b/modules/core/src/main/java/org/apache/ignite/lang/NodeStoppingException.java
index 7c3407f51..183622952 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/NodeStoppingException.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/NodeStoppingException.java
@@ -17,6 +17,9 @@
 
 package org.apache.ignite.lang;
 
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+
+import java.util.UUID;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -30,7 +33,7 @@ public class NodeStoppingException extends IgniteInternalCheckedException {
      * Creates an empty node stopping exception.
      */
     public NodeStoppingException() {
-        super("Operation has been cancelled (node is stopping).");
+        super(NODE_STOPPING_ERR, "Operation has been cancelled (node is stopping).");
     }
 
     /**
@@ -39,7 +42,7 @@ public class NodeStoppingException extends IgniteInternalCheckedException {
      * @param msg Error message.
      */
     public NodeStoppingException(String msg) {
-        super(msg);
+        super(NODE_STOPPING_ERR, msg);
     }
 
     /**
@@ -48,7 +51,7 @@ public class NodeStoppingException extends IgniteInternalCheckedException {
      * @param cause Non-null throwable cause.
      */
     public NodeStoppingException(Throwable cause) {
-        this(cause.getMessage(), cause);
+        super(NODE_STOPPING_ERR, cause);
     }
 
     /**
@@ -59,7 +62,7 @@ public class NodeStoppingException extends IgniteInternalCheckedException {
      * @param writableStackTrace Whether or not the stack trace should be writable.
      */
     public NodeStoppingException(String msg, @Nullable Throwable cause, boolean writableStackTrace) {
-        super(msg, cause, writableStackTrace);
+        super(UUID.randomUUID(), NODE_STOPPING_ERR, msg, cause, writableStackTrace);
     }
 
     /**
@@ -69,6 +72,6 @@ public class NodeStoppingException extends IgniteInternalCheckedException {
      * @param cause Optional nested exception (can be {@code null}).
      */
     public NodeStoppingException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
+        super(NODE_STOPPING_ERR, msg, cause);
     }
 }
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CompactedException.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CompactedException.java
index 921eb3a37..cf230af2a 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CompactedException.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CompactedException.java
@@ -17,16 +17,20 @@
 
 package org.apache.ignite.internal.metastorage.client;
 
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.COMPACTION_ERR;
+
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
+
 /**
  * Thrown when a requested operation on meta storage could not be performed because target revisions were removed from storage due to a
  * compaction procedure. In such case the operation should be retried with actual revision.
  */
-public class CompactedException extends RuntimeException {
+public class CompactedException extends MetaStorageException {
     /**
      * Constructs an exception.
      */
     public CompactedException() {
-        super();
+        super(COMPACTION_ERR);
     }
 
     /**
@@ -35,7 +39,7 @@ public class CompactedException extends RuntimeException {
      * @param message Detail message.
      */
     public CompactedException(String message) {
-        super(message);
+        super(COMPACTION_ERR, message);
     }
 
     /**
@@ -45,7 +49,7 @@ public class CompactedException extends RuntimeException {
      * @param cause   Cause.
      */
     public CompactedException(String message, Throwable cause) {
-        super(message, cause);
+        super(COMPACTION_ERR, message, cause);
     }
 
     /**
@@ -54,6 +58,6 @@ public class CompactedException extends RuntimeException {
      * @param cause Cause.
      */
     public CompactedException(Throwable cause) {
-        super(cause);
+        super(COMPACTION_ERR, cause);
     }
 }
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java
index 62650ba1d..4fbadae9f 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/CursorImpl.java
@@ -17,6 +17,10 @@
 
 package org.apache.ignite.internal.metastorage.client;
 
+import static org.apache.ignite.internal.util.ExceptionUtils.withCauseAndCode;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_EXECUTION_ERR;
+
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.concurrent.CompletableFuture;
@@ -24,11 +28,11 @@ import java.util.concurrent.ExecutionException;
 import java.util.function.Function;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorCloseCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorHasNextCommand;
 import org.apache.ignite.internal.metastorage.common.command.cursor.CursorNextCommand;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.raft.client.service.RaftGroupService;
@@ -78,13 +82,13 @@ public class CursorImpl<T> implements Cursor<T> {
 
             ((InnerIterator) it).close();
         } catch (InterruptedException | ExecutionException e) {
-            if (e.getCause() != null && e.getCause().getClass().equals(NodeStoppingException.class)) {
+            if (e.getCause() instanceof NodeStoppingException) {
                 return;
             }
 
             LOG.debug("Unable to evaluate cursor close command", e);
 
-            throw new IgniteInternalException(e);
+            throw withCauseAndCode(MetaStorageException::new, CURSOR_CLOSING_ERR, e);
         }
     }
 
@@ -125,13 +129,13 @@ public class CursorImpl<T> implements Cursor<T> {
                             .get();
                 }
             } catch (InterruptedException | ExecutionException e) {
-                if (e.getCause() != null && e.getCause().getClass().equals(NodeStoppingException.class)) {
+                if (e.getCause() instanceof NodeStoppingException) {
                     return false;
                 }
 
                 LOG.debug("Unable to evaluate cursor hasNext command", e);
 
-                throw new IgniteInternalException(e);
+                throw withCauseAndCode(MetaStorageException::new, CURSOR_EXECUTION_ERR, e);
             }
         }
 
@@ -159,19 +163,17 @@ public class CursorImpl<T> implements Cursor<T> {
             } catch (InterruptedException | ExecutionException e) {
                 Throwable cause = e.getCause();
 
-                if (cause != null) {
-                    if (cause.getClass().equals(NodeStoppingException.class)) {
-                        throw new NoSuchElementException();
-                    } else {
-                        if (cause.getClass().equals(NoSuchElementException.class)) {
-                            throw (NoSuchElementException) cause;
-                        }
-                    }
+                if (cause instanceof NodeStoppingException) {
+                    throw new NoSuchElementException();
+                }
+
+                if (cause instanceof NoSuchElementException) {
+                    throw (NoSuchElementException) cause;
                 }
 
                 LOG.debug("Unable to evaluate cursor hasNext command", e);
 
-                throw new IgniteInternalException(e);
+                throw withCauseAndCode(MetaStorageException::new, CURSOR_EXECUTION_ERR, e);
             }
         }
 
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java
index 620abb536..eb2895c7f 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/MetaStorageServiceImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.metastorage.client;
 
 import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_STOPPING_ERR;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -32,6 +33,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.RejectedExecutionException;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.metastorage.common.OperationType;
 import org.apache.ignite.internal.metastorage.common.StatementInfo;
 import org.apache.ignite.internal.metastorage.common.StatementResultInfo;
@@ -498,7 +500,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
 
                                 v.cursor.close();
                             } catch (InterruptedException e) {
-                                throw new IgniteInternalException(e);
+                                throw new MetaStorageException(WATCH_STOPPING_ERR, e);
                             } catch (Exception e) {
                                 if (e instanceof IgniteInternalException && e.getCause().getCause() instanceof RejectedExecutionException) {
                                     LOG.debug("Cursor close command was rejected because raft executor has been already stopped");
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/OperationTimeoutException.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/OperationTimeoutException.java
index 5177d04fa..077f2a5f7 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/OperationTimeoutException.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/OperationTimeoutException.java
@@ -17,15 +17,19 @@
 
 package org.apache.ignite.internal.metastorage.client;
 
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_TIMEOUT_ERR;
+
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
+
 /**
  * Thrown when an operation is not executed within a specified time period. Usually in such cases the operation should be retried.
  */
-public class OperationTimeoutException extends RuntimeException {
+public class OperationTimeoutException extends MetaStorageException {
     /**
      * Constructs an exception.
      */
     public OperationTimeoutException() {
-        super();
+        super(OP_EXECUTION_TIMEOUT_ERR);
     }
 
     /**
@@ -34,7 +38,7 @@ public class OperationTimeoutException extends RuntimeException {
      * @param message Detail message.
      */
     public OperationTimeoutException(String message) {
-        super(message);
+        super(OP_EXECUTION_TIMEOUT_ERR, message);
     }
 
     /**
@@ -44,7 +48,7 @@ public class OperationTimeoutException extends RuntimeException {
      * @param cause   Cause.
      */
     public OperationTimeoutException(String message, Throwable cause) {
-        super(message, cause);
+        super(OP_EXECUTION_TIMEOUT_ERR, message, cause);
     }
 
     /**
@@ -53,6 +57,6 @@ public class OperationTimeoutException extends RuntimeException {
      * @param cause Cause.
      */
     public OperationTimeoutException(Throwable cause) {
-        super(cause);
+        super(OP_EXECUTION_TIMEOUT_ERR, cause);
     }
 }
diff --git a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/StatementResult.java b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/StatementResult.java
index b56ecbcf6..9a345f306 100644
--- a/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/StatementResult.java
+++ b/modules/metastorage-client/src/main/java/org/apache/ignite/internal/metastorage/client/StatementResult.java
@@ -17,7 +17,10 @@
 
 package org.apache.ignite.internal.metastorage.client;
 
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_ERR;
+
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 
 /**
  * Simple result of statement execution, backed by byte[] array.
@@ -95,7 +98,7 @@ public class StatementResult {
     /**
      * Exception to propagate result type conversion issues.
      */
-    public static class ResultConversionException extends RuntimeException {
+    public static class ResultConversionException extends MetaStorageException {
 
         /**
          * Constructs new conversion exception.
@@ -103,7 +106,7 @@ public class StatementResult {
          * @param msg exception message.
          */
         public ResultConversionException(String msg) {
-            super(msg);
+            super(OP_EXECUTION_ERR, msg);
         }
     }
 }
diff --git a/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/MetaStorageException.java b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/MetaStorageException.java
new file mode 100755
index 000000000..816bcebc9
--- /dev/null
+++ b/modules/metastorage-common/src/main/java/org/apache/ignite/internal/metastorage/common/MetaStorageException.java
@@ -0,0 +1,110 @@
+/*
+ * 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.metastorage.common;
+
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteInternalException;
+
+/**
+ * Base class for meta storage exceptions.
+ */
+public class MetaStorageException extends IgniteInternalException {
+    /**
+     * Creates a new exception with the given error code.
+     *
+     * @param code Full error code.
+     */
+    public MetaStorageException(int code) {
+        super(code);
+    }
+
+    /**
+     * Creates a new exception with the given trace id and error code.
+     *
+     * @param traceId Unique identifier of this exception.
+     * @param code Full error code.
+     */
+    public MetaStorageException(UUID traceId, int code) {
+        super(traceId, code);
+    }
+
+    /**
+     * Creates a new exception with the given error code and detail message.
+     *
+     * @param code Full error code.
+     * @param message Detail message.
+     */
+    public MetaStorageException(int code, String message) {
+        super(code, message);
+    }
+
+    /**
+     * Creates a new exception with the given trace id, error code and detail message.
+     *
+     * @param traceId Unique identifier of this exception.
+     * @param code Full error code.
+     * @param message Detail message.
+     */
+    public MetaStorageException(UUID traceId, int code, String message) {
+        super(traceId, code, message);
+    }
+
+    /**
+     * Creates a new exception with the given error code and cause.
+     *
+     * @param code Full error code.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public MetaStorageException(int code, Throwable cause) {
+        super(code, cause);
+    }
+
+    /**
+     * Creates a new exception with the given trace id, error code and cause.
+     *
+     * @param traceId Unique identifier of this exception.
+     * @param code Full error code.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public MetaStorageException(UUID traceId, int code, Throwable cause) {
+        super(traceId, code, cause);
+    }
+
+    /**
+     * Creates a new exception with the given error code, detail message and cause.
+     *
+     * @param code Full error code.
+     * @param message Detail message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public MetaStorageException(int code, String message, Throwable cause) {
+        super(code, message, cause);
+    }
+
+    /**
+     * Creates a new exception with the given trace id, error code, detail message and cause.
+     *
+     * @param traceId Unique identifier of this exception.
+     * @param code Full error code.
+     * @param message Detail message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public MetaStorageException(UUID traceId, int code, String message, Throwable cause) {
+        super(traceId, code, message, cause);
+    }
+}
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
index 989dddf71..1d7a838e9 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
@@ -31,6 +31,10 @@ import static org.apache.ignite.internal.rocksdb.RocksUtils.find;
 import static org.apache.ignite.internal.rocksdb.RocksUtils.forEach;
 import static org.apache.ignite.internal.rocksdb.snapshot.ColumnFamilyRange.fullRange;
 import static org.apache.ignite.internal.util.ArrayUtils.LONG_EMPTY_ARRAY;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.COMPACTION_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.RESTORING_STORAGE_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.STARTING_STORAGE_ERR;
 
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Path;
@@ -47,6 +51,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.metastorage.server.Condition;
 import org.apache.ignite.internal.metastorage.server.Entry;
 import org.apache.ignite.internal.metastorage.server.If;
@@ -63,7 +68,6 @@ import org.apache.ignite.internal.rocksdb.snapshot.RocksSnapshotManager;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteInternalException;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.jetbrains.annotations.TestOnly;
@@ -164,7 +168,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
             // Delete existing data, relying on the raft's snapshot and log playback
             recreateDb();
         } catch (RocksDBException e) {
-            throw new IgniteInternalException("Failed to start the storage", e);
+            throw new MetaStorageException(STARTING_STORAGE_ERR, "Failed to start the storage", e);
         }
     }
 
@@ -251,7 +255,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             updCntr = bytesToLong(data.get(UPDATE_COUNTER_KEY));
         } catch (Exception e) {
-            throw new IgniteInternalException("Failed to restore snapshot", e);
+            throw new MetaStorageException(RESTORING_STORAGE_ERR, "Failed to restore snapshot", e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -285,7 +289,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             fillAndWriteBatch(batch, curRev, cntr);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -306,7 +310,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
             // Store the new value
             index.put(batch, key, appendLong(array, curRev));
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         }
     }
 
@@ -353,7 +357,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
             // Return previous value.
             return doGetValue(key, lastRev);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -375,7 +379,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             fillAndWriteBatch(batch, curRev, counter);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -402,7 +406,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             fillAndWriteBatch(batch, curRev, counter);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -465,7 +469,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
                 fillAndWriteBatch(batch, curRev, counter);
             }
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -516,7 +520,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             fillAndWriteBatch(batch, curRev, counter);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -559,7 +563,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             fillAndWriteBatch(batch, curRev, counter);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -583,7 +587,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             return branch;
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -600,7 +604,8 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             while (true) {
                 if (maximumNumOfNestedBranch-- <= 0) {
-                    throw new IgniteInternalException(
+                    throw new MetaStorageException(
+                            OP_EXECUTION_ERR,
                             "Too many nested (" + maximumNumOfNestedBranch + ") statements in multi-invoke command.");
                 }
 
@@ -619,7 +624,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
                 }
             }
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -669,7 +674,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
                         break;
 
                     default:
-                        throw new IllegalArgumentException("Unknown operation type: " + op.type());
+                        throw new MetaStorageException(OP_EXECUTION_ERR, "Unknown operation type: " + op.type());
                 }
             }
 
@@ -742,7 +747,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             fillAndWriteBatch(batch, rev, updCntr);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(COMPACTION_ERR, e);
         } finally {
             rwLock.writeLock().unlock();
         }
@@ -841,7 +846,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
         try {
             revs = getRevisions(key);
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         }
 
         if (revs == null || revs.length == 0) {
@@ -919,7 +924,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
         try {
             valueBytes = data.get(keyToRocksKey(revision, key));
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         }
 
         if (valueBytes == null || valueBytes.length == 0) {
@@ -1025,7 +1030,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
 
             return new IgniteBiTuple<>(iterator.key(), getAsLongs(iterator.value()));
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(OP_EXECUTION_ERR, e);
         }
     }
 
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
index c3dabd4aa..2ac2b730f 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
@@ -22,11 +22,13 @@ import static org.apache.ignite.internal.metastorage.server.persistence.RocksSto
 import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.revisionFromRocksKey;
 import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.rocksKeyToBytes;
 import static org.apache.ignite.internal.rocksdb.RocksUtils.checkIterator;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_EXECUTION_ERR;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.function.Predicate;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.metastorage.server.Entry;
 import org.apache.ignite.internal.metastorage.server.EntryEvent;
 import org.apache.ignite.internal.metastorage.server.Value;
@@ -34,7 +36,6 @@ import org.apache.ignite.internal.metastorage.server.WatchEvent;
 import org.apache.ignite.internal.rocksdb.RocksUtils;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.lang.IgniteInternalException;
 import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.RocksIterator;
@@ -111,7 +112,7 @@ class WatchCursor implements Cursor<WatchEvent> {
 
             return currentHasNext;
         } catch (RocksDBException e) {
-            throw new IgniteInternalException(e);
+            throw new MetaStorageException(WATCH_EXECUTION_ERR, e);
         } finally {
             storage.lock().readLock().unlock();
         }
diff --git a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
index 141f0f64f..8e79e416f 100644
--- a/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
+++ b/modules/metastorage-server/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.metastorage.server.raft;
 
 import static java.util.Objects.requireNonNull;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CLOSING_STORAGE_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
 
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -30,6 +32,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Consumer;
 import org.apache.ignite.internal.metastorage.common.ConditionType;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.metastorage.common.StatementInfo;
 import org.apache.ignite.internal.metastorage.common.StatementResultInfo;
 import org.apache.ignite.internal.metastorage.common.UpdateInfo;
@@ -77,7 +80,6 @@ import org.apache.ignite.internal.metastorage.server.Update;
 import org.apache.ignite.internal.metastorage.server.ValueCondition;
 import org.apache.ignite.internal.metastorage.server.WatchEvent;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.raft.client.Command;
 import org.apache.ignite.raft.client.ReadCommand;
@@ -328,7 +330,7 @@ public class MetaStorageListener implements RaftGroupListener {
                 try {
                     cursorDesc.cursor().close();
                 } catch (Exception e) {
-                    throw new IgniteInternalException(e);
+                    throw new MetaStorageException(CURSOR_CLOSING_ERR, e);
                 }
 
                 clo.result(null);
@@ -381,7 +383,7 @@ public class MetaStorageListener implements RaftGroupListener {
                         try {
                             cursorDesc.cursor().close();
                         } catch (Exception e) {
-                            throw new IgniteInternalException(e);
+                            throw new MetaStorageException(CURSOR_CLOSING_ERR, e);
                         }
 
                         cursorsIter.remove();
@@ -417,7 +419,7 @@ public class MetaStorageListener implements RaftGroupListener {
         try {
             storage.close();
         } catch (Exception e) {
-            throw new IgniteInternalException("Failed to close storage: " + e.getMessage(), e);
+            throw new MetaStorageException(CLOSING_STORAGE_ERR, "Failed to close storage: " + e.getMessage(), e);
         }
     }
 
diff --git a/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
index 71f202314..de32afe82 100644
--- a/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
+++ b/modules/metastorage-server/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.metastorage.server;
 
 import static org.apache.ignite.internal.metastorage.server.Value.TOMBSTONE;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_ERR;
 
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -33,6 +34,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Predicate;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.util.Cursor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -291,7 +293,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
                         break;
 
                     default:
-                        throw new IllegalArgumentException("Unknown operation type: " + op.type());
+                        throw new MetaStorageException(OP_EXECUTION_ERR, "Unknown operation type: " + op.type());
                 }
             }
 
@@ -335,7 +337,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
                                 break;
 
                             default:
-                                throw new IllegalArgumentException("Unknown operation type: " + op.type());
+                                throw new MetaStorageException(OP_EXECUTION_ERR, "Unknown operation type: " + op.type());
                         }
                     }
 
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index 163762ffb..542b3e544 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -19,6 +19,9 @@ package org.apache.ignite.internal.metastorage;
 
 import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
 import static org.apache.ignite.internal.util.ByteUtils.longToBytes;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_EXECUTION_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.DEPLOYING_WATCH_ERR;
 import static org.apache.ignite.network.util.ClusterServiceUtils.resolveNodes;
 
 import java.util.Collection;
@@ -42,6 +45,7 @@ import org.apache.ignite.internal.metastorage.client.Operation;
 import org.apache.ignite.internal.metastorage.client.OperationTimeoutException;
 import org.apache.ignite.internal.metastorage.client.StatementResult;
 import org.apache.ignite.internal.metastorage.client.WatchListener;
+import org.apache.ignite.internal.metastorage.common.MetaStorageException;
 import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
 import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
 import org.apache.ignite.internal.metastorage.watch.AggregatedWatch;
@@ -55,7 +59,6 @@ import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.vault.VaultManager;
 import org.apache.ignite.lang.ByteArray;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.network.ClusterNode;
@@ -862,7 +865,7 @@ public class MetaStorageManager implements IgniteComponent {
         appliedRevision()
                 .thenCompose(appliedRevision -> {
                     if (revision <= appliedRevision) {
-                        throw new IgniteInternalException(String.format(
+                        throw new MetaStorageException(DEPLOYING_WATCH_ERR, String.format(
                                 "Current revision (%d) must be greater than the revision in the Vault (%d)",
                                 revision, appliedRevision
                         ));
@@ -911,7 +914,7 @@ public class MetaStorageManager implements IgniteComponent {
                     try {
                         cursor.close();
                     } catch (Exception e) {
-                        throw new IgniteInternalException(e);
+                        throw new MetaStorageException(CURSOR_CLOSING_ERR, e);
                     }
                 }).get();
             } finally {
@@ -930,7 +933,7 @@ public class MetaStorageManager implements IgniteComponent {
                 try {
                     return innerIterFut.thenApply(Iterator::hasNext).get();
                 } catch (InterruptedException | ExecutionException e) {
-                    throw new IgniteInternalException(e);
+                    throw new MetaStorageException(CURSOR_EXECUTION_ERR, e);
                 }
             } finally {
                 busyLock.leaveBusy();
@@ -948,7 +951,7 @@ public class MetaStorageManager implements IgniteComponent {
                 try {
                     return innerIterFut.thenApply(Iterator::next).get();
                 } catch (InterruptedException | ExecutionException e) {
-                    throw new IgniteInternalException(e);
+                    throw new MetaStorageException(CURSOR_EXECUTION_ERR, e);
                 }
             } finally {
                 busyLock.leaveBusy();
@@ -957,7 +960,7 @@ public class MetaStorageManager implements IgniteComponent {
     }
 
     /**
-     * Dispatches appropriate metastorage watch method according to inferred watch criterion.
+     * Dispatches appropriate meta storage watch method according to inferred watch criterion.
      *
      * @param aggregatedWatch Aggregated watch.
      * @return Future, which will be completed after new watch registration finished.