You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/11 02:04:18 UTC

[GitHub] [iceberg] rdblue opened a new pull request #1909: Add ExceptionUtil.runSafely

rdblue opened a new pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909


   This adds a utility method that accepts blocks as lambdas and correctly handles exceptions. Catch and finally blocks are always run and any exceptions thrown in those blocks are added to the main block's exception as suppressed exceptions. If the main block does not throw an exception, but finally does then the finally exception is thrown.
   
   This currently supports up to 3 checked exception classes.


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#discussion_r545124571



##########
File path: core/src/main/java/org/apache/iceberg/util/ExceptionUtil.java
##########
@@ -36,4 +40,96 @@ private ExceptionUtil() {
     }
     throw new RuntimeException(exception);
   }
+
+  interface Block<R, E1 extends Exception, E2 extends Exception, E3 extends Exception> {
+    R run() throws E1, E2, E3;
+  }
+
+  interface CatchBlock {
+    void run(Throwable failure) throws Exception;
+  }
+
+  interface FinallyBlock {
+    void run() throws Exception;
+  }
+
+  public static <R> R runSafely(
+      Block<R, RuntimeException, RuntimeException, RuntimeException> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock) {
+    return runSafely(block, catchBlock, finallyBlock,
+        RuntimeException.class, RuntimeException.class, RuntimeException.class);
+  }
+
+  public static <R, E1 extends Exception> R runSafely(
+      Block<R, E1, RuntimeException, RuntimeException> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock,
+      Class<? extends E1> e1Class) throws E1 {
+    return runSafely(block, catchBlock, finallyBlock, e1Class, RuntimeException.class, RuntimeException.class);
+  }
+
+  public static <R, E1 extends Exception, E2 extends Exception> R runSafely(
+      Block<R, E1, E2, RuntimeException> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock,
+      Class<? extends E1> e1Class,
+      Class<? extends E2> e2Class) throws E1, E2 {
+    return runSafely(block, catchBlock, finallyBlock, e1Class, e2Class, RuntimeException.class);
+  }
+
+  public static <R, E1 extends Exception, E2 extends Exception, E3 extends Exception> R runSafely(
+      Block<R, E1, E2, E3> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock,
+      Class<? extends E1> e1Class,
+      Class<? extends E2> e2Class,
+      Class<? extends E3> e3Class) throws E1, E2, E3 {
+    Exception failure = null;
+    try {
+      return block.run();
+
+    } catch (Exception t) {
+      failure = t;
+
+      if (catchBlock != null) {
+        try {
+          catchBlock.run(failure);
+        } catch (Exception e) {
+          LOG.warn("Suppressing failure in catch block", e);
+          failure.addSuppressed(e);
+        }
+      }
+
+      tryThrowAs(failure, e1Class);
+      tryThrowAs(failure, e2Class);
+      tryThrowAs(failure, e3Class);
+      tryThrowAs(failure, RuntimeException.class);
+      throw new RuntimeException("Unknown throwable", failure);

Review comment:
       How do we get here? Is this just for safety? From what I can tell we only allow the block to have called exceptions for e1 -> e3, and all that's left should be RuntimeExceptions right?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909


   


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#discussion_r545301217



##########
File path: core/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestExceptionUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(TestExceptionUtil.class);
+
+  private static class CustomCheckedException extends Exception {
+    private CustomCheckedException(String message) {
+      super(message);
+    }
+  }
+
+  @Test
+  public void testRunSafely() {
+    CustomCheckedException exc = new CustomCheckedException("test");
+    try {
+      ExceptionUtil.runSafely(() -> {
+            throw exc;
+          }, e -> {
+            throw new Exception("test catch suppression");
+          }, () -> {
+            throw new RuntimeException("test finally suppression");
+          }, CustomCheckedException.class
+      );
+
+      Assert.fail("Should have thrown CustomCheckedException");
+
+    } catch (CustomCheckedException e) {
+      LOG.info("Final exception", e);
+      Assert.assertEquals("Should throw correct exception instance", exc, e);
+      Assert.assertEquals("Should not alter exception message", "test", e.getMessage());
+      Assert.assertEquals("Should have 2 suppressed exceptions", 2, e.getSuppressed().length);
+
+      Throwable throwSuppressed = e.getSuppressed()[0];
+      Assert.assertTrue("Should be a CustomCheckedException", throwSuppressed instanceof Exception);

Review comment:
       Fixed




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#issuecomment-747682529


   Tests are passing, so I'll merge this. Thanks @yyanyy, @RussellSpitzer, and @aokolnychyi for reviewing!


----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#discussion_r545131874



##########
File path: core/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestExceptionUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(TestExceptionUtil.class);
+
+  private static class CustomCheckedException extends Exception {
+    private CustomCheckedException(String message) {
+      super(message);
+    }
+  }
+
+  @Test
+  public void testRunSafely() {

Review comment:
       I may be a bit paranoid, but I'd like to have a test which uses three exception types. Also a case for just testing runtime exceptions in each block.
   
   Other than that I think this is a good utility! Thanks for working on it




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#discussion_r545309427



##########
File path: core/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestExceptionUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(TestExceptionUtil.class);
+
+  private static class CustomCheckedException extends Exception {
+    private CustomCheckedException(String message) {
+      super(message);
+    }
+  }
+
+  @Test
+  public void testRunSafely() {

Review comment:
       Done




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on a change in pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#discussion_r544769619



##########
File path: core/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestExceptionUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(TestExceptionUtil.class);
+
+  private static class CustomCheckedException extends Exception {
+    private CustomCheckedException(String message) {
+      super(message);
+    }
+  }
+
+  @Test
+  public void testRunSafely() {
+    CustomCheckedException exc = new CustomCheckedException("test");
+    try {
+      ExceptionUtil.runSafely(() -> {
+            throw exc;
+          }, e -> {
+            throw new Exception("test catch suppression");
+          }, () -> {
+            throw new RuntimeException("test finally suppression");
+          }, CustomCheckedException.class
+      );
+
+      Assert.fail("Should have thrown CustomCheckedException");
+
+    } catch (CustomCheckedException e) {
+      LOG.info("Final exception", e);
+      Assert.assertEquals("Should throw correct exception instance", exc, e);
+      Assert.assertEquals("Should not alter exception message", "test", e.getMessage());
+      Assert.assertEquals("Should have 2 suppressed exceptions", 2, e.getSuppressed().length);
+
+      Throwable throwSuppressed = e.getSuppressed()[0];
+      Assert.assertTrue("Should be a CustomCheckedException", throwSuppressed instanceof Exception);

Review comment:
       Other than the copy-paste error in this assert failure message and L62, change LGTM! 




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1909: Add ExceptionUtil.runSafely

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1909:
URL: https://github.com/apache/iceberg/pull/1909#discussion_r545312393



##########
File path: core/src/main/java/org/apache/iceberg/util/ExceptionUtil.java
##########
@@ -36,4 +40,96 @@ private ExceptionUtil() {
     }
     throw new RuntimeException(exception);
   }
+
+  interface Block<R, E1 extends Exception, E2 extends Exception, E3 extends Exception> {
+    R run() throws E1, E2, E3;
+  }
+
+  interface CatchBlock {
+    void run(Throwable failure) throws Exception;
+  }
+
+  interface FinallyBlock {
+    void run() throws Exception;
+  }
+
+  public static <R> R runSafely(
+      Block<R, RuntimeException, RuntimeException, RuntimeException> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock) {
+    return runSafely(block, catchBlock, finallyBlock,
+        RuntimeException.class, RuntimeException.class, RuntimeException.class);
+  }
+
+  public static <R, E1 extends Exception> R runSafely(
+      Block<R, E1, RuntimeException, RuntimeException> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock,
+      Class<? extends E1> e1Class) throws E1 {
+    return runSafely(block, catchBlock, finallyBlock, e1Class, RuntimeException.class, RuntimeException.class);
+  }
+
+  public static <R, E1 extends Exception, E2 extends Exception> R runSafely(
+      Block<R, E1, E2, RuntimeException> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock,
+      Class<? extends E1> e1Class,
+      Class<? extends E2> e2Class) throws E1, E2 {
+    return runSafely(block, catchBlock, finallyBlock, e1Class, e2Class, RuntimeException.class);
+  }
+
+  public static <R, E1 extends Exception, E2 extends Exception, E3 extends Exception> R runSafely(
+      Block<R, E1, E2, E3> block,
+      CatchBlock catchBlock,
+      FinallyBlock finallyBlock,
+      Class<? extends E1> e1Class,
+      Class<? extends E2> e2Class,
+      Class<? extends E3> e3Class) throws E1, E2, E3 {
+    Exception failure = null;
+    try {
+      return block.run();
+
+    } catch (Exception t) {
+      failure = t;
+
+      if (catchBlock != null) {
+        try {
+          catchBlock.run(failure);
+        } catch (Exception e) {
+          LOG.warn("Suppressing failure in catch block", e);
+          failure.addSuppressed(e);
+        }
+      }
+
+      tryThrowAs(failure, e1Class);
+      tryThrowAs(failure, e2Class);
+      tryThrowAs(failure, e3Class);
+      tryThrowAs(failure, RuntimeException.class);
+      throw new RuntimeException("Unknown throwable", failure);

Review comment:
       Originally, this block caught `Throwable`, but I changed it to `Exception` thinking that we should not catch `Error` and run the `catch` block. In that case, this line was just for safety.
   
   But I've changed it back to `Throwable` after looking at this comment. I thought more about it and this block _does_ need to run if `Error` is thrown instead of `Exception`. If `Error` is thrown but not caught here, then the finally block will run without the `failure` set and would throw any exception from that block instead of suppressing it.
   
   I think if we want to do no work when `Error` is thrown, then this should specifically check for `Error`, but I think that it is correct to run everything on `Error` because that's the superclass of things like `AssertionError`. If an `assert` fails, we still want to run everything like normal.




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org