From 55b71c8b9bf2840f3bae05c97703eb2c08672b58 Mon Sep 17 00:00:00 2001 From: Remi Catherinot Date: Mon, 2 May 2022 09:55:45 +0200 Subject: [PATCH 1/5] HADOOP-18217. ExitUtil synchronized blocks reduced to avoid exit blocking halt + enlarged catches (robustness) to all Throwables (not just Exceptions) --- .../java/org/apache/hadoop/util/ExitUtil.java | 94 ++++++++++++++----- 1 file changed, 71 insertions(+), 23 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java index 32f4b5b7a72d4..101ab00d99fd0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java @@ -200,23 +200,48 @@ public static void resetFirstHaltException() { * or, if system exits are disabled, rethrow the exception. * @param ee exit exception */ - public static synchronized void terminate(ExitException ee) + public static void terminate(ExitException ee) throws ExitException { - int status = ee.getExitCode(); - String msg = ee.getMessage(); + final int status = ee.getExitCode(); + Error catched = null; if (status != 0) { - //exit indicates a problem, log it - LOG.debug("Exiting with status {}: {}", status, msg, ee); - LOG.info("Exiting with status {}: {}", status, msg); + try { + //exit indicates a problem, log it + String msg = ee.getMessage(); + LOG.debug("Exiting with status {}: {}", status, msg, ee); + LOG.info("Exiting with status {}: {}", status, msg); + } catch (Error e) { + catched = e; // errors have higher priority than HaltException, it may be re-thrown. OOM and ThreadDeath are 2 examples of Errors to re-throw + } catch (Throwable t) { + // all other kind of throwables are supressed + ee.addSuppressed(t); + } } if (systemExitDisabled) { - LOG.error("Terminate called", ee); - if (!terminateCalled()) { - firstExitException = ee; + try { + LOG.error("Terminate called", ee); + } catch (Error e) { + if (catched == null) { + catched = e; // errors will be re-thrown + } else { + catched.addSuppressed(e); // 1st raised error has priority and will be re-thrown, so the 1st error supresses the 2nd + } + } catch (Throwable t) { + ee.addSuppressed(t); // all other kind of throwables are supressed + } + synchronized (ExitUtil.class) { + if (!terminateCalled()) { + firstExitException = ee; + } + } + if (catched != null) { + catched.addSuppressed(ee); + throw catched; } throw ee; + } else { + System.exit(status); // System.exit has higher priority than any catched error } - System.exit(status); } /** @@ -226,25 +251,48 @@ public static synchronized void terminate(ExitException ee) * trace. * @throws HaltException if {@link Runtime#halt(int)} is disabled. */ - public static synchronized void halt(HaltException ee) throws HaltException { - int status = ee.getExitCode(); - String msg = ee.getMessage(); - try { - if (status != 0) { + public static void halt(HaltException ee) throws HaltException { + final int status = ee.getExitCode(); + Error catched = null; + if (status != 0) { + try { //exit indicates a problem, log it + String msg = ee.getMessage(); LOG.info("Halt with status {}: {}", status, msg, ee); + } catch (Error e) { + catched = e; // errors have higher priority than HaltException, it may be re-thrown. OOM and ThreadDeath are 2 examples of Errors to re-throw + } catch (Throwable t) { + // all other kind of throwables are supressed + ee.addSuppressed(t); } - } catch (Exception ignored) { - // ignore exceptions here, as it may be due to an out of memory situation } - if (systemHaltDisabled) { - LOG.error("Halt called", ee); - if (!haltCalled()) { - firstHaltException = ee; + if (systemHaltDisabled) { // this is a volatile so reading it does not need a synchronized block + try { + LOG.error("Halt called", ee); + } catch (Error e) { + if (catched == null) { + catched = e; // errors will be re-thrown + } else { + catched.addSuppressed(e); + } + } catch (Throwable t) { + // all other kind of throwables are supressed + ee.addSuppressed(t); } - throw ee; + synchronized (ExitUtil.class) { + if (!haltCalled()) { + firstHaltException = ee; + } + } + if (catched != null) { + catched.addSuppressed(ee); + throw catched; + } + throw ee; // not supressed by a higher prority error + } else { + // when halt is not disabled, whatever Throwable happened, we halt the VM + Runtime.getRuntime().halt(status); } - Runtime.getRuntime().halt(status); } /** From f279d644ee91e335c63cd090baf0635edc325655 Mon Sep 17 00:00:00 2001 From: Remi Catherinot Date: Thu, 12 May 2022 21:06:08 +0200 Subject: [PATCH 2/5] HADOOP-18217. use atomics, added tests, updated javadoc, fixed code style --- .../java/org/apache/hadoop/util/ExitUtil.java | 133 ++++++++++-------- .../org/apache/hadoop/util/TestExitUtil.java | 122 ++++++++++++++++ 2 files changed, 199 insertions(+), 56 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java index 101ab00d99fd0..71808bfc5d406 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.util; +import java.util.concurrent.atomic.AtomicReference; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.slf4j.Logger; @@ -36,8 +38,10 @@ public final class ExitUtil { LOG = LoggerFactory.getLogger(ExitUtil.class.getName()); private static volatile boolean systemExitDisabled = false; private static volatile boolean systemHaltDisabled = false; - private static volatile ExitException firstExitException; - private static volatile HaltException firstHaltException; + private static final AtomicReference FIRST_EXIT_EXCEPTION = + new AtomicReference<>(); + private static final AtomicReference FIRST_HALT_EXCEPTION = + new AtomicReference<>(); /** Message raised from an exit exception if none were provided: {@value}. */ public static final String EXIT_EXCEPTION_MESSAGE = "ExitException"; /** Message raised from a halt exception if none were provided: {@value}. */ @@ -159,28 +163,29 @@ public static void disableSystemHalt() { */ public static boolean terminateCalled() { // Either we set this member or we actually called System#exit - return firstExitException != null; + return FIRST_EXIT_EXCEPTION.get()!=null; } /** * @return true if halt has been called. */ public static boolean haltCalled() { - return firstHaltException != null; + // Either we set this member or we actually called Runtime#halt + return FIRST_HALT_EXCEPTION.get()!=null; } /** - * @return the first ExitException thrown, null if none thrown yet. + * @return the first {@code ExitException} thrown, null if none thrown yet. */ public static ExitException getFirstExitException() { - return firstExitException; + return FIRST_EXIT_EXCEPTION.get(); } /** * @return the first {@code HaltException} thrown, null if none thrown yet. */ public static HaltException getFirstHaltException() { - return firstHaltException; + return FIRST_HALT_EXCEPTION.get(); } /** @@ -188,32 +193,42 @@ public static HaltException getFirstHaltException() { * where one test in the suite expects an exit but others do not. */ public static void resetFirstExitException() { - firstExitException = null; + FIRST_EXIT_EXCEPTION.set(null); } + /** + * Reset the tracking of process termination. This is for use in unit tests + * where one test in the suite expects a halt but others do not. + */ public static void resetFirstHaltException() { - firstHaltException = null; + FIRST_HALT_EXCEPTION.set(null); } /** + * Exits the JVM if exit is enabled, rethrow provided exception or any raised error otherwise. * Inner termination: either exit with the exception's exit code, * or, if system exits are disabled, rethrow the exception. * @param ee exit exception + * @throws ExitException if {@link System#exit(int)} is disabled and not suppressed by an Error + * @throws Error if {@link System#exit(int)} is disabled and one Error arise, suppressing + * anything else, even ee */ public static void terminate(ExitException ee) throws ExitException { final int status = ee.getExitCode(); - Error catched = null; + Error caught = null; if (status != 0) { try { - //exit indicates a problem, log it + // exit indicates a problem, log it String msg = ee.getMessage(); LOG.debug("Exiting with status {}: {}", status, msg, ee); LOG.info("Exiting with status {}: {}", status, msg); } catch (Error e) { - catched = e; // errors have higher priority than HaltException, it may be re-thrown. OOM and ThreadDeath are 2 examples of Errors to re-throw + // errors have higher priority than HaltException, it may be re-thrown. + // OOM and ThreadDeath are 2 examples of Errors to re-throw + caught = e; } catch (Throwable t) { - // all other kind of throwables are supressed + // all other kind of throwables are suppressed ee.addSuppressed(t); } } @@ -221,76 +236,82 @@ public static void terminate(ExitException ee) try { LOG.error("Terminate called", ee); } catch (Error e) { - if (catched == null) { - catched = e; // errors will be re-thrown + // errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it + if (caught == null) { + caught = e; } else { - catched.addSuppressed(e); // 1st raised error has priority and will be re-thrown, so the 1st error supresses the 2nd + caught.addSuppressed(e); } } catch (Throwable t) { - ee.addSuppressed(t); // all other kind of throwables are supressed - } - synchronized (ExitUtil.class) { - if (!terminateCalled()) { - firstExitException = ee; - } + // all other kind of throwables are suppressed + ee.addSuppressed(t); } - if (catched != null) { - catched.addSuppressed(ee); - throw catched; + FIRST_EXIT_EXCEPTION.compareAndSet(null, ee); + if (caught != null) { + caught.addSuppressed(ee); + throw caught; } throw ee; } else { - System.exit(status); // System.exit has higher priority than any catched error + // when exit is enabled, whatever Throwable happened, we exit the VM + System.exit(status); } } /** - * Forcibly terminates the currently running Java virtual machine. - * The exception argument is rethrown if JVM halting is disabled. - * @param ee the exception containing the status code, message and any stack + * Halts the JVM if halt is enabled, rethrow provided exception or any raised error otherwise. + * If halt is disabled, this method throws either the exception argument if no + * error arise, the first error if at least one arise, suppressing he. + * If halt is enabled, all throwables are caught, even errors. + * + * @param he the exception containing the status code, message and any stack * trace. - * @throws HaltException if {@link Runtime#halt(int)} is disabled. + * @throws HaltException if {@link Runtime#halt(int)} is disabled and not suppressed by an Error + * @throws Error if {@link Runtime#halt(int)} is disabled and one Error arise, suppressing + * anyuthing else, even he */ - public static void halt(HaltException ee) throws HaltException { - final int status = ee.getExitCode(); - Error catched = null; + public static void halt(HaltException he) throws HaltException { + final int status = he.getExitCode(); + Error caught = null; if (status != 0) { try { - //exit indicates a problem, log it - String msg = ee.getMessage(); - LOG.info("Halt with status {}: {}", status, msg, ee); + // exit indicates a problem, log it + String msg = he.getMessage(); + LOG.info("Halt with status {}: {}", status, msg, he); } catch (Error e) { - catched = e; // errors have higher priority than HaltException, it may be re-thrown. OOM and ThreadDeath are 2 examples of Errors to re-throw + // errors have higher priority than HaltException, it may be re-thrown. + // OOM and ThreadDeath are 2 examples of Errors to re-throw + caught = e; } catch (Throwable t) { - // all other kind of throwables are supressed - ee.addSuppressed(t); + // all other kind of throwables are suppressed + he.addSuppressed(t); } } - if (systemHaltDisabled) { // this is a volatile so reading it does not need a synchronized block + // systemHaltDisabled is volatile and not used in scenario nheding atomicty, + // thus it does not nhed a synchronized access nor a atomic access + if (systemHaltDisabled) { try { - LOG.error("Halt called", ee); + LOG.error("Halt called", he); } catch (Error e) { - if (catched == null) { - catched = e; // errors will be re-thrown + // errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it + if (caught == null) { + caught = e; } else { - catched.addSuppressed(e); + caught.addSuppressed(e); } } catch (Throwable t) { - // all other kind of throwables are supressed - ee.addSuppressed(t); - } - synchronized (ExitUtil.class) { - if (!haltCalled()) { - firstHaltException = ee; - } + // all other kind of throwables are suppressed + he.addSuppressed(t); } - if (catched != null) { - catched.addSuppressed(ee); - throw catched; + FIRST_HALT_EXCEPTION.compareAndSet(null, he); + if (caught != null) { + caught.addSuppressed(he); + throw caught; } - throw ee; // not supressed by a higher prority error + // not suppressed by a higher prority error + throw he; } else { - // when halt is not disabled, whatever Throwable happened, we halt the VM + // when halt is enabled, whatever Throwable happened, we halt the VM Runtime.getRuntime().halt(status); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java new file mode 100644 index 0000000000000..8c22a03813ce5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java @@ -0,0 +1,122 @@ +/** + * 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.hadoop.util; + +import static org.junit.Assert.*; + +import org.apache.hadoop.util.ExitUtil.ExitException; +import org.apache.hadoop.util.ExitUtil.HaltException; + +import org.junit.Test; + +public class TestExitUtil { + + @Test + public void testGetSetExitExceptions() { + // prepare states and exceptions + ExitUtil.disableSystemExit(); + ExitUtil.resetFirstExitException(); + ExitException ee1 = new ExitException(1, "TestExitUtil forged 1st ExitException"); + ExitException ee2 = new ExitException(2, "TestExitUtil forged 2nd ExitException"); + try { + // check proper initial settings + assertFalse(ExitUtil.terminateCalled()); + assertNull(ExitUtil.getFirstExitException()); + + // simulate/check 1st call + try { + ExitUtil.terminate(ee1); + fail("ExitUtil.terminate should have rethrown its ExitException argument but it returned"); + } catch (ExitException ee) { + assertEquals("ExitUtil.terminate should have rethrown its ExitException argument but it " + +"had thrown something else", ee1, ee); + } + assertTrue(ExitUtil.terminateCalled()); + assertEquals("ExitUtil.terminate should store its 1st call's ExitException", + ee1, ExitUtil.getFirstExitException()); + + // simulate/check 2nd call not overwritting 1st one + try { + ExitUtil.terminate(ee2); + fail("ExitUtil.terminate should have rethrown its ExitException argument but it returned"); + } catch (ExitException ee) { + assertEquals("ExitUtil.terminate should have rethrown its HaltException argument but it " + +"had thrown something else", ee2, ee); + } + assertTrue(ExitUtil.terminateCalled()); + // 2nd call rethrown the 2nd ExitException yet only the 1st only should have been stored + assertEquals("ExitUtil.terminate when called twice should only remember 1st call's " + +"ExitException", ee1, ExitUtil.getFirstExitException()); + + // simulate cleanup, also tries to make sure state is ok for all junit still has to do + ExitUtil.resetFirstExitException(); + assertFalse(ExitUtil.terminateCalled()); + assertNull(ExitUtil.getFirstExitException()); + } finally { + // cleanup + ExitUtil.resetFirstExitException(); + } + } + + @Test + public void testGetSetHaltExceptions() { + // prepare states and exceptions + ExitUtil.disableSystemHalt(); + ExitUtil.resetFirstHaltException(); + HaltException he1 = new HaltException(1, "TestExitUtil forged 1st HaltException"); + HaltException he2 = new HaltException(2, "TestExitUtil forged 2nd HaltException"); + try { + // check proper initial settings + assertFalse(ExitUtil.haltCalled()); + assertNull(ExitUtil.getFirstHaltException()); + + // simulate/check 1st call + try { + ExitUtil.halt(he1); + fail("ExitUtil.halt should have rethrown its HaltException argument but it returned"); + } catch (HaltException he) { + assertEquals("ExitUtil.halt should have rethrown its HaltException argument but it had " + +"thrown something else", he1, he); + } + assertTrue("ExitUtil.halt/haltCalled should remember halt has been called", + ExitUtil.haltCalled()); + assertEquals("ExitUtil.halt should store its 1st call's HaltException", + he1, ExitUtil.getFirstHaltException()); + + // simulate/check 2nd call not overwritting 1st one + try { + ExitUtil.halt(he2); + fail("ExitUtil.halt should have rethrown its HaltException argument but it returned"); + } catch (HaltException he) { + assertEquals("ExitUtil.halt should have rethrown its HaltException argument but it had " + +"thrown something else", he2, he); + } + assertTrue(ExitUtil.haltCalled()); + assertEquals("ExitUtil.halt when called twice should only remember 1st call's HaltException", + he1, ExitUtil.getFirstHaltException()); + + // simulate cleanup, also tries to make sure state is ok for all junit still has to do + ExitUtil.resetFirstHaltException(); + assertFalse(ExitUtil.haltCalled()); + assertNull(ExitUtil.getFirstHaltException()); + } finally { + // cleanup, useless if last test succeed, useful if not + ExitUtil.resetFirstHaltException(); + } + } +} From 7d6468bf35af9f4b56bf2c8d0102852d0de73bc7 Mon Sep 17 00:00:00 2001 From: Remi Catherinot Date: Mon, 11 Jul 2022 18:39:22 +0200 Subject: [PATCH 3/5] HADOOP-18217. added addSuppressed checks, fixed code style, tests use intercept+messages+assertSame --- .../java/org/apache/hadoop/util/ExitUtil.java | 24 +++-- .../org/apache/hadoop/util/TestExitUtil.java | 98 +++++++++---------- 2 files changed, 65 insertions(+), 57 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java index 71808bfc5d406..533177f1506c8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java @@ -163,7 +163,7 @@ public static void disableSystemHalt() { */ public static boolean terminateCalled() { // Either we set this member or we actually called System#exit - return FIRST_EXIT_EXCEPTION.get()!=null; + return FIRST_EXIT_EXCEPTION.get() != null; } /** @@ -171,7 +171,7 @@ public static boolean terminateCalled() { */ public static boolean haltCalled() { // Either we set this member or we actually called Runtime#halt - return FIRST_HALT_EXCEPTION.get()!=null; + return FIRST_HALT_EXCEPTION.get() != null; } /** @@ -229,7 +229,9 @@ public static void terminate(ExitException ee) caught = e; } catch (Throwable t) { // all other kind of throwables are suppressed - ee.addSuppressed(t); + if (ee != t) { + ee.addSuppressed(t); + } } } if (systemExitDisabled) { @@ -239,12 +241,14 @@ public static void terminate(ExitException ee) // errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it if (caught == null) { caught = e; - } else { + } else if (caught != e) { caught.addSuppressed(e); } } catch (Throwable t) { // all other kind of throwables are suppressed - ee.addSuppressed(t); + if (ee != t) { + ee.addSuppressed(t); + } } FIRST_EXIT_EXCEPTION.compareAndSet(null, ee); if (caught != null) { @@ -284,7 +288,9 @@ public static void halt(HaltException he) throws HaltException { caught = e; } catch (Throwable t) { // all other kind of throwables are suppressed - he.addSuppressed(t); + if (he != t) { + he.addSuppressed(t); + } } } // systemHaltDisabled is volatile and not used in scenario nheding atomicty, @@ -296,12 +302,14 @@ public static void halt(HaltException he) throws HaltException { // errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it if (caught == null) { caught = e; - } else { + } else if (caught != e) { caught.addSuppressed(e); } } catch (Throwable t) { // all other kind of throwables are suppressed - he.addSuppressed(t); + if (he != t) { + he.addSuppressed(t); + } } FIRST_HALT_EXCEPTION.compareAndSet(null, he); if (caught != null) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java index 8c22a03813ce5..c7fa2f9b49551 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java @@ -17,17 +17,22 @@ */ package org.apache.hadoop.util; -import static org.junit.Assert.*; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; import org.apache.hadoop.util.ExitUtil.ExitException; import org.apache.hadoop.util.ExitUtil.HaltException; -import org.junit.Test; public class TestExitUtil { @Test - public void testGetSetExitExceptions() { + public void testGetSetExitExceptions() throws Throwable { // prepare states and exceptions ExitUtil.disableSystemExit(); ExitUtil.resetFirstExitException(); @@ -35,38 +40,36 @@ public void testGetSetExitExceptions() { ExitException ee2 = new ExitException(2, "TestExitUtil forged 2nd ExitException"); try { // check proper initial settings - assertFalse(ExitUtil.terminateCalled()); - assertNull(ExitUtil.getFirstExitException()); + assertFalse("ExitUtil.terminateCalled initial value should be false", + ExitUtil.terminateCalled()); + assertNull("ExitUtil.getFirstExitException initial value should be null", + ExitUtil.getFirstExitException()); // simulate/check 1st call - try { - ExitUtil.terminate(ee1); - fail("ExitUtil.terminate should have rethrown its ExitException argument but it returned"); - } catch (ExitException ee) { - assertEquals("ExitUtil.terminate should have rethrown its ExitException argument but it " - +"had thrown something else", ee1, ee); - } - assertTrue(ExitUtil.terminateCalled()); - assertEquals("ExitUtil.terminate should store its 1st call's ExitException", + ExitException ee = intercept(ExitException.class, ()->ExitUtil.terminate(ee1)); + assertSame("ExitUtil.terminate should have rethrown its ExitException argument but it " + + "had thrown something else", ee1, ee); + assertTrue("ExitUtil.terminateCalled should be true after 1st ExitUtil.terminate call", + ExitUtil.terminateCalled()); + assertSame("ExitUtil.terminate should store its 1st call's ExitException", ee1, ExitUtil.getFirstExitException()); // simulate/check 2nd call not overwritting 1st one - try { - ExitUtil.terminate(ee2); - fail("ExitUtil.terminate should have rethrown its ExitException argument but it returned"); - } catch (ExitException ee) { - assertEquals("ExitUtil.terminate should have rethrown its HaltException argument but it " - +"had thrown something else", ee2, ee); - } - assertTrue(ExitUtil.terminateCalled()); + ee = intercept(ExitException.class, ()->ExitUtil.terminate(ee2)); + assertSame("ExitUtil.terminate should have rethrown its HaltException argument but it " + + "had thrown something else", ee2, ee); + assertTrue("ExitUtil.terminateCalled should still be true after 2nd ExitUtil.terminate call", + ExitUtil.terminateCalled()); // 2nd call rethrown the 2nd ExitException yet only the 1st only should have been stored - assertEquals("ExitUtil.terminate when called twice should only remember 1st call's " - +"ExitException", ee1, ExitUtil.getFirstExitException()); + assertSame("ExitUtil.terminate when called twice should only remember 1st call's " + + "ExitException", ee1, ExitUtil.getFirstExitException()); // simulate cleanup, also tries to make sure state is ok for all junit still has to do ExitUtil.resetFirstExitException(); - assertFalse(ExitUtil.terminateCalled()); - assertNull(ExitUtil.getFirstExitException()); + assertFalse("ExitUtil.terminateCalled should be false after " + + "ExitUtil.resetFirstExitException call", ExitUtil.terminateCalled()); + assertNull("ExitUtil.getFirstExitException should be null after " + + "ExitUtil.resetFirstExitException call", ExitUtil.getFirstExitException()); } finally { // cleanup ExitUtil.resetFirstExitException(); @@ -74,7 +77,7 @@ public void testGetSetExitExceptions() { } @Test - public void testGetSetHaltExceptions() { + public void testGetSetHaltExceptions() throws Throwable { // prepare states and exceptions ExitUtil.disableSystemHalt(); ExitUtil.resetFirstHaltException(); @@ -82,38 +85,35 @@ public void testGetSetHaltExceptions() { HaltException he2 = new HaltException(2, "TestExitUtil forged 2nd HaltException"); try { // check proper initial settings - assertFalse(ExitUtil.haltCalled()); - assertNull(ExitUtil.getFirstHaltException()); + assertFalse("ExitUtil.haltCalled initial value should be false", + ExitUtil.haltCalled()); + assertNull("ExitUtil.getFirstHaltException initial value should be null", + ExitUtil.getFirstHaltException()); // simulate/check 1st call - try { - ExitUtil.halt(he1); - fail("ExitUtil.halt should have rethrown its HaltException argument but it returned"); - } catch (HaltException he) { - assertEquals("ExitUtil.halt should have rethrown its HaltException argument but it had " - +"thrown something else", he1, he); - } - assertTrue("ExitUtil.halt/haltCalled should remember halt has been called", + HaltException he = intercept(HaltException.class, ()->ExitUtil.halt(he1)); + assertSame("ExitUtil.halt should have rethrown its HaltException argument but it had " + +"thrown something else", he1, he); + assertTrue("ExitUtil.haltCalled should be true after 1st ExitUtil.halt call", ExitUtil.haltCalled()); - assertEquals("ExitUtil.halt should store its 1st call's HaltException", + assertSame("ExitUtil.halt should store its 1st call's HaltException", he1, ExitUtil.getFirstHaltException()); // simulate/check 2nd call not overwritting 1st one - try { - ExitUtil.halt(he2); - fail("ExitUtil.halt should have rethrown its HaltException argument but it returned"); - } catch (HaltException he) { - assertEquals("ExitUtil.halt should have rethrown its HaltException argument but it had " - +"thrown something else", he2, he); - } - assertTrue(ExitUtil.haltCalled()); - assertEquals("ExitUtil.halt when called twice should only remember 1st call's HaltException", + he = intercept(HaltException.class, ()->ExitUtil.halt(he2)); + assertSame("ExitUtil.halt should have rethrown its HaltException argument but it had " + +"thrown something else", he2, he); + assertTrue("ExitUtil.haltCalled should still be true after 2nd ExitUtil.halt call", + ExitUtil.haltCalled()); + assertSame("ExitUtil.halt when called twice should only remember 1st call's HaltException", he1, ExitUtil.getFirstHaltException()); // simulate cleanup, also tries to make sure state is ok for all junit still has to do ExitUtil.resetFirstHaltException(); - assertFalse(ExitUtil.haltCalled()); - assertNull(ExitUtil.getFirstHaltException()); + assertFalse("ExitUtil.haltCalled should be false after " + + "ExitUtil.resetFirstHaltException call", ExitUtil.haltCalled()); + assertNull("ExitUtil.getFirstHaltException should be null after " + + "ExitUtil.resetFirstHaltException call", ExitUtil.getFirstHaltException()); } finally { // cleanup, useless if last test succeed, useful if not ExitUtil.resetFirstHaltException(); From 9e9dc762380d4f9c2a55c24412d3cbbdb19394bf Mon Sep 17 00:00:00 2001 From: Remi Catherinot Date: Mon, 11 Jul 2022 20:57:53 +0200 Subject: [PATCH 4/5] HADOOP-18217. added addSuppressed helper + JUnit Before+After --- .../java/org/apache/hadoop/util/ExitUtil.java | 48 +++--- .../org/apache/hadoop/util/TestExitUtil.java | 141 +++++++++--------- 2 files changed, 96 insertions(+), 93 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java index 533177f1506c8..ab96ecc7a76e8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java @@ -204,6 +204,20 @@ public static void resetFirstHaltException() { FIRST_HALT_EXCEPTION.set(null); } + /** + * Suppresses if legit and returns the first non-null of the two. Legit means + * suppressor if neither null nor suppressed. + */ + private static T addSuppressed(T suppressor, T suppressed) { + if (suppressor == null) { + return suppressed; + } + if (suppressor != suppressed) { + suppressor.addSuppressed(suppressed); + } + return suppressor; + } + /** * Exits the JVM if exit is enabled, rethrow provided exception or any raised error otherwise. * Inner termination: either exit with the exception's exit code, @@ -213,8 +227,7 @@ public static void resetFirstHaltException() { * @throws Error if {@link System#exit(int)} is disabled and one Error arise, suppressing * anything else, even ee */ - public static void terminate(ExitException ee) - throws ExitException { + public static void terminate(final ExitException ee) throws ExitException { final int status = ee.getExitCode(); Error caught = null; if (status != 0) { @@ -229,9 +242,7 @@ public static void terminate(ExitException ee) caught = e; } catch (Throwable t) { // all other kind of throwables are suppressed - if (ee != t) { - ee.addSuppressed(t); - } + addSuppressed(ee, t); } } if (systemExitDisabled) { @@ -239,22 +250,17 @@ public static void terminate(ExitException ee) LOG.error("Terminate called", ee); } catch (Error e) { // errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it - if (caught == null) { - caught = e; - } else if (caught != e) { - caught.addSuppressed(e); - } + caught = addSuppressed(caught, e); } catch (Throwable t) { // all other kind of throwables are suppressed - if (ee != t) { - ee.addSuppressed(t); - } + addSuppressed(ee, t); } FIRST_EXIT_EXCEPTION.compareAndSet(null, ee); if (caught != null) { caught.addSuppressed(ee); throw caught; } + // not suppressed by a higher prority error throw ee; } else { // when exit is enabled, whatever Throwable happened, we exit the VM @@ -274,7 +280,7 @@ public static void terminate(ExitException ee) * @throws Error if {@link Runtime#halt(int)} is disabled and one Error arise, suppressing * anyuthing else, even he */ - public static void halt(HaltException he) throws HaltException { + public static void halt(final HaltException he) throws HaltException { final int status = he.getExitCode(); Error caught = null; if (status != 0) { @@ -288,9 +294,7 @@ public static void halt(HaltException he) throws HaltException { caught = e; } catch (Throwable t) { // all other kind of throwables are suppressed - if (he != t) { - he.addSuppressed(t); - } + addSuppressed(he, t); } } // systemHaltDisabled is volatile and not used in scenario nheding atomicty, @@ -300,16 +304,10 @@ public static void halt(HaltException he) throws HaltException { LOG.error("Halt called", he); } catch (Error e) { // errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it - if (caught == null) { - caught = e; - } else if (caught != e) { - caught.addSuppressed(e); - } + caught = addSuppressed(caught, e); } catch (Throwable t) { // all other kind of throwables are suppressed - if (he != t) { - he.addSuppressed(t); - } + addSuppressed(he, t); } FIRST_HALT_EXCEPTION.compareAndSet(null, he); if (caught != null) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java index c7fa2f9b49551..58a1997e9bc59 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestExitUtil.java @@ -23,57 +23,66 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.apache.hadoop.util.ExitUtil.ExitException; import org.apache.hadoop.util.ExitUtil.HaltException; +import org.apache.hadoop.test.AbstractHadoopTestBase; +public class TestExitUtil extends AbstractHadoopTestBase { -public class TestExitUtil { + @Before + public void before() { + ExitUtil.disableSystemExit(); + ExitUtil.disableSystemHalt(); + ExitUtil.resetFirstExitException(); + ExitUtil.resetFirstHaltException(); + } + + @After + public void after() { + ExitUtil.resetFirstExitException(); + ExitUtil.resetFirstHaltException(); + } @Test public void testGetSetExitExceptions() throws Throwable { // prepare states and exceptions - ExitUtil.disableSystemExit(); - ExitUtil.resetFirstExitException(); ExitException ee1 = new ExitException(1, "TestExitUtil forged 1st ExitException"); ExitException ee2 = new ExitException(2, "TestExitUtil forged 2nd ExitException"); - try { - // check proper initial settings - assertFalse("ExitUtil.terminateCalled initial value should be false", - ExitUtil.terminateCalled()); - assertNull("ExitUtil.getFirstExitException initial value should be null", - ExitUtil.getFirstExitException()); + // check proper initial settings + assertFalse("ExitUtil.terminateCalled initial value should be false", + ExitUtil.terminateCalled()); + assertNull("ExitUtil.getFirstExitException initial value should be null", + ExitUtil.getFirstExitException()); - // simulate/check 1st call - ExitException ee = intercept(ExitException.class, ()->ExitUtil.terminate(ee1)); - assertSame("ExitUtil.terminate should have rethrown its ExitException argument but it " - + "had thrown something else", ee1, ee); - assertTrue("ExitUtil.terminateCalled should be true after 1st ExitUtil.terminate call", - ExitUtil.terminateCalled()); - assertSame("ExitUtil.terminate should store its 1st call's ExitException", - ee1, ExitUtil.getFirstExitException()); + // simulate/check 1st call + ExitException ee = intercept(ExitException.class, ()->ExitUtil.terminate(ee1)); + assertSame("ExitUtil.terminate should have rethrown its ExitException argument but it " + + "had thrown something else", ee1, ee); + assertTrue("ExitUtil.terminateCalled should be true after 1st ExitUtil.terminate call", + ExitUtil.terminateCalled()); + assertSame("ExitUtil.terminate should store its 1st call's ExitException", + ee1, ExitUtil.getFirstExitException()); - // simulate/check 2nd call not overwritting 1st one - ee = intercept(ExitException.class, ()->ExitUtil.terminate(ee2)); - assertSame("ExitUtil.terminate should have rethrown its HaltException argument but it " - + "had thrown something else", ee2, ee); - assertTrue("ExitUtil.terminateCalled should still be true after 2nd ExitUtil.terminate call", - ExitUtil.terminateCalled()); - // 2nd call rethrown the 2nd ExitException yet only the 1st only should have been stored - assertSame("ExitUtil.terminate when called twice should only remember 1st call's " - + "ExitException", ee1, ExitUtil.getFirstExitException()); + // simulate/check 2nd call not overwritting 1st one + ee = intercept(ExitException.class, ()->ExitUtil.terminate(ee2)); + assertSame("ExitUtil.terminate should have rethrown its HaltException argument but it " + + "had thrown something else", ee2, ee); + assertTrue("ExitUtil.terminateCalled should still be true after 2nd ExitUtil.terminate call", + ExitUtil.terminateCalled()); + // 2nd call rethrown the 2nd ExitException yet only the 1st only should have been stored + assertSame("ExitUtil.terminate when called twice should only remember 1st call's " + + "ExitException", ee1, ExitUtil.getFirstExitException()); - // simulate cleanup, also tries to make sure state is ok for all junit still has to do - ExitUtil.resetFirstExitException(); - assertFalse("ExitUtil.terminateCalled should be false after " - + "ExitUtil.resetFirstExitException call", ExitUtil.terminateCalled()); - assertNull("ExitUtil.getFirstExitException should be null after " - + "ExitUtil.resetFirstExitException call", ExitUtil.getFirstExitException()); - } finally { - // cleanup - ExitUtil.resetFirstExitException(); - } + // simulate cleanup, also tries to make sure state is ok for all junit still has to do + ExitUtil.resetFirstExitException(); + assertFalse("ExitUtil.terminateCalled should be false after " + + "ExitUtil.resetFirstExitException call", ExitUtil.terminateCalled()); + assertNull("ExitUtil.getFirstExitException should be null after " + + "ExitUtil.resetFirstExitException call", ExitUtil.getFirstExitException()); } @Test @@ -83,40 +92,36 @@ public void testGetSetHaltExceptions() throws Throwable { ExitUtil.resetFirstHaltException(); HaltException he1 = new HaltException(1, "TestExitUtil forged 1st HaltException"); HaltException he2 = new HaltException(2, "TestExitUtil forged 2nd HaltException"); - try { - // check proper initial settings - assertFalse("ExitUtil.haltCalled initial value should be false", - ExitUtil.haltCalled()); - assertNull("ExitUtil.getFirstHaltException initial value should be null", - ExitUtil.getFirstHaltException()); - // simulate/check 1st call - HaltException he = intercept(HaltException.class, ()->ExitUtil.halt(he1)); - assertSame("ExitUtil.halt should have rethrown its HaltException argument but it had " - +"thrown something else", he1, he); - assertTrue("ExitUtil.haltCalled should be true after 1st ExitUtil.halt call", - ExitUtil.haltCalled()); - assertSame("ExitUtil.halt should store its 1st call's HaltException", - he1, ExitUtil.getFirstHaltException()); + // check proper initial settings + assertFalse("ExitUtil.haltCalled initial value should be false", + ExitUtil.haltCalled()); + assertNull("ExitUtil.getFirstHaltException initial value should be null", + ExitUtil.getFirstHaltException()); + + // simulate/check 1st call + HaltException he = intercept(HaltException.class, ()->ExitUtil.halt(he1)); + assertSame("ExitUtil.halt should have rethrown its HaltException argument but it had " + +"thrown something else", he1, he); + assertTrue("ExitUtil.haltCalled should be true after 1st ExitUtil.halt call", + ExitUtil.haltCalled()); + assertSame("ExitUtil.halt should store its 1st call's HaltException", + he1, ExitUtil.getFirstHaltException()); - // simulate/check 2nd call not overwritting 1st one - he = intercept(HaltException.class, ()->ExitUtil.halt(he2)); - assertSame("ExitUtil.halt should have rethrown its HaltException argument but it had " - +"thrown something else", he2, he); - assertTrue("ExitUtil.haltCalled should still be true after 2nd ExitUtil.halt call", - ExitUtil.haltCalled()); - assertSame("ExitUtil.halt when called twice should only remember 1st call's HaltException", - he1, ExitUtil.getFirstHaltException()); + // simulate/check 2nd call not overwritting 1st one + he = intercept(HaltException.class, ()->ExitUtil.halt(he2)); + assertSame("ExitUtil.halt should have rethrown its HaltException argument but it had " + +"thrown something else", he2, he); + assertTrue("ExitUtil.haltCalled should still be true after 2nd ExitUtil.halt call", + ExitUtil.haltCalled()); + assertSame("ExitUtil.halt when called twice should only remember 1st call's HaltException", + he1, ExitUtil.getFirstHaltException()); - // simulate cleanup, also tries to make sure state is ok for all junit still has to do - ExitUtil.resetFirstHaltException(); - assertFalse("ExitUtil.haltCalled should be false after " - + "ExitUtil.resetFirstHaltException call", ExitUtil.haltCalled()); - assertNull("ExitUtil.getFirstHaltException should be null after " - + "ExitUtil.resetFirstHaltException call", ExitUtil.getFirstHaltException()); - } finally { - // cleanup, useless if last test succeed, useful if not - ExitUtil.resetFirstHaltException(); - } + // simulate cleanup, also tries to make sure state is ok for all junit still has to do + ExitUtil.resetFirstHaltException(); + assertFalse("ExitUtil.haltCalled should be false after " + + "ExitUtil.resetFirstHaltException call", ExitUtil.haltCalled()); + assertNull("ExitUtil.getFirstHaltException should be null after " + + "ExitUtil.resetFirstHaltException call", ExitUtil.getFirstHaltException()); } } From 518fefc06263d557206b21afe17ceb5338025371 Mon Sep 17 00:00:00 2001 From: Remi Catherinot Date: Tue, 12 Jul 2022 14:44:28 +0200 Subject: [PATCH 5/5] HADOOP-18217. fixed javadoc --- .../src/main/java/org/apache/hadoop/util/ExitUtil.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java index ab96ecc7a76e8..dd47aeeefac2c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java @@ -207,6 +207,9 @@ public static void resetFirstHaltException() { /** * Suppresses if legit and returns the first non-null of the two. Legit means * suppressor if neither null nor suppressed. + * @param suppressor Throwable that suppresses suppressed + * @param suppressed Throwable that is suppressed by suppressor + * @return suppressor if not null, suppressed otherwise */ private static T addSuppressed(T suppressor, T suppressed) { if (suppressor == null) {