You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "sk0x50 (via GitHub)" <gi...@apache.org> on 2023/06/20 11:47:50 UTC

[GitHub] [ignite-3] sk0x50 opened a new pull request, #2225: IGNITE-19539 Added exception mapper utility

sk0x50 opened a new pull request, #2225:
URL: https://github.com/apache/ignite-3/pull/2225

   https://issues.apache.org/jira/browse/IGNITE-19539


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236708527


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMappersProvider.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.Collection;
+
+/**
+ * This interface provides the ability to register specific mappers from Ignite internal exceptions a public ones.
+ *
+ * <p>
+ *     Each module can provide such a mapping for its own internal exceptions.
+ *     Designed for integration with {@link java.util.ServiceLoader} mechanism, so IgniteExceptionMapper instances
+ *     provided by a library are to be defined as services either
+ *     in {@code META-INF/services/org.apache.ignite.lang.IgniteExceptionMapper}, or in a {@code module-info.java}.
+ * </p>
+ *
+ * <p>There are the following constraints that should be taken into account by a particular implementation of this interface:</p>
+ * <ul>
+ *     <li>it is prohibited to register more than one mapper for the same internal exception.</li>
+ *     <li>mapper should only provide mappings either to {@link IgniteException}, or {@link IgniteCheckedException}.</li>
+ *     <li>mapper should not provide mapping for Java standard exception like {@link NullPointerException},
+ *     {@link IllegalArgumentException}, etc.</li>
+ *     <li>mapper should not provide any mappings for errors {@link Error}.</li>
+ * </ul>
+ */
+public interface IgniteExceptionMappersProvider {
+    /**
+     * Returns a collection of mappers to be used to map internal exceptions to public ones.
+     *
+     * @return Collection of mappers.
+     */
+    Collection<IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers();

Review Comment:
   Same one about the signature. I get that explicit is better than implicit, but this name is too long to justify the argument



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236832620


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperFuture.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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 static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.mapToPublicException;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * This class represents a {@link CompletableFuture}, the main purpose of this future is to automatically map
+ * internal exceptions to public ones on completion.
+ *
+ * @param <T> The result type.
+ */
+public class IgniteExceptionMapperFuture<T> extends CompletableFuture<T> {
+    /** Internal future, whose result will be used to map to a public exception if this delegate completed exceptionally. */
+    private final CompletableFuture<T> delegate;
+
+    /**
+     * Creates a new instance {@link IgniteExceptionMapperFuture} with the given {@code delegate}
+     * as an underlying future that serves all requests.
+     *
+     * @param delegate Future to be wrapped.
+     * @param <U> The result Type.
+     * @return New instance {@link IgniteExceptionMapperFuture}.
+     */
+    public static <U> IgniteExceptionMapperFuture<U> of(CompletableFuture<U> delegate) {

Review Comment:
   This makes sense to me. I will provide a utility method instead of this future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236972592


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);

Review Comment:
   I don't think so. Let's assume that we have hundreds of codes (it is an absolutely possible case). Should we create hundreds of exception classes? IMHO, it is OK to map several error codes to one exception class, assuming that these error codes relate to the one error group of course.
   
   In this particular case, we can introduce a new type with a proper name, I think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236963103


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new instance of public exception provided by the {@code mapper}.
+     *
+     * @param mapper Mapper function to produce a public exception.
+     * @param t Internal exception.
+     * @param <T> Type of an internal exception.
+     * @param <R> Type of a public exception.
+     * @return New public exception.
+     */
+    private static <T extends Exception, R extends Exception> Exception map(IgniteExceptionMapper<T, R> mapper, Throwable t) {
+        return mapper.map(mapper.mappingFrom().cast(t));
+    }
+
+    /**
+     * Returns {@code true} if the given exception {@code t} is instance of {@link NullPointerException}, {@link IllegalArgumentException}.
+     *
+     * @return {@code true} if the given exception is Java standard exception.
+     */
+    private static boolean isJavaStandardException(Throwable t) {
+        return t instanceof NullPointerException || t instanceof IllegalArgumentException;

Review Comment:
   Well, I think that we should not support any of them, but for now, there are interfaces that use the following approach (just an example):
   ```
       /**
        * Creates a new distribution zone with the given {@code name} asynchronously.
        *
        * @param distributionZoneCfg Distribution zone configuration.
        * @return Future with the id of the zone. 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
        *      or distribution zone name is {@code DEFAULT_ZONE_NAME},
        *      {@link NodeStoppingException} if the node is stopping.
        */
       public CompletableFuture<Integer> createZone(DistributionZoneConfigurationParameters distributionZoneCfg) {
   ```
   
   In that particular case, `IllegalArgumentException` should not be wrapped into `IgniteException` with the `INTERNAL_ERR` code. I don't think that this approach is good enough. IMHO, we should check incoming parameters (and throw an exception if it is needed) before creating an internal future. On the other hand, we can introduce special mapping rules to handle such cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242448068


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());

Review Comment:
   It's correct from the substitution principle. Alternative here, the one you've implemented, is converting SpecificExceptionB to root IgniteException, that should be considered incorrect according to aforementioned logic, meaning we have A extends B extends C, and B to B_public mapper along with anything to C_public. You claim that it's incorrect to consider A as B and thus convert it to B_public however it's correct to consider A as C and thus convert it to C_public, where C_public is IgniteException in this particular case.That seems confusing.
   In real life that would actually mean that in case of TransactionInternalCheckedException->TransactionException mapper we will convert LockException extends TransactionInternalCheckedException to IgniteException instead of TransactionException, that is more specific and thus better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242111422


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new CompletableFuture that, when the given {@code origin} future completes exceptionally,
+     * maps the origin's exception to a public Ignite exception if it is needed.
+     *
+     * @param origin The future to use to create a new stage.
+     * @param <T> Type os result.
+     * @return New CompletableFuture.
+     */
+    public static <T> CompletableFuture<T> convertToPublicFuture(CompletableFuture<T> origin) {
+        return origin
+                .handle((res, err) -> {
+                    if (err != null) {
+                        throw new CompletionException(mapToPublicException(unwrapCause(err)));

Review Comment:
   Could you please provide an example? I didn't get the idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242072073


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {

Review Comment:
   Ok, will change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 merged pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 merged PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236972592


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);

Review Comment:
   I don't think so. Let's assume that we have hundreds of codes (it is an absolutely possible case). Should we create hundreds of exception classes? IMHO, it is OK to map several error codes to one exception class, assuming that these error codes relate to the same error group of course.
   
   In this particular case, we can introduce a new type with a proper name, I think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1243209920


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new CompletableFuture that, when the given {@code origin} future completes exceptionally,
+     * maps the origin's exception to a public Ignite exception if it is needed.
+     *
+     * @param origin The future to use to create a new stage.
+     * @param <T> Type os result.
+     * @return New CompletableFuture.
+     */
+    public static <T> CompletableFuture<T> convertToPublicFuture(CompletableFuture<T> origin) {
+        return origin
+                .handle((res, err) -> {
+                    if (err != null) {
+                        throw new CompletionException(mapToPublicException(unwrapCause(err)));

Review Comment:
   I mean that it's possible to have CompletionException(ExecutionException(NPE)) or similar. However unwrapCause actually flattens stack on it's own:
   ```
       public static Throwable unwrapCause(Throwable e) {
           while ((e instanceof CompletionException || e instanceof ExecutionException) && e.getCause() != null) {
               e = e.getCause();
           }
   
           return e;
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242434599


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {
+        return mapper.apply(error);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public runtime exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public runtime exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteException> IgniteExceptionMapper<T, R> unchecked(

Review Comment:
   Yep, ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236963103


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new instance of public exception provided by the {@code mapper}.
+     *
+     * @param mapper Mapper function to produce a public exception.
+     * @param t Internal exception.
+     * @param <T> Type of an internal exception.
+     * @param <R> Type of a public exception.
+     * @return New public exception.
+     */
+    private static <T extends Exception, R extends Exception> Exception map(IgniteExceptionMapper<T, R> mapper, Throwable t) {
+        return mapper.map(mapper.mappingFrom().cast(t));
+    }
+
+    /**
+     * Returns {@code true} if the given exception {@code t} is instance of {@link NullPointerException}, {@link IllegalArgumentException}.
+     *
+     * @return {@code true} if the given exception is Java standard exception.
+     */
+    private static boolean isJavaStandardException(Throwable t) {
+        return t instanceof NullPointerException || t instanceof IllegalArgumentException;

Review Comment:
   Well, I think that should not support any of them, but for now, there are interfaces that use the following approach (just an example):
   ```
       /**
        * Creates a new distribution zone with the given {@code name} asynchronously.
        *
        * @param distributionZoneCfg Distribution zone configuration.
        * @return Future with the id of the zone. 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
        *      or distribution zone name is {@code DEFAULT_ZONE_NAME},
        *      {@link NodeStoppingException} if the node is stopping.
        */
       public CompletableFuture<Integer> createZone(DistributionZoneConfigurationParameters distributionZoneCfg) {
   ```
   
   In that particular case, `IllegalArgumentException` should not be wrapped into `IgniteException` with the `INTERNAL_ERR` code. I don't think that this approach is good enough. IMHO, we should check incoming parameters (and throw an exception if it is needed) before creating an internal future. On the other hand, we can introduce special mapping rules to handle such cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242109276


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());

Review Comment:
   Well, It is arguable. IMHO, a particular component should provide a comprehensive list of mappers. Let's consider the following case:
   ```
       public class ComponentException {};
       public class SpecificExceptionA extends ComponentException {};
   
       // @throws SpecificExceptionA If ...
       publicApiCall() throws SpecificExceptionA
   ```
   and after that, we decided to introduce a new type of exception to clarify the behavior in some particular case:
   ```
       public class SpecificExceptionB extends ComponentException {};
       or
       public class SpecificExceptionB extends SpecificExceptionA{};
   
       // @throws SpecificExceptionA If ...
       // @throws SpecificExceptionB If ...
       publicApiCall() throws SpecificExceptionA
   ```
   and our component does not provide a mapping from InternalSpecificExceptionB to SpecificExceptionB.
   
   In this case, the user will observe SpecificExceptionA or  ComponentException instead of SpecificExceptionB.
   IMHO, this behavior would not be correct.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1243283024


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());

Review Comment:
   > I would say, this is a bug caused by the absence of the required mapping and we should clearly state that it is incorrect behavior via throwing IgniteException(INTERNAL_ERR) I think.
   
   Why it's a bug? Is it mandatory to have explicit internal to public mappers for all internal exceptions? Why?
   
   > When ComponentException is a checked exception, the situation is even worse - the code should not be compiled:
   
   What do you mean? You either expect ComponentException to be thrown and thus add it to throws clause, or provide required mapping.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1243236656


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());

Review Comment:
   Well, I don't think so, just because the logic you proposed breaks handling exceptions on the user's side, and even cannot be compiled in case of checked exceptions.
   
   Let's consider the following examples:
   (1)
   ```
   Public hierarchy:
       ComponentException extends IgniteException;
       SpecificExceptionA extends ComponentException
       SpecificExceptionB extends ComponentException
   
   Internal hierarchy:
       InternalComponentException extends IgniteInternalException;
       InternalSpecificExceptionA extends InternalComponentException
       InternalSpecificExceptionB extends InternalComponentException
   
   Mappings:
       InternalComponentException -> ComponentException
       InternalSpecificExceptionA -> SpecificExceptionA
       mapping for InternalSpecificExceptionB is absent due to a bug
   ```
   ```
   Public api:
       public void apiCall() throws SpecificExceptionA, SpecificExceptionB;
   
       try {
           apiCall();
       catch (SpecificExceptionB e) {
           // handle specific case (B)
       } catch (SpecificExceptionA e) {
           // handle specific case (A)
       } catch (ComponentException e) {
           // optional catch block, just to illustrate the issue (C)
       }
   ```
   
   If I understand your proposal correctly, the mapper should convert InternalSpecificExceptionB to ComponentException, because the most specific mapping is InternalComponentException -> ComponentException and InternalSpecificExceptionB extends InternalComponentException.
   
   In this case, exception handler (B) will be skipped at all, only (C) will be executed. I would say, this is a bug caused by the absence of the required mapping and we should clearly state that it is incorrect behavior via throwing `IgniteException(INTERNAL_ERR)` I think.
   
   When ComponentException is a checked exception, the situation is even worse - the code should not be compiled:
   ```
       public static void apiCall() throws SpecificExceptionA, SpecificExceptionB {
           ...
           throw new ComponentException();
       }
   ```
   
   (2) 
   ```
   Public hierarchy:
       ComponentException extends IgniteException;
       SpecificExceptionA extends ComponentException
       SpecificExceptionB extends SpecificExceptionA
   ```
   
   The fact, that the right handler is skipped, is still in play.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236712924


##########
modules/api/src/test/java/org/apache/ignite/lang/IgniteExceptionMapperUtilTest.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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 static org.apache.ignite.lang.ErrorGroups.Common.COMMON_ERR_GROUP;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+import static org.apache.ignite.lang.IgniteExceptionMapper.checked;
+import static org.apache.ignite.lang.IgniteExceptionMapper.unchecked;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.mapToPublicException;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.registerMapping;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Stream;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests mapping internal exceptions to public ones.
+ */
+public class IgniteExceptionMapperUtilTest {
+    /** Internal collection of mappers for tests. */
+    private Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();

Review Comment:
   You get my point, please make types shorter



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1245139714


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());

Review Comment:
   As was disccussed with @ibessonov and @sanpwc the improvement of the mapping rules will be partially addressed by https://issues.apache.org/jira/browse/IGNITE-19871. All other questions mentioned in this topic will be postponed until the real use cases shows the need.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1241987943


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {
+        return mapper.apply(error);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public runtime exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public runtime exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteException> IgniteExceptionMapper<T, R> unchecked(

Review Comment:
   This method seems strange. It's either mapper to be returned or converted exception. In first case we need only Function<T, R> mapper in second we may have both from and mapper params but should return R. What's the point of given method?



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {

Review Comment:
   Why it's an error? Meaning that there's an [Error](https://docs.oracle.com/javase/8/docs/api/?java/lang/Error.html) in java Throwable hierarchy, thus it might be a bit confusing.



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new instance of public exception provided by the {@code mapper}.
+     *
+     * @param mapper Mapper function to produce a public exception.
+     * @param t Internal exception.
+     * @param <T> Type of an internal exception.
+     * @param <R> Type of a public exception.
+     * @return New public exception.
+     */
+    private static <T extends Exception, R extends Exception> Exception map(IgniteExceptionMapper<T, R> mapper, Throwable t) {
+        return mapper.map(mapper.mappingFrom().cast(t));
+    }
+
+    /**
+     * Returns {@code true} if the given exception {@code t} is instance of {@link NullPointerException}, {@link IllegalArgumentException}.
+     *
+     * @return {@code true} if the given exception is Java standard exception.
+     */
+    private static boolean isJavaStandardException(Throwable t) {
+        return t instanceof NullPointerException || t instanceof IllegalArgumentException;

Review Comment:
   Not sure that I got an idea. What about all that common IllegalStateException, IllegalArgumentException etc? You've mentioned that 
   
   > I think that we should not support any of them
   
   But still we consider NPE as valid internalException to throw in public.



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {
+        return mapper.apply(error);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public runtime exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public runtime exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteException> IgniteExceptionMapper<T, R> unchecked(
+            Class<T> from,
+            Function<T, R> mapper
+    ) {
+        return new IgniteExceptionMapper<T, R>(from, mapper);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public checked exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public checked exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteCheckedException> IgniteExceptionMapper<T, R> checked(

Review Comment:
   Same as above.



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());

Review Comment:
   Why do we check only exact matching? Let's say InternalTransactionException -> PublicTransactionException mapper was registered. And we have SomeSpecficInternalTransactionException extends InternalTransactionException. Do we expect this SomeSpecficInternalTransactionException to be converted to PublicTransactionException? I believe we should, because from the hierarchy point of view SomeSpecficInternalTransactionException is an InternalTransactionException. 
   Thus I believe we should find the most specific mapper with a top level IgniteException(INTERNAL_ERR, origin);



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<?, ?> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<?, ?>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that will always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new CompletableFuture that, when the given {@code origin} future completes exceptionally,
+     * maps the origin's exception to a public Ignite exception if it is needed.
+     *
+     * @param origin The future to use to create a new stage.
+     * @param <T> Type os result.
+     * @return New CompletableFuture.
+     */
+    public static <T> CompletableFuture<T> convertToPublicFuture(CompletableFuture<T> origin) {
+        return origin
+                .handle((res, err) -> {
+                    if (err != null) {
+                        throw new CompletionException(mapToPublicException(unwrapCause(err)));

Review Comment:
   There's still a gap here. In case of Future of future we may have CompletionException of similar with InternalException inside. So I believe we should flat all top level futures here.



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionUtils.java:
##########
@@ -177,14 +177,15 @@ public static int extractCodeFrom(Throwable t) {
         return INTERNAL_ERR;
     }
 
-    // TODO: https://issues.apache.org/jira/browse/IGNITE-19539 this method should be removed or re-worked.
+    // TODO: This method should be removed or re-worked and usages should be changed to IgniteExceptionMapperUtil.mapToPublicException.

Review Comment:
   I don't think that it's valid to have todo without ignite link.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242080574


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {
+        return mapper.apply(error);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public runtime exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public runtime exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteException> IgniteExceptionMapper<T, R> unchecked(

Review Comment:
   The main goal is to restrict the type of mapper. The exception mapper should provide either a mapping Exception -> IgniteException (unchecked) or Exception -> IgniteCheckedException (checked). The mapper should not try to provide a mapping Exception -> IgniteInternalException, for instance.
   Java generics do not allow using something as follows:
   ```
       <T, R extends IgniteException> Function<T, R> mapper();
       <T, R extends IgniteCheckedException> Function<T, R> mapper();
   ```
   just because these methods have exactly the same `erasure`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242080574


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {
+        return mapper.apply(error);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public runtime exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public runtime exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteException> IgniteExceptionMapper<T, R> unchecked(

Review Comment:
   The main goal is to restrict the type of mapper. The exception mapper should provide either a mapping Exception -> IgniteException (unchecked) or Exception -> IgniteCheckedException (checked). The mapper should not try to provide a mapping Exception -> IgniteInternalException, for instance.
   Java generics do not allow using something as follows:
   ```
       <T, R extends IgniteException> Function<T, R> mapper();
       <T, R extends IgniteCheckedException> Function<T, R> mapper();
   ```
   just because these methods have exactly the same `erasure`
   
   The method `IgniteExceptionMapper.mappingFrom` allows us to avoid incorrect registration of the mapper:
   ```
       Function<CustomExceptionB, IgniteException> mapper = err -> ...;
       mappers.put(CustomExceptionA.class, mapper);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1236693712


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperFuture.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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 static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.mapToPublicException;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * This class represents a {@link CompletableFuture}, the main purpose of this future is to automatically map
+ * internal exceptions to public ones on completion.
+ *
+ * @param <T> The result type.
+ */
+public class IgniteExceptionMapperFuture<T> extends CompletableFuture<T> {
+    /** Internal future, whose result will be used to map to a public exception if this delegate completed exceptionally. */
+    private final CompletableFuture<T> delegate;
+
+    /**
+     * Creates a new instance {@link IgniteExceptionMapperFuture} with the given {@code delegate}
+     * as an underlying future that serves all requests.
+     *
+     * @param delegate Future to be wrapped.
+     * @param <U> The result Type.
+     * @return New instance {@link IgniteExceptionMapperFuture}.
+     */
+    public static <U> IgniteExceptionMapperFuture<U> of(CompletableFuture<U> delegate) {

Review Comment:
   Why not returning `delegate.handle(...)`? This entire subclass does nothing useful as far as I see.
   If it does, it must be documented. Please consider removing the inheritance and making this class utilitarian.



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);

Review Comment:
   Shouldn't we have a specific exception class for each error code?



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,

Review Comment:
   Given that `Exception` is already an upped bound for type parameters, this could be safely replaced with `IgniteExceptionMapper<?, ?>`.



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMappersProvider.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.Collection;
+
+/**
+ * This interface provides the ability to register specific mappers from Ignite internal exceptions a public ones.
+ *
+ * <p>
+ *     Each module can provide such a mapping for its own internal exceptions.
+ *     Designed for integration with {@link java.util.ServiceLoader} mechanism, so IgniteExceptionMapper instances
+ *     provided by a library are to be defined as services either
+ *     in {@code META-INF/services/org.apache.ignite.lang.IgniteExceptionMapper}, or in a {@code module-info.java}.
+ * </p>
+ *
+ * <p>There are the following constraints that should be taken into account by a particular implementation of this interface:</p>
+ * <ul>
+ *     <li>it is prohibited to register more than one mapper for the same internal exception.</li>
+ *     <li>mapper should only provide mappings either to {@link IgniteException}, or {@link IgniteCheckedException}.</li>
+ *     <li>mapper should not provide mapping for Java standard exception like {@link NullPointerException},
+ *     {@link IllegalArgumentException}, etc.</li>
+ *     <li>mapper should not provide any mappings for errors {@link Error}.</li>
+ * </ul>
+ */
+public interface IgniteExceptionMappersProvider {
+    /**
+     * Returns a collection of mappers to be used to map internal exceptions to public ones.
+     *
+     * @return Collection of mappers.
+     */
+    Collection<IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers();

Review Comment:
   Same not about the signature. I get that explicit is better than implicit, but this name is too long to justify the argument



##########
modules/api/src/test/java/org/apache/ignite/lang/IgniteExceptionMapperUtilTest.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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 static org.apache.ignite.lang.ErrorGroups.Common.COMMON_ERR_GROUP;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+import static org.apache.ignite.lang.IgniteExceptionMapper.checked;
+import static org.apache.ignite.lang.IgniteExceptionMapper.unchecked;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.mapToPublicException;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.registerMapping;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Stream;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests mapping internal exceptions to public ones.
+ */
+public class IgniteExceptionMapperUtilTest {
+    /** Internal collection of mappers for tests. */
+    private Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();

Review Comment:
   I get my point, please make types shorter



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new instance of public exception provided by the {@code mapper}.
+     *
+     * @param mapper Mapper function to produce a public exception.
+     * @param t Internal exception.
+     * @param <T> Type of an internal exception.
+     * @param <R> Type of a public exception.
+     * @return New public exception.
+     */
+    private static <T extends Exception, R extends Exception> Exception map(IgniteExceptionMapper<T, R> mapper, Throwable t) {
+        return mapper.map(mapper.mappingFrom().cast(t));
+    }
+
+    /**
+     * Returns {@code true} if the given exception {@code t} is instance of {@link NullPointerException}, {@link IllegalArgumentException}.
+     *
+     * @return {@code true} if the given exception is Java standard exception.
+     */
+    private static boolean isJavaStandardException(Throwable t) {
+        return t instanceof NullPointerException || t instanceof IllegalArgumentException;

Review Comment:
   Interesting. Is this all that we support?



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>

Review Comment:
   "...that _will_ always be mapped..."? I think that some word is missing here



##########
modules/api/src/test/java/org/apache/ignite/lang/IgniteExceptionMapperUtilTest.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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 static org.apache.ignite.lang.ErrorGroups.Common.COMMON_ERR_GROUP;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
+import static org.apache.ignite.lang.IgniteExceptionMapper.checked;
+import static org.apache.ignite.lang.IgniteExceptionMapper.unchecked;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.mapToPublicException;
+import static org.apache.ignite.lang.IgniteExceptionMapperUtil.registerMapping;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Stream;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests mapping internal exceptions to public ones.
+ */
+public class IgniteExceptionMapperUtilTest {
+    /** Internal collection of mappers for tests. */
+    private Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+    /**
+     * Tests a simple scenario od registering mapper for internal exceptions.

Review Comment:
   ```suggestion
        * Tests a simple scenario of registering mapper for internal exceptions.
   ```



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMappersProvider.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.Collection;
+
+/**
+ * This interface provides the ability to register specific mappers from Ignite internal exceptions a public ones.

Review Comment:
   ''...exceptions _to_ a public ones", I guess. I think a word is missing here as well



##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {

Review Comment:
   Same here, shorter types are easier to read



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242112542


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionUtils.java:
##########
@@ -177,14 +177,15 @@ public static int extractCodeFrom(Throwable t) {
         return INTERNAL_ERR;
     }
 
-    // TODO: https://issues.apache.org/jira/browse/IGNITE-19539 this method should be removed or re-worked.
+    // TODO: This method should be removed or re-worked and usages should be changed to IgniteExceptionMapperUtil.mapToPublicException.

Review Comment:
   Yep, you are right. Will create a ticket.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242093277


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapper.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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 mapper from an internal exception {@code T} to a public one {@code R}.
+ */
+public class IgniteExceptionMapper<T extends Exception, R extends Exception> {
+    /** Class that represents an internal exception for mapping. */
+    private final Class<T> from;
+
+    /** Mapping function. */
+    private final Function<T, R> mapper;
+
+    /**
+     * Creates a new instance of mapper.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     */
+    private IgniteExceptionMapper(Class<T> from, Function<T, R> mapper) {
+        this.from = from;
+        this.mapper = mapper;
+    }
+
+    /**
+     * Returns a class instance that represents an internal exception to be used for mapping.
+     *
+     * @return Class instance that represents an internal exception to be used for mapping.
+     */
+    public Class<T> mappingFrom() {
+        return from;
+    }
+
+    /**
+     * Maps the provided internal exception to a public one.
+     *
+     * @param error Exception instance to be mapped.
+     * @return Public exception instance.
+     */
+    public R map(T error) {
+        return mapper.apply(error);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public runtime exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public runtime exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteException> IgniteExceptionMapper<T, R> unchecked(
+            Class<T> from,
+            Function<T, R> mapper
+    ) {
+        return new IgniteExceptionMapper<T, R>(from, mapper);
+    }
+
+    /**
+     * Creates a new exception mapper from an internal exception {@code T} to a public checked exception {@code R}.
+     *
+     * @param from Class instance that represents a class of internal exception.
+     * @param mapper Mapping function to map an internal exception to a public one.
+     * @param <T> Internal exception type.
+     * @param <R> Public checked exception type.
+     *
+     * @return New instance of {@link IgniteExceptionMapper}.
+     */
+    public static <T extends Exception, R extends IgniteCheckedException> IgniteExceptionMapper<T, R> checked(

Review Comment:
   Please take a look at the comment above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242149763


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new instance of public exception provided by the {@code mapper}.
+     *
+     * @param mapper Mapper function to produce a public exception.
+     * @param t Internal exception.
+     * @param <T> Type of an internal exception.
+     * @param <R> Type of a public exception.
+     * @return New public exception.
+     */
+    private static <T extends Exception, R extends Exception> Exception map(IgniteExceptionMapper<T, R> mapper, Throwable t) {
+        return mapper.map(mapper.mappingFrom().cast(t));
+    }
+
+    /**
+     * Returns {@code true} if the given exception {@code t} is instance of {@link NullPointerException}, {@link IllegalArgumentException}.
+     *
+     * @return {@code true} if the given exception is Java standard exception.
+     */
+    private static boolean isJavaStandardException(Throwable t) {
+        return t instanceof NullPointerException || t instanceof IllegalArgumentException;

Review Comment:
   Well, after some thought, I think that NPE should not be considered a valid case.
   ```
       /**
        * @throws NullPointerException If the provided name is {@code null}.
        */
       public CompletableFuture<Res> asycnOp(String name) throws NullPointerException {
           Object.requireNonNull(name);
   
           // This future should not be completed with NPE
           CompletableFuture<Res> r = internalApiCall();
           ...
           return res;
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242150615


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 static java.util.Collections.unmodifiableMap;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+/**
+ * This utility class provides an ability to map Ignite internal exceptions to Ignite public ones.
+ */
+public class IgniteExceptionMapperUtil {
+    /** All exception mappers to be used to map internal exceptions to public ones. */
+    private static final Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>>
+            EXCEPTION_CONVERTERS;
+
+    static {
+        Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> mappers = new HashMap<>();
+
+        ServiceLoader
+                .load(IgniteExceptionMappersProvider.class)
+                .forEach(provider -> provider.mappers().forEach(m -> registerMapping(m, mappers)));
+
+        EXCEPTION_CONVERTERS = unmodifiableMap(mappers);
+    }
+
+    /**
+     * Add a new mapping to already registered ones.
+     *
+     * @param mapper Exception mapper from internal exception to a public one.
+     * @param registeredMappings Already registered mappings.
+     * @throws IgniteException If a mapper for the given {@code clazz} already registered,
+     *      or {@code clazz} represents Java standard exception like {@link NullPointerException}, {@link IllegalArgumentException}.
+     */
+    static void registerMapping(
+            IgniteExceptionMapper<? extends Exception, ? extends Exception> mapper,
+            Map<Class<? extends Exception>, IgniteExceptionMapper<? extends Exception, ? extends Exception>> registeredMappings) {
+        if (registeredMappings.containsKey(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper, duplicate found [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        if (isJavaStandardException(mapper.mappingFrom())) {
+            throw new IgniteException(
+                    INTERNAL_ERR,
+                    "Failed to register exception mapper. "
+                            + "Mapping for this class is prohibited [class=" + mapper.mappingFrom().getCanonicalName() + ']');
+        }
+
+        registeredMappings.put(mapper.mappingFrom(), mapper);
+    }
+
+    /**
+     * This method provides a mapping from internal exception to Ignite public ones.
+     *
+     * <p>The rules of mapping are the following:</p>
+     * <ul>
+     *     <li>any instance of {@link Error} is returned as is, except {@link AssertionError}
+     *     that always be mapped to {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code.</li>
+     *     <li>any instance of Java standard exception like {@link NullPointerException} is returned as is.</li>
+     *     <li>any instance of {@link IgniteException} or {@link IgniteCheckedException} is returned as is.</li>
+     *     <li>if there are no any mappers that can do a mapping from the given error to a public exception,
+     *     then {@link IgniteException} with the {@link ErrorGroups.Common#INTERNAL_ERR} error code is returned.</li>
+     * </ul>
+     *
+     * @param origin Exception to be mapped.
+     * @return Public exception.
+     */
+    public static Throwable mapToPublicException(Throwable origin) {
+        if (origin instanceof Error) {
+            if (origin instanceof AssertionError) {
+                return new IgniteException(INTERNAL_ERR, origin);
+            }
+
+            return origin;
+        }
+
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
+        if (isJavaStandardException(origin)) {
+            return origin;
+        }
+
+        IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
+        if (m != null) {
+            Exception mapped = map(m, origin);
+
+            assert mapped instanceof IgniteException || mapped instanceof IgniteCheckedException :
+                    "Unexpected mapping of internal exception to a public one [origin=" + origin + ", mapped=" + mapped + ']';
+
+            return mapped;
+        }
+
+        // There are no exception mappings for the given exception. This case should be considered as internal error.
+        return new IgniteException(INTERNAL_ERR, origin);
+    }
+
+    /**
+     * Returns a new instance of public exception provided by the {@code mapper}.
+     *
+     * @param mapper Mapper function to produce a public exception.
+     * @param t Internal exception.
+     * @param <T> Type of an internal exception.
+     * @param <R> Type of a public exception.
+     * @return New public exception.
+     */
+    private static <T extends Exception, R extends Exception> Exception map(IgniteExceptionMapper<T, R> mapper, Throwable t) {
+        return mapper.map(mapper.mappingFrom().cast(t));
+    }
+
+    /**
+     * Returns {@code true} if the given exception {@code t} is instance of {@link NullPointerException}, {@link IllegalArgumentException}.
+     *
+     * @return {@code true} if the given exception is Java standard exception.
+     */
+    private static boolean isJavaStandardException(Throwable t) {
+        return t instanceof NullPointerException || t instanceof IllegalArgumentException;

Review Comment:
   All in all, this particular step of mapping should be removed from the PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #2225: IGNITE-19539 Added exception mapper utility

Posted by "sk0x50 (via GitHub)" <gi...@apache.org>.
sk0x50 commented on code in PR #2225:
URL: https://github.com/apache/ignite-3/pull/2225#discussion_r1242112542


##########
modules/api/src/main/java/org/apache/ignite/lang/IgniteExceptionUtils.java:
##########
@@ -177,14 +177,15 @@ public static int extractCodeFrom(Throwable t) {
         return INTERNAL_ERR;
     }
 
-    // TODO: https://issues.apache.org/jira/browse/IGNITE-19539 this method should be removed or re-worked.
+    // TODO: This method should be removed or re-worked and usages should be changed to IgniteExceptionMapperUtil.mapToPublicException.

Review Comment:
   Yep, you are right. Will create a ticket. https://issues.apache.org/jira/browse/IGNITE-19870



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org