Repository: flink
Updated Branches:
  refs/heads/release-1.2 852a710b4 -> 52b6e2fda


[FLINK-5718] [core] Remove unnecessary configuration parameters


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

Branch: refs/heads/release-1.2
Commit: 52b6e2fda6681ada64bb4fdae90b1c9bbb448bb2
Parents: daa5469
Author: Till Rohrmann <trohrm...@apache.org>
Authored: Wed May 3 17:16:10 2017 +0200
Committer: Till Rohrmann <trohrm...@apache.org>
Committed: Fri May 5 09:28:51 2017 +0200

----------------------------------------------------------------------
 .../org/apache/flink/runtime/taskmanager/TaskManager.scala   | 3 +--
 .../flink/runtime/taskmanager/TaskManagerConfiguration.scala | 8 +++-----
 .../apache/flink/runtime/util/JvmExitOnFatalErrorTest.java   | 3 ++-
 3 files changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/52b6e2fd/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git 
a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 
b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index bb93fa1..bc63655 100644
--- 
a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ 
b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -2382,8 +2382,7 @@ object TaskManager {
       configuration,
       initialRegistrationPause,
       maxRegistrationPause,
-      refusedRegistrationPause,
-      configuration.getBoolean(TaskManagerOptions.KILL_ON_OUT_OF_MEMORY))
+      refusedRegistrationPause)
 
     (taskManagerConfig, networkConfig, taskManagerInetSocketAddress, memType)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/52b6e2fd/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
----------------------------------------------------------------------
diff --git 
a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
 
b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
index 929ff55..aab3c5f 100644
--- 
a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
+++ 
b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.taskmanager
 
 import java.util.concurrent.TimeUnit
 
-import org.apache.flink.configuration.{Configuration, TaskManagerOptions}
+import org.apache.flink.configuration.Configuration
 
 import scala.concurrent.duration.FiniteDuration
 
@@ -33,8 +33,7 @@ case class TaskManagerConfiguration(
     configuration: Configuration,
     initialRegistrationPause: FiniteDuration,
     maxRegistrationPause: FiniteDuration,
-    refusedRegistrationPause: FiniteDuration,
-    exitJvmOnOutOfMemory: Boolean) {
+    refusedRegistrationPause: FiniteDuration) {
 
   def this(
       tmpDirPaths: Array[String],
@@ -52,7 +51,6 @@ case class TaskManagerConfiguration(
       configuration,
       FiniteDuration(500, TimeUnit.MILLISECONDS),
       FiniteDuration(30, TimeUnit.SECONDS),
-      FiniteDuration(10, TimeUnit.SECONDS),
-      configuration.getBoolean(TaskManagerOptions.KILL_ON_OUT_OF_MEMORY))
+      FiniteDuration(10, TimeUnit.SECONDS))
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/52b6e2fd/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
----------------------------------------------------------------------
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
index bf75549..8e49559 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
@@ -58,6 +58,7 @@ import org.apache.flink.runtime.testutils.TestJvmProcess;
 import org.apache.flink.util.OperatingSystem;
 import org.apache.flink.util.SerializedValue;
 
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import java.io.File;
@@ -73,7 +74,7 @@ import static org.mockito.Mockito.*;
  * Test that verifies the behavior of blocking shutdown hooks and of the
  * {@link JvmShutdownSafeguard} that guards against it.
  */
-public class JvmExitOnFatalErrorTest {
+public class JvmExitOnFatalErrorTest extends TestLogger {
 
        @Test
        public void testExitJvmOnOutOfMemory() throws Exception {

Reply via email to