Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.samza.container;

import java.lang.Thread.UncaughtExceptionHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* An UncaughtExceptionHandler for SamzaContainer that simply executes the configured {@link #runnable}
* when any thread throws an uncaught exception.
*/
public class SamzaContainerExceptionHandler implements UncaughtExceptionHandler {
private static final Logger LOGGER = LoggerFactory.getLogger(SamzaContainerExceptionHandler.class);
private final Runnable runnable;

public SamzaContainerExceptionHandler(Runnable runnable) {
this.runnable = runnable;
}
/**
* Method invoked when the given thread terminates due to the
* given uncaught exception.
* <p>Any exception thrown by this method will be ignored by the
* Java Virtual Machine.
*
* @param t the thread
* @param e the exception
*/
@Override
public void uncaughtException(Thread t, Throwable e) {
LOGGER.error(
String.format("Uncaught exception in thread (name=%s). Exiting process now.", t.getName()), e);
e.printStackTrace(System.err);
try {
runnable.run();
} catch (Throwable throwable) {
// Ignore to avoid further exception propagation
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

package org.apache.samza.runtime;

import java.util.HashMap;
import java.util.Random;
import org.apache.log4j.MDC;
import org.apache.samza.SamzaException;
import org.apache.samza.application.StreamApplication;
Expand All @@ -29,6 +27,7 @@
import org.apache.samza.config.ShellCommandConfig;
import org.apache.samza.container.SamzaContainer;
import org.apache.samza.container.SamzaContainer$;
import org.apache.samza.container.SamzaContainerExceptionHandler;
import org.apache.samza.job.ApplicationStatus;
import org.apache.samza.job.model.ContainerModel;
import org.apache.samza.job.model.JobModel;
Expand All @@ -40,6 +39,9 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.HashMap;
import java.util.Random;

/**
* LocalContainerRunner is the local runner for Yarn {@link SamzaContainer}s. It is an intermediate step to
* have a local runner for yarn before we consolidate the Yarn container and coordination into a
Expand Down Expand Up @@ -99,11 +101,11 @@ public ApplicationStatus status(StreamApplication streamApp) {
}

public static void main(String[] args) throws Exception {
setExceptionHandler(() -> {
log.info("Exiting process now.");
System.exit(1);
});

Thread.setDefaultUncaughtExceptionHandler(
new SamzaContainerExceptionHandler(() -> {
log.info("Exiting process now.");
System.exit(1);
}));
String containerId = System.getenv(ShellCommandConfig.ENV_CONTAINER_ID());
log.info(String.format("Got container ID: %s", containerId));
String coordinatorUrl = System.getenv(ShellCommandConfig.ENV_COORDINATOR_URL());
Expand All @@ -124,13 +126,4 @@ public static void main(String[] args) throws Exception {
StreamApplication streamApp = TaskFactoryUtil.createStreamApplication(config);
new LocalContainerRunner(jobModel, containerId).run(streamApp);
}

/* package private */ static void setExceptionHandler(Runnable runnable) {
Thread.UncaughtExceptionHandler exceptionHandler = (t, e) -> {
log.error(String.format("Uncaught exception in thread (name=%s).", t.getName()), e);
e.printStackTrace(System.err);
runnable.run();
};
Thread.setDefaultUncaughtExceptionHandler(exceptionHandler);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -628,6 +628,7 @@ class SamzaContainer(

val shutdownMs = containerContext.config.getShutdownMs.getOrElse(5000L)
private val runLoopStartLatch: CountDownLatch = new CountDownLatch(1)
var shutdownHookThread: Thread = null

def awaitStart(timeoutMs: Long): Boolean = {
try {
Expand Down Expand Up @@ -665,6 +666,8 @@ class SamzaContainer(
} finally {
info("Shutting down.")

removeShutdownHook

shutdownConsumers
shutdownTask
shutdownStores
Expand Down Expand Up @@ -803,21 +806,37 @@ class SamzaContainer(

def addShutdownHook {
val runLoopThread = Thread.currentThread()
Runtime.getRuntime().addShutdownHook(new Thread() {
shutdownHookThread = new Thread("CONTAINER-SHUTDOWN-HOOK") {
override def run() = {
info("Shutting down, will wait up to %s ms" format shutdownMs)
runLoop match {
case runLoop: RunLoop => runLoop.shutdown
case asyncRunLoop: AsyncRunLoop => asyncRunLoop.shutdown()
}
runLoopThread.join(shutdownMs)
if (runLoopThread.isAlive) {
warn("Did not shut down within %s ms, exiting" format shutdownMs)
} else {
try {
runLoopThread.join(shutdownMs)
} catch {
case e: Throwable => // Ignore to avoid deadlock with uncaughtExceptionHandler. See SAMZA-1220
error("Did not shut down within %s ms, exiting" format shutdownMs, e)
}
if (!runLoopThread.isAlive) {
info("Shutdown complete")
}
}
})
}
Runtime.getRuntime().addShutdownHook(shutdownHookThread)
}

def removeShutdownHook = {
try {
if (shutdownHookThread != null) {
Runtime.getRuntime.removeShutdownHook(shutdownHookThread)
}
} catch {
case e: IllegalStateException => {
// Thrown when then JVM is already shutting down, so safe to ignore.
}
}
}

def shutdownConsumers {
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -17,20 +17,23 @@
* under the License.
*/

package org.apache.samza.container
package org.apache.samza.container;

import org.junit.Test
import org.junit.Assert._
import org.junit.Before
import org.apache.samza.SamzaException
import org.junit.After
import org.apache.samza.SamzaException;
import org.junit.Test;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor: organize these imports.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That is how IntelliJ organizes imports. Were you referring some specific ordering?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense, I thought the junit imports could have been potentially grouped together. I think I may have missed the static clause in the assertTrue.


import java.util.concurrent.atomic.AtomicBoolean;

import static org.junit.Assert.assertTrue;

public class TestSamzaContainerExceptionHandler {

class TestSamzaContainerExceptionHandler {
@Test
def testShutdownProcess {
var exitCalled = false
val exceptionHandler = new SamzaContainerExceptionHandler(() => exitCalled = true)
exceptionHandler.uncaughtException(Thread.currentThread, new SamzaException)
assertTrue(exitCalled)
public void testExceptionHandler() {
final AtomicBoolean exitCalled = new AtomicBoolean(false);
Thread.UncaughtExceptionHandler exceptionHandler =
new SamzaContainerExceptionHandler(() -> exitCalled.getAndSet(true));
exceptionHandler.uncaughtException(Thread.currentThread(), new SamzaException());
assertTrue(exitCalled.get());
}
}

This file was deleted.