[
https://issues.apache.org/jira/browse/HADOOP-18217?focusedWorklogId=775903&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-775903
]
ASF GitHub Bot logged work on HADOOP-18217:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 30/May/22 13:18
Start Date: 30/May/22 13:18
Worklog Time Spent: 10m
Work Description: steveloughran commented on code in PR #4255:
URL: https://github.com/apache/hadoop/pull/4255#discussion_r884819180
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java:
##########
@@ -159,138 +163,155 @@ 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);
}
/**
+ * 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 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);
}
}
if (systemExitDisabled) {
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) {
Review Comment:
do you think we need a safety check to see if ee != caught before
addSuppressed()? we've had problems with abfs in the past, where a cached
exception was being rethrown in close()
##########
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);
Review Comment:
use LambdaTestUtils.intercept() to make the call; and we can use assertSame
for equivalenced
```
ExitException ex = intercept(ExitException.class, () ->
ExitUtil.terminate(ee1));
assertSame("ExitUtil.terminate should have rethrown its ExitException
argument but it "
+"had thrown something else", ee1, ee);
```
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java:
##########
@@ -159,138 +163,155 @@ 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;
Review Comment:
nit, add some spaces around !=
##########
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",
Review Comment:
assertSame
##########
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());
Review Comment:
add messages here, so junit test failures woudl be meaningful.
##########
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);
Review Comment:
intercept() and assertSame again
##########
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() {
Review Comment:
same intercept and assertSame recommendations as the previous test
Issue Time Tracking
-------------------
Worklog Id: (was: 775903)
Time Spent: 1h 40m (was: 1.5h)
> shutdownhookmanager should not be multithreaded (deadlock possible)
> -------------------------------------------------------------------
>
> Key: HADOOP-18217
> URL: https://issues.apache.org/jira/browse/HADOOP-18217
> Project: Hadoop Common
> Issue Type: Bug
> Components: util
> Affects Versions: 2.10.1
> Environment: linux, windows, any version
> Reporter: Catherinot Remi
> Priority: Minor
> Labels: pull-request-available
> Attachments: wtf.java
>
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
> the ShutdownHookManager class uses an executor to run hooks to have a
> "timeout" notion around them. It does this using a single threaded executor.
> It can leads to deadlock leaving a never-shutting-down JVM with this
> execution flow:
> * JVM need to exit (only daemon threads remaining or someone called
> System.exit)
> * ShutdowHookManager kicks in
> * SHMngr executor start running some hooks
> * SHMngr executor thread kicks in and, as a side effect, run some code from
> one of the hook that calls System.exit (as a side effect from an external lib
> for example)
> * the executor thread is waiting for a lock because another thread already
> entered System.exit and has its internal lock, so the executor never returns.
> * SHMngr never returns
> * 1st call to System.exit never returns
> * JVM stuck
>
> using an executor with a single thread does "fake" timeouts (the task keeps
> running, you can interrupt it but until it stumble upon some piece of code
> that is interruptible (like an IO) it will keep running) especially since the
> executor is a single threaded one. So it has this bug for example :
> * caller submit 1st hook (bad one that would need 1 hour of runtime and that
> cannot be interrupted)
> * executor start 1st hook
> * caller of the future 1st hook result timeout
> * caller submit 2nd hook
> * bug : 1 hook still running, 2nd hook triggers a timeout but never got the
> chance to run anyway, so 1st faulty hook makes it impossible for any other
> hook to have a chance to run, so running hooks in a single separate thread
> does not allow to run other hooks in parallel to long ones.
>
> If we really really want to timeout the JVM shutdown, even accepting maybe
> dirty shutdown, it should rather handle the hooks inside the initial thread
> (not spawning new one(s) so not triggering the deadlock described on the 1st
> place) and if a timeout was configured, only spawn a single parallel daemon
> thread that sleeps the timeout delay, and then use Runtime.halt (which bypass
> the hook system so should not trigger the deadlock). If the normal
> System.exit ends before the timeout delay everything is fine. If the
> System.exit took to much time, the JVM is killed and so the reason why this
> multithreaded shutdown hook implementation was created is satisfied (avoding
> having hanging JVMs)
>
> Had the bug with both oracle and open jdk builds, all in 1.8 major version.
> hadoop 2.6 and 2.7 did not have the issue because they do not run hooks in
> another thread
>
> Another solution is of course to configure the timeout AND to have as many
> threads as needed to run the hooks so to have at least some gain to offset
> the pain of the dealock scenario
>
> EDIT: added some logs and reproduced the problem. in fact it is located after
> triggering all the hook entries and before shutting down the executor.
> Current code, after running the hooks, creates a new Configuration object and
> reads the configured timeout from it, applies this timeout to shutdown the
> executor. I sometimes run with a classloader doing remote classloading,
> Configuration loads its content using this classloader, so when shutting down
> the JVM and some network error occurs the classloader fails to load the
> ressources needed by Configuration. So the code crash before shutting down
> the executor and ends up inside the thread's default uncaught throwable
> handler, which was calling System.exit, so got stuck, so shutting down the
> executor never returned, so does the JVM.
> So, forget about the halt stuff (even if it is a last ressort very robust
> safety net). Still I'll do a small adjustement to the final executor shutdown
> code to be slightly more robust to even the strangest exceptions/errors it
> encounters.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]