[ 
https://issues.apache.org/jira/browse/FLINK-2111?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14718207#comment-14718207
 ] 

ASF GitHub Bot commented on FLINK-2111:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/750#discussion_r38179681
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
 ---
    @@ -218,4 +228,113 @@ public void testRequestPartitionState() throws 
Exception {
                        }
                }};
        }
    +
    +   @Test
    +   public void testStopSignal() throws Exception {
    +           new JavaTestKit(system) {{
    +                   // Setup
    +                   TestingCluster cluster = null;
    +
    +                   try {
    +                           cluster = startTestingCluster(2, 1, 
DEFAULT_AKKA_ASK_TIMEOUT());
    +
    +                           // Create a task
    +                           final JobVertex sender = new 
JobVertex("Sender");
    +                           sender.setParallelism(2);
    +                           
sender.setInvokableClass(StopableInvokable.class);
    +
    +                           final JobGraph jobGraph = new 
JobGraph("Blocking test job", JobType.STREAMING, sender);
    +                           final JobID jid = jobGraph.getJobID();
    +
    +                           final ActorGateway jm = 
cluster.getJobManagerGateway();
    +
    +                           // we can set the leader session ID to None 
because we don't use this gateway to send messages
    +                           final ActorGateway testActorGateway = new 
AkkaActorGateway(getTestActor(), Option.<UUID>empty());
    +
    +                           // Submit the job and wait for all vertices to 
be running
    +                           jm.tell(new SubmitJob(jobGraph, false), 
testActorGateway);
    +                           expectMsgClass(Status.Success.class);
    +
    +                           jm.tell(new StopJob(jid), testActorGateway);
    +
    +                           // - The test 
----------------------------------------------------------------------
    +                           expectMsgClass(StoppingSuccess.class);
    +                   }
    +                   catch (Exception e) {
    +                           e.printStackTrace();
    +                           fail(e.getMessage());
    --- End diff --
    
    A try-catch block does not necessarily have to have a `catch` block. Thus, 
it is an option to simply let the exception bubble up and get rid off `fail()`.


> Add "stop" signal to cleanly shutdown streaming jobs
> ----------------------------------------------------
>
>                 Key: FLINK-2111
>                 URL: https://issues.apache.org/jira/browse/FLINK-2111
>             Project: Flink
>          Issue Type: Improvement
>          Components: Distributed Runtime, JobManager, Local Runtime, 
> Streaming, TaskManager, Webfrontend
>            Reporter: Matthias J. Sax
>            Assignee: Matthias J. Sax
>            Priority: Minor
>
> Currently, streaming jobs can only be stopped using "cancel" command, what is 
> a "hard" stop with no clean shutdown.
> The new introduced "stop" signal, will only affect streaming source tasks 
> such that the sources can stop emitting data and shutdown cleanly, resulting 
> in a clean shutdown of the whole streaming job.
> This feature is a pre-requirment for 
> https://issues.apache.org/jira/browse/FLINK-1929



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

Reply via email to