You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@reef.apache.org by "Dongjoon Hyun (JIRA)" <ji...@apache.org> on 2015/10/24 21:05:27 UTC

[jira] [Updated] (REEF-865) RunningTaskImpl should close tasks in both INIT and RUNNING status

     [ https://issues.apache.org/jira/browse/REEF-865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Dongjoon Hyun updated REEF-865:
-------------------------------
       Priority: Major  (was: Minor)
    Description: 
`TaskRepresenter` is initially created in ReefServiceProtos.State.INIT status and the current implementation of `RunningTaskImpl` can not close a task if its status is not RUNNING status. This was the main root cause of `testFailTaskClose` failures in Dockerized environment. 

This issue make `RunningTaskImpl` can close a task with INIT or RUNNING.
{code}
   public void close() {
     LOG.log(Level.FINEST, "CLOSE: TaskRuntime id[" + taskId + "] on evaluator id[" + evaluatorManager.getId() + "]");
 
-    if (this.taskRepresenter.isNotRunning()) {
-      LOG.log(Level.FINE, "Ignoring call to .close() because the task is no longer RUNNING.");
-    } else {
+    if (this.taskRepresenter.isClosable()) {
       final ContextControlProto contextControlProto = ContextControlProto.newBuilder()
           .setStopTask(StopTaskProto.newBuilder().build())
           .build();
       this.evaluatorManager.sendContextControlMessage(contextControlProto);
+    } else {
+      LOG.log(Level.FINE, "Ignoring call to .close() because the task is no longer RUNNING.");
     }
   }
{code}

  was:
FailTaskStop/FailTaskClose fails frequently in Docker environment with the following error messages.
 
{code}
org.apache.reef.runtime.common.driver.defaults.DefaultClientCloseHandler onNext
WARNING: Received a close message from the client, but no handler was bound for it. Active threads:
{code}

In a successful case, FailTaskStop/FailTaskClose register their handlers. In a error case like the above, 'registering' is not executed yet. So, `DefaultClientCloseHandler.onNext()` is invoked and the test case hangs until its timeout occurs.

By adding a sleep time in `Driver.java` of this testsuite, we can make the above test cases robust on resource-limited environments. 
{code}
      // In case of slow machines, it takes some time until task is ready.
      try {
        Thread.sleep(1000);
      } catch (final InterruptedException ex) {
        LOG.log(Level.INFO, "Interrupted: {0}", ex);
      }
{code}

Note, this does not change any functionality of this testcases.

I experimented this fix on my labtop docker and travis-ci.org 10 times. 

https://travis-ci.org/dongjoon-hyun/incubator-reef/builds

    Component/s:     (was: REEF-Tests)
                 REEF-Common
     Issue Type: Bug  (was: Improvement)
        Summary: RunningTaskImpl should close tasks in both INIT and RUNNING status  (was: Make FailTask testsuite robust by adding sleeping to wait for tasks ready )

> RunningTaskImpl should close tasks in both INIT and RUNNING status
> ------------------------------------------------------------------
>
>                 Key: REEF-865
>                 URL: https://issues.apache.org/jira/browse/REEF-865
>             Project: REEF
>          Issue Type: Bug
>          Components: REEF-Common
>    Affects Versions: 0.14
>            Reporter: Dongjoon Hyun
>            Assignee: Dongjoon Hyun
>
> `TaskRepresenter` is initially created in ReefServiceProtos.State.INIT status and the current implementation of `RunningTaskImpl` can not close a task if its status is not RUNNING status. This was the main root cause of `testFailTaskClose` failures in Dockerized environment. 
> This issue make `RunningTaskImpl` can close a task with INIT or RUNNING.
> {code}
>    public void close() {
>      LOG.log(Level.FINEST, "CLOSE: TaskRuntime id[" + taskId + "] on evaluator id[" + evaluatorManager.getId() + "]");
>  
> -    if (this.taskRepresenter.isNotRunning()) {
> -      LOG.log(Level.FINE, "Ignoring call to .close() because the task is no longer RUNNING.");
> -    } else {
> +    if (this.taskRepresenter.isClosable()) {
>        final ContextControlProto contextControlProto = ContextControlProto.newBuilder()
>            .setStopTask(StopTaskProto.newBuilder().build())
>            .build();
>        this.evaluatorManager.sendContextControlMessage(contextControlProto);
> +    } else {
> +      LOG.log(Level.FINE, "Ignoring call to .close() because the task is no longer RUNNING.");
>      }
>    }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)