[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-23 Thread zentol
Github user zentol closed the pull request at:

https://github.com/apache/flink/pull/2974


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-23 Thread zentol
Github user zentol commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r97354396
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
 ---
@@ -1074,6 +1080,79 @@ protected void run() {
}};
}
 
+   @Test
+   public void testLogNotFoundHandling() throws Exception {
+
+   new JavaTestKit(system){{
+
+   ActorGateway jobManager = null;
+   ActorGateway taskManager = null;
+
+   final ActorGateway testActorGateway = new 
AkkaActorGateway(
+   getTestActor(),
+   leaderSessionID);
+
+   try {
+   final IntermediateDataSetID resultId = new 
IntermediateDataSetID();
+
+   // Create the JM
+   ActorRef jm = system.actorOf(Props.create(
+   new 
SimplePartitionStateLookupJobManagerCreator(leaderSessionID, getTestActor(;
+
+   jobManager = new AkkaActorGateway(jm, 
leaderSessionID);
+
+   final int dataPort = 
NetUtils.getAvailablePort();
+   Configuration config = new Configuration();
+   
config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_MAX, 200);
+   
config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, "/i/dont/exist");
+
+   taskManager = TestingUtils.createTaskManager(
+   system,
+   jobManager,
--- End diff --

Without a JobManager no BlobService is started. This means the TM would 
fail earlier then we want him to.

While trying it out i found another exception that should be wrapped though 
:>


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-23 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r97346693
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
 ---
@@ -1074,6 +1080,79 @@ protected void run() {
}};
}
 
+   @Test
+   public void testLogNotFoundHandling() throws Exception {
+
+   new JavaTestKit(system){{
+
+   ActorGateway jobManager = null;
+   ActorGateway taskManager = null;
+
+   final ActorGateway testActorGateway = new 
AkkaActorGateway(
+   getTestActor(),
+   leaderSessionID);
+
+   try {
+   final IntermediateDataSetID resultId = new 
IntermediateDataSetID();
+
+   // Create the JM
+   ActorRef jm = system.actorOf(Props.create(
+   new 
SimplePartitionStateLookupJobManagerCreator(leaderSessionID, getTestActor(;
+
+   jobManager = new AkkaActorGateway(jm, 
leaderSessionID);
+
+   final int dataPort = 
NetUtils.getAvailablePort();
+   Configuration config = new Configuration();
+   
config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_MAX, 200);
+   
config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, "/i/dont/exist");
+
+   taskManager = TestingUtils.createTaskManager(
+   system,
+   jobManager,
+   config,
+   false,
+   true);
+
+   // 
-
+
+   final ActorGateway tm = taskManager;
+   final ExecutionContextExecutor context = 
ExecutionContext$.MODULE$.fromExecutor(Executors.newSingleThreadExecutor());
+
+   new Within(d) {
+   @Override
+   protected void run() {
+   Future logFuture = 
tm.ask(TaskManagerMessages.getRequestTaskManagerLog(), timeout);
+
+   logFuture.onSuccess(new 
OnSuccess() {
+   @Override
+   public void 
onSuccess(Object result) throws Throwable {
+   
Assert.fail();
+   }
+   }, context);
+   logFuture.onFailure(new 
OnFailure() {
+   @Override
+   public void 
onFailure(Throwable failure) throws Throwable {
+   
testActorGateway.tell(new Status.Success("success"));
+   }
+   }, context);
--- End diff --

Maybe `Await.result` is a bit more succinct.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-23 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r97346342
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
 ---
@@ -1074,6 +1080,79 @@ protected void run() {
}};
}
 
+   @Test
+   public void testLogNotFoundHandling() throws Exception {
+
+   new JavaTestKit(system){{
+
+   ActorGateway jobManager = null;
+   ActorGateway taskManager = null;
+
+   final ActorGateway testActorGateway = new 
AkkaActorGateway(
+   getTestActor(),
+   leaderSessionID);
+
+   try {
+   final IntermediateDataSetID resultId = new 
IntermediateDataSetID();
+
+   // Create the JM
+   ActorRef jm = system.actorOf(Props.create(
+   new 
SimplePartitionStateLookupJobManagerCreator(leaderSessionID, getTestActor(;
+
+   jobManager = new AkkaActorGateway(jm, 
leaderSessionID);
+
+   final int dataPort = 
NetUtils.getAvailablePort();
+   Configuration config = new Configuration();
+   
config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_MAX, 200);
+   
config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, "/i/dont/exist");
+
+   taskManager = TestingUtils.createTaskManager(
+   system,
+   jobManager,
--- End diff --

maybe `ActorRef.noSender()` is enough here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-23 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r97347534
  
--- Diff: 
flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandlerTest.java
 ---
@@ -0,0 +1,134 @@
+/*
+ * 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.webmonitor.handlers;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.http.DefaultFullHttpRequest;
+import io.netty.handler.codec.http.HttpMethod;
+import io.netty.handler.codec.http.HttpVersion;
+import io.netty.handler.codec.http.router.Routed;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.CompletableFuture;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.webmonitor.JobManagerRetriever;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import scala.Option;
+import scala.collection.JavaConverters;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.Future$;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+public class TaskManagerLogHandlerTest {
+   @Test
+   public void testLogFetchingFailure() throws Exception {
+   // = setup TaskManager 
=
+   InstanceID tmID = new InstanceID();
+   ResourceID tmRID = new ResourceID(tmID.toString());
+   TaskManagerGateway taskManagerGateway = 
mock(TaskManagerGateway.class);
+   when(taskManagerGateway.getAddress()).thenReturn("/tm/address");
+
+   Instance taskManager = mock(Instance.class);
+   when(taskManager.getId()).thenReturn(tmID);
+   when(taskManager.getTaskManagerID()).thenReturn(tmRID);
+   
when(taskManager.getTaskManagerGateway()).thenReturn(taskManagerGateway);
+   CompletableFuture future = new 
FlinkCompletableFuture<>();
+   future.completeExceptionally(new IOException("failure"));
+   
when(taskManagerGateway.requestTaskManagerLog(any(Time.class))).thenReturn(future);
+
+   // = setup JobManager 
==
+
+   ActorGateway jobManagerGateway = mock(ActorGateway.class);
+   Object registeredTaskManagersAnswer = new 
JobManagerMessages.RegisteredTaskManagers(
+   
JavaConverters.collectionAsScalaIterableConverter(Collections.singletonList(taskManager)).asScala());
+
+   
when(jobManagerGateway.ask(isA(JobManagerMessages.RequestRegisteredTaskManagers$.class),
 any(FiniteDuration.class)))
+   
.thenReturn(Future$.MODULE$.successful(registeredTaskManagersAnswer));
+   

[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-23 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r97346835
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
 ---
@@ -1074,6 +1080,79 @@ protected void run() {
}};
}
 
+   @Test
+   public void testLogNotFoundHandling() throws Exception {
+
+   new JavaTestKit(system){{
+
+   ActorGateway jobManager = null;
+   ActorGateway taskManager = null;
+
+   final ActorGateway testActorGateway = new 
AkkaActorGateway(
+   getTestActor(),
+   leaderSessionID);
+
+   try {
+   final IntermediateDataSetID resultId = new 
IntermediateDataSetID();
+
+   // Create the JM
+   ActorRef jm = system.actorOf(Props.create(
+   new 
SimplePartitionStateLookupJobManagerCreator(leaderSessionID, getTestActor(;
+
+   jobManager = new AkkaActorGateway(jm, 
leaderSessionID);
+
+   final int dataPort = 
NetUtils.getAvailablePort();
+   Configuration config = new Configuration();
+   
config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100);
+   
config.setInteger(TaskManagerOptions.NETWORK_REQUEST_BACKOFF_MAX, 200);
+   
config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, "/i/dont/exist");
+
+   taskManager = TestingUtils.createTaskManager(
+   system,
+   jobManager,
+   config,
+   false,
+   true);
+
+   // 
-
+
+   final ActorGateway tm = taskManager;
+   final ExecutionContextExecutor context = 
ExecutionContext$.MODULE$.fromExecutor(Executors.newSingleThreadExecutor());
+
+   new Within(d) {
+   @Override
+   protected void run() {
+   Future logFuture = 
tm.ask(TaskManagerMessages.getRequestTaskManagerLog(), timeout);
+
+   logFuture.onSuccess(new 
OnSuccess() {
+   @Override
+   public void 
onSuccess(Object result) throws Throwable {
+   
Assert.fail();
+   }
+   }, context);
+   logFuture.onFailure(new 
OnFailure() {
+   @Override
+   public void 
onFailure(Throwable failure) throws Throwable {
+   
testActorGateway.tell(new Status.Success("success"));
+   }
+   }, context);
+   
+   Status.Success msg = 
expectMsgClass(Status.Success.class);
+   Assert.assertEquals("success", 
msg.status());
+   }
+   };
+   }
+   catch(Exception e) {
+   e.printStackTrace();
+   fail(e.getMessage());
--- End diff --

Better let the exception bubble up. Less code ;-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-19 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r96843830
  
--- Diff: 
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 ---
@@ -830,19 +830,24 @@ class TaskManager(
   case StdOutFileRequest =>
 new File(logFilePath.substring(0, logFilePath.length - 4) + 
".out");
 }
-val fis = new FileInputStream(file);
-Future {
-  val client: BlobClient = blobService.get.createClient()
-  client.put(fis);
-}(context.dispatcher)
-  .onComplete {
-case Success(value) => 
-  sender ! value
-  fis.close()
-case Failure(e) =>
-  sender ! e
-  fis.close()
+if (file.exists()) {
+  val fis = new FileInputStream(file);
+  Future {
+val client: BlobClient = blobService.get.createClient()
+client.put(fis);
   }(context.dispatcher)
+.onComplete {
+  case Success(value) =>
+sender ! value
+fis.close()
+  case Failure(e) =>
+sender ! e
+fis.close()
+}(context.dispatcher)
+} else {
+  sender ! new IOException("TaskManager log files are unavailable. 
" +
+"Log file does not exist.")
--- End diff --

Maybe we could add the path under which we've looked for the log file to 
the error message.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-19 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r96843696
  
--- Diff: 
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 ---
@@ -821,7 +821,7 @@ class TaskManager(
 val logFilePathOption = Option(config.configuration.getString(
   ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, 
System.getProperty("log.file")));
 logFilePathOption match {
-  case None => throw new IOException("TaskManager log files are 
unavailable. " +
+  case None => sender ! new IOException("TaskManager log files are 
unavailable. " +
--- End diff --

In order to send an `Exception` over the wire to the sender it has to be 
packed into `akka.actor.Status.Failure`. Otherwise it will only be sent as a 
normal response type.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-19 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r96843719
  
--- Diff: 
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 ---
@@ -830,19 +830,24 @@ class TaskManager(
   case StdOutFileRequest =>
 new File(logFilePath.substring(0, logFilePath.length - 4) + 
".out");
 }
-val fis = new FileInputStream(file);
-Future {
-  val client: BlobClient = blobService.get.createClient()
-  client.put(fis);
-}(context.dispatcher)
-  .onComplete {
-case Success(value) => 
-  sender ! value
-  fis.close()
-case Failure(e) =>
-  sender ! e
-  fis.close()
+if (file.exists()) {
+  val fis = new FileInputStream(file);
+  Future {
+val client: BlobClient = blobService.get.createClient()
+client.put(fis);
   }(context.dispatcher)
+.onComplete {
+  case Success(value) =>
+sender ! value
+fis.close()
+  case Failure(e) =>
+sender ! e
+fis.close()
+}(context.dispatcher)
+} else {
+  sender ! new IOException("TaskManager log files are unavailable. 
" +
--- End diff --

The same here with `akka.actor.Status.Failure`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2017-01-19 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/2974#discussion_r96843747
  
--- Diff: 
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 ---
@@ -830,19 +830,24 @@ class TaskManager(
   case StdOutFileRequest =>
 new File(logFilePath.substring(0, logFilePath.length - 4) + 
".out");
 }
-val fis = new FileInputStream(file);
-Future {
-  val client: BlobClient = blobService.get.createClient()
-  client.put(fis);
-}(context.dispatcher)
-  .onComplete {
-case Success(value) => 
-  sender ! value
-  fis.close()
-case Failure(e) =>
-  sender ! e
-  fis.close()
+if (file.exists()) {
+  val fis = new FileInputStream(file);
+  Future {
+val client: BlobClient = blobService.get.createClient()
+client.put(fis);
   }(context.dispatcher)
+.onComplete {
+  case Success(value) =>
+sender ! value
+fis.close()
+  case Failure(e) =>
+sender ! e
--- End diff --

And here `akka.actor.Status.Failure`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #2974: [FLINK-5298] TM checks that log file exists

2016-12-08 Thread zentol
GitHub user zentol opened a pull request:

https://github.com/apache/flink/pull/2974

[FLINK-5298] TM checks that log file exists

This PR slightly modifies the `TaskManager#handleRequestTaskManagerLog`. 
For one it verifies that the log file actually exists before opening it. 
Second, if the logFilePathOption is empty it no longer throws an IOException 
(which _could_ crash the TM) but instead forwards it to the sender.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/zentol/flink 5298_tm_log

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/2974.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #2974


commit c135d1940e61a4a80b274042e6e095f3369ec911
Author: zentol 
Date:   2016-12-08T18:28:12Z

[FLINK-5298] TM checks that log file exists




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---