flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From u..@apache.org
Subject [1/2] flink git commit: [FLINK-5745] [runtime] Extract ExecutorThreadFactory#FatalExitExceptionHandler
Date Wed, 15 Feb 2017 10:31:50 GMT
Repository: flink
Updated Branches:
  refs/heads/master 87d09342b -> 9b4cd349f


[FLINK-5745] [runtime] Extract ExecutorThreadFactory#FatalExitExceptionHandler

Make it a top-level class so that it can be re-used.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b455f22c
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b455f22c
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b455f22c

Branch: refs/heads/master
Commit: b455f22c9bcac565945cef4897af50c20ded94b5
Parents: 87d0934
Author: Nico Kruber <nico@data-artisans.com>
Authored: Fri Feb 10 18:28:34 2017 +0100
Committer: Ufuk Celebi <uce@apache.org>
Committed: Wed Feb 15 11:26:24 2017 +0100

----------------------------------------------------------------------
 .../runtime/util/ExecutorThreadFactory.java     | 15 -------
 .../runtime/util/FatalExitExceptionHandler.java | 42 ++++++++++++++++++++
 2 files changed, 42 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b455f22c/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
index 4a79db3..7673111 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
@@ -22,9 +22,6 @@ import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -116,16 +113,4 @@ public class ExecutorThreadFactory implements ThreadFactory {
 
 	// --------------------------------------------------------------------------------------------
 
-	private static final class FatalExitExceptionHandler implements UncaughtExceptionHandler
{
-
-		private static final Logger LOG = LoggerFactory.getLogger(FatalExitExceptionHandler.class);
-
-		static final FatalExitExceptionHandler INSTANCE = new FatalExitExceptionHandler(); 
-
-		@Override
-		public void uncaughtException(Thread t, Throwable e) {
-			LOG.error("FATAL: Thread '" + t.getName() + "' produced an uncaught exception. Stopping
the process...", e);
-			System.exit(-17);
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b455f22c/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java
new file mode 100644
index 0000000..c57b75a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java
@@ -0,0 +1,42 @@
+/*
+ * 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.flink.runtime.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handler for uncaught exceptions that will log the exception and kill the process afterwards.
+ *
+ * <p>This guarantees that critical exceptions are not accidentally lost and leave
the system
+ * running in an inconsistent state.
+ */
+public final class FatalExitExceptionHandler implements Thread.UncaughtExceptionHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FatalExitExceptionHandler.class);
+
+	public static final FatalExitExceptionHandler INSTANCE = new FatalExitExceptionHandler();
+
+	@Override
+	public void uncaughtException(Thread t, Throwable e) {
+		LOG.error("FATAL: Thread '" + t.getName() +
+			"' produced an uncaught exception. Stopping the process...", e);
+		System.exit(-17);
+	}
+}


Mime
View raw message