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

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

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