Skip to content

Commit

Permalink
HADOOP-18217. ExitUtil synchronized blocks reduced. #4255
Browse files Browse the repository at this point in the history
Reduce the ExitUtil synchronized block scopes so System.exit
and Runtime.halt calls aren't within their boundaries,
so ExitUtil wrappers do not block each other.

Enlarged catches to all Throwables (not just Exceptions).

Contributed by Remi Catherinot
  • Loading branch information
HerCath authored and steveloughran committed Jul 13, 2022
1 parent f4e8a4f commit e0abdd8
Show file tree
Hide file tree
Showing 2 changed files with 243 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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<ExitException> FIRST_EXIT_EXCEPTION =
new AtomicReference<>();
private static final AtomicReference<HaltException> 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}. */
Expand Down Expand Up @@ -159,92 +163,166 @@ 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();
}

/**
* Reset the tracking of process termination. This is for use in unit tests
* 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);
}

/**
* Suppresses if legit and returns the first non-null of the two. Legit means
* <code>suppressor</code> if neither <code>null</code> nor <code>suppressed</code>.
* @param suppressor <code>Throwable</code> that suppresses <code>suppressed</code>
* @param suppressed <code>Throwable</code> that is suppressed by <code>suppressor</code>
* @return <code>suppressor</code> if not <code>null</code>, <code>suppressed</code> otherwise
*/
private static <T extends Throwable> 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,
* 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 <code>ee</code>
*/
public static synchronized void terminate(ExitException ee)
throws ExitException {
int status = ee.getExitCode();
String msg = ee.getMessage();
public static void terminate(final ExitException ee) throws ExitException {
final int status = ee.getExitCode();
Error caught = 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) {
// 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 suppressed
addSuppressed(ee, t);
}
}
if (systemExitDisabled) {
LOG.error("Terminate called", ee);
if (!terminateCalled()) {
firstExitException = ee;
try {
LOG.error("Terminate called", ee);
} catch (Error e) {
// errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it
caught = addSuppressed(caught, e);
} catch (Throwable t) {
// all other kind of throwables are suppressed
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
System.exit(status);
}
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 <code>he</code>.
* 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 <code>he</code>
*/
public static synchronized void halt(HaltException ee) throws HaltException {
int status = ee.getExitCode();
String msg = ee.getMessage();
try {
if (status != 0) {
//exit indicates a problem, log it
LOG.info("Halt with status {}: {}", status, msg, ee);
public static void halt(final HaltException he) throws HaltException {
final int status = he.getExitCode();
Error caught = null;
if (status != 0) {
try {
// exit indicates a problem, log it
String msg = he.getMessage();
LOG.info("Halt with status {}: {}", status, msg, he);
} catch (Error e) {
// 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 suppressed
addSuppressed(he, t);
}
} catch (Exception ignored) {
// ignore exceptions here, as it may be due to an out of memory situation
}
// systemHaltDisabled is volatile and not used in scenario nheding atomicty,
// thus it does not nhed a synchronized access nor a atomic access
if (systemHaltDisabled) {
LOG.error("Halt called", ee);
if (!haltCalled()) {
firstHaltException = ee;
try {
LOG.error("Halt called", he);
} catch (Error e) {
// errors have higher priority again, if it's a 2nd error, the 1st one suprpesses it
caught = addSuppressed(caught, e);
} catch (Throwable t) {
// all other kind of throwables are suppressed
addSuppressed(he, t);
}
throw ee;
FIRST_HALT_EXCEPTION.compareAndSet(null, he);
if (caught != null) {
caught.addSuppressed(he);
throw caught;
}
// not suppressed by a higher prority error
throw he;
} else {
// when halt is enabled, whatever Throwable happened, we halt the VM
Runtime.getRuntime().halt(status);
}
Runtime.getRuntime().halt(status);
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/**
* 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.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.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 {

@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
ExitException ee1 = new ExitException(1, "TestExitUtil forged 1st ExitException");
ExitException ee2 = new ExitException(2, "TestExitUtil forged 2nd ExitException");
// 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 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());
}

@Test
public void testGetSetHaltExceptions() throws Throwable {
// 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");

// 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 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());
}
}

0 comments on commit e0abdd8

Please sign in to comment.