[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-09 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


asfgit closed pull request #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java
deleted file mode 100644
index d729dbbf5ad..000
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.taskmanager;
-
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.TaskMessages;
-import org.apache.flink.util.Preconditions;
-
-/**
- * Implementation using {@link ActorGateway} to forward the messages.
- */
-public class ActorGatewayTaskExecutionStateListener implements 
TaskExecutionStateListener {
-
-   private final ActorGateway actorGateway;
-
-   public ActorGatewayTaskExecutionStateListener(ActorGateway 
actorGateway) {
-   this.actorGateway = Preconditions.checkNotNull(actorGateway);
-   }
-
-   @Override
-   public void notifyTaskExecutionStateChanged(TaskExecutionState 
taskExecutionState) {
-   TaskMessages.UpdateTaskExecutionState actorMessage = new 
TaskMessages.UpdateTaskExecutionState(taskExecutionState);
-
-   actorGateway.tell(actorMessage);
-   }
-}
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 92ae1676ed6..66f212f0fee 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -86,10 +86,8 @@
 import java.net.URL;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -205,9 +203,6 @@
/** Checkpoint notifier used to communicate with the 
CheckpointCoordinator. */
private final CheckpointResponder checkpointResponder;
 
-   /** All listener that want to be notified about changes in the task's 
execution state. */
-   private final List 
taskExecutionStateListeners;
-
/** The BLOB cache, from which the task can request BLOB files. */
private final BlobCacheService blobService;
 
@@ -348,7 +343,6 @@ public Task(
this.network = Preconditions.checkNotNull(networkEnvironment);
this.taskManagerConfig = 
Preconditions.checkNotNull(taskManagerConfig);
 
-   this.taskExecutionStateListeners = new CopyOnWriteArrayList<>();
this.metrics = metricGroup;
 
this.partitionProducerStateChecker = 
Preconditions.checkNotNull(partitionProducerStateChecker);
@@ -701,7 +695,6 @@ else if (current == ExecutionState.CANCELING) {
}
 
// notify everyone that we switched to running
-   notifyObservers(ExecutionState.RUNNING, null);
taskManagerActions.updateTaskExecutionState(new 
TaskExecutionState(jobId, executionId, ExecutionState.RUNNING));
 
// make sure the user code classloader is accessible 
thread-locally
@@ -729,10 +722,7 @@ else if 

[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-08 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on issue #6729: [FLINK-10386] [taskmanager] Remove 
legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427780989
 
 
   Alright @TisonKun. I'll do the removal of `notifyFinalState` as a follow up.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: TisonKun
>Assignee: TisonKun
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-08 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427768967
 
 
   @tillrohrmann sounds good. As you suggest, we can replace `notifyFinalState` 
with `updateTaskExecutionState` since there is no different from API view and a 
`updateTaskExecutionState` without final state seems wired.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: TisonKun
>Assignee: TisonKun
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-08 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r223286665
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -846,6 +824,23 @@ public void 
testSetsUserCodeClassLoaderForTimerThreadFactory() throws Throwable
//  Test Utilities
// 

 
+   private static void waitUntilExecutionState(Task task, ExecutionState 
exceptedState, Deadline deadline) {
 
 Review comment:
   This method can now be removed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: TisonKun
>Assignee: TisonKun
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-08 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427761218
 
 
   @tillrohrmann thanks for your review!


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-08 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r223285835
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -181,43 +175,26 @@
 */
@Test
public void testEarlyCanceling() throws Exception {
-   Deadline deadline = Deadline.fromNow(Duration.ofMinutes(2));
-   StreamConfig cfg = new StreamConfig(new Configuration());
+   final StreamConfig cfg = new StreamConfig(new Configuration());
cfg.setOperatorID(new OperatorID(4711L, 42L));
cfg.setStreamOperator(new SlowlyDeserializingOperator());
cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-   Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration());
+   final TaskManagerActions taskManagerActions = 
mock(TaskManagerActions.class);
+   final Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration(), taskManagerActions);
 
-   TestingExecutionStateListener testingExecutionStateListener = 
new TestingExecutionStateListener();
+   final TaskExecutionState state = new TaskExecutionState(
+   task.getJobID(), task.getExecutionId(), 
ExecutionState.RUNNING);
 
-   task.registerExecutionListener(testingExecutionStateListener);
task.startTaskThread();
 
-   Future running = 
testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING);
-
-   // wait until the task thread reached state RUNNING
-   ExecutionState executionState = 
running.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-
-   // make sure the task is really running
-   if (executionState != ExecutionState.RUNNING) {
-   fail("Task entered state " + task.getExecutionState() + 
" with error "
-   + 
ExceptionUtils.stringifyException(task.getFailureCause()));
-   }
+   verify(taskManagerActions, 
timeout(2000L)).updateTaskExecutionState(eq(state));
 
// send a cancel. because the operator takes a long time to 
deserialize, this should
// hit the task before the operator is deserialized
task.cancelExecution();
 
-   Future canceling = 
testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING);
-
-   executionState = canceling.get(deadline.timeLeft().toMillis(), 
TimeUnit.MILLISECONDS);
-
-   // the task should reach state canceled eventually
-   assertTrue(executionState == ExecutionState.CANCELING ||
-   executionState == ExecutionState.CANCELED);
-
-   task.getExecutingThread().join(deadline.timeLeft().toMillis());
 
 Review comment:
   You're right. Thanks for the clarification.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-08 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on issue #6729: [FLINK-10386] [taskmanager] Remove 
legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427755128
 
 
   Thanks for the clarification @TisonKun. I'll give it another pass with your 
comments in mind.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-05 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun edited a comment on issue #6729: [FLINK-10386] [taskmanager] Remove 
legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427071700
 
 
   All "Why are we removing these assertions?" is as you say, we do not send 
those state updates via `TaskManagerActions.updateTaskExecutionState`. Maybe we 
can verify 
`TaskManagerActions.notifyFatalError`/`TaskManagerActions.failTask`/`TaskManagerActions.notifyFinalState`
 but all of them don't include detail task state. **And I think verify the 
final state is enough for the test**
   
   Besides, as the communication with @Clark , `TaskTest` is a test based 
on legacy mode, there is a pull request #6778 to port it to FLIP-6 code base.
   
   I think the main issue is whether we replace `notifyObservers` with 
`TaskManagerActions.updateTaskExecutionState`. My opinion is not for now to 
hold the current(FLIP-6) message flow and defer the decision. If so, we remove 
those assertion because we don't send those message and the assertion should 
not hold.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222733909
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -181,43 +175,26 @@
 */
@Test
public void testEarlyCanceling() throws Exception {
-   Deadline deadline = Deadline.fromNow(Duration.ofMinutes(2));
-   StreamConfig cfg = new StreamConfig(new Configuration());
+   final StreamConfig cfg = new StreamConfig(new Configuration());
cfg.setOperatorID(new OperatorID(4711L, 42L));
cfg.setStreamOperator(new SlowlyDeserializingOperator());
cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-   Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration());
+   final TaskManagerActions taskManagerActions = 
mock(TaskManagerActions.class);
 
 Review comment:
   I introduce a `NoOpTaskManagerActions` (in fact move an existing inner class 
to public) and `spy` on it. Because here we are mainly concerned about a 
specific function call happened, `verify` is the natural way IMO. Otherwise we 
might use a blocking queue and verify the message queue, more complex without 
additional benefit.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222721494
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -181,43 +175,26 @@
 */
@Test
public void testEarlyCanceling() throws Exception {
-   Deadline deadline = Deadline.fromNow(Duration.ofMinutes(2));
-   StreamConfig cfg = new StreamConfig(new Configuration());
+   final StreamConfig cfg = new StreamConfig(new Configuration());
cfg.setOperatorID(new OperatorID(4711L, 42L));
cfg.setStreamOperator(new SlowlyDeserializingOperator());
cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-   Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration());
+   final TaskManagerActions taskManagerActions = 
mock(TaskManagerActions.class);
 
 Review comment:
   Agree.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun edited a comment on issue #6729: [FLINK-10386] [taskmanager] Remove 
legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427071700
 
 
   All "Why are we removing these assertions?" is as you say, we do not send 
those state updates via `TaskManagerActions.updateTaskExecutionState`. Maybe we 
can verify 
`TaskManagerActions.notifyFatalError`/`TaskManagerActions.failTask`/`TaskManagerActions.notifyFinalState`
 but all of them don't include detail task state.
   
   Besides, as the communication with @Clark , `TaskTest` is a test based 
on legacy mode, there is a pull request #6778 to port it to FLIP-6 code base.
   
   I think the main issue is whether we replace `notifyObservers` with 
`TaskManagerActions.updateTaskExecutionState`. My opinion is not for now to 
hold the current(FLIP-6) message flow and defer the decision. If so, we remove 
those assertion because we don't send those message and the assertion should 
not hold.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427071700
 
 
   All "Why are we removing these assertions?" is as you say, we do not send 
those state updates via `TaskManagerActions.updateTaskExecutionState`. Maybe we 
can verify 
`TaskManagerActions.notifyFatalError`/`TaskManagerActions.failTask`/`TaskManagerActions.notifyFinalState`
 but all of them don't include detail task state.
   
   Besides, as the communication with @Clark , `TaskTest` is a test based 
on legacy mode, there is a pull request #6778 to port it to FLIP-6 code base.
   
   I think the main issue is whether we replace `notifyObservers` with 
`TaskManagerActions.updateTaskExecutionState`. My opinion is not for now to 
hold the current(FLIP-6) message flow and defer the decision later. If so, we 
remove those assertion because we don't send those message and the assertion 
should not hold.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-427068734
 
 
   @tillrohrmann currently our production codes never deal with the message 
send by `notifyObservers`, because the only class deal with this message is 
`ActorGatewayTaskExecutionStateListener` who is only used in `TaskTest`. In 
other word, these message are never sent to `JobMaster`. So replace them with 
`TaskManagerActions.updateTaskExecutionState` increase messages sent to 
`JobMaster`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222721303
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -181,43 +175,26 @@
 */
@Test
public void testEarlyCanceling() throws Exception {
-   Deadline deadline = Deadline.fromNow(Duration.ofMinutes(2));
-   StreamConfig cfg = new StreamConfig(new Configuration());
+   final StreamConfig cfg = new StreamConfig(new Configuration());
cfg.setOperatorID(new OperatorID(4711L, 42L));
cfg.setStreamOperator(new SlowlyDeserializingOperator());
cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-   Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration());
+   final TaskManagerActions taskManagerActions = 
mock(TaskManagerActions.class);
+   final Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration(), taskManagerActions);
 
-   TestingExecutionStateListener testingExecutionStateListener = 
new TestingExecutionStateListener();
+   final TaskExecutionState state = new TaskExecutionState(
+   task.getJobID(), task.getExecutionId(), 
ExecutionState.RUNNING);
 
-   task.registerExecutionListener(testingExecutionStateListener);
task.startTaskThread();
 
-   Future running = 
testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING);
-
-   // wait until the task thread reached state RUNNING
-   ExecutionState executionState = 
running.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-
-   // make sure the task is really running
-   if (executionState != ExecutionState.RUNNING) {
-   fail("Task entered state " + task.getExecutionState() + 
" with error "
-   + 
ExceptionUtils.stringifyException(task.getFailureCause()));
-   }
+   verify(taskManagerActions, 
timeout(2000L)).updateTaskExecutionState(eq(state));
 
// send a cancel. because the operator takes a long time to 
deserialize, this should
// hit the task before the operator is deserialized
task.cancelExecution();
 
-   Future canceling = 
testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING);
-
-   executionState = canceling.get(deadline.timeLeft().toMillis(), 
TimeUnit.MILLISECONDS);
-
-   // the task should reach state canceled eventually
-   assertTrue(executionState == ExecutionState.CANCELING ||
-   executionState == ExecutionState.CANCELED);
-
-   task.getExecutingThread().join(deadline.timeLeft().toMillis());
 
 Review comment:
   use `task.getExecutingThread().join();` to wait for the task transit into a 
final state, we can/should see it is in `CANCELLED` state. thus not weakens 
test.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was 

[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222720265
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -846,6 +817,23 @@ public void 
testSetsUserCodeClassLoaderForTimerThreadFactory() throws Throwable
//  Test Utilities
// 

 
+   private static void waitUntilExecutionState(Task task, ExecutionState 
exceptedState, Deadline deadline) {
+   while (deadline.hasTimeLeft()) {
+   if (exceptedState == task.getExecutionState()) {
+   return;
+   }
+
+   try {
+   
Thread.sleep(Math.min(deadline.timeLeft().toMillis(), 200));
 
 Review comment:
   @tillrohrmann yes it has been addressed, please see the following commit.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222603939
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -292,16 +285,13 @@ public void testExecutionFailsInNetworkRegistration() {
 
Task task = createTask(TestInvokableCorrect.class, 
blobService, libCache, network, consumableNotifier, 
partitionProducerStateChecker, executor);
 
-   task.registerExecutionListener(listener);
-
task.run();
 
assertEquals(ExecutionState.FAILED, 
task.getExecutionState());
assertTrue(task.isCanceledOrFailed());

assertTrue(task.getFailureCause().getMessage().contains("buffers"));
 
validateUnregisterTask(task.getExecutionId());
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222605330
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -846,6 +817,23 @@ public void 
testSetsUserCodeClassLoaderForTimerThreadFactory() throws Throwable
//  Test Utilities
// 

 
+   private static void waitUntilExecutionState(Task task, ExecutionState 
exceptedState, Deadline deadline) {
+   while (deadline.hasTimeLeft()) {
+   if (exceptedState == task.getExecutionState()) {
+   return;
+   }
+
+   try {
+   
Thread.sleep(Math.min(deadline.timeLeft().toMillis(), 200));
 
 Review comment:
   Did you already addressed this comment @TisonKun? I agree with @Clark 
that we should try to never do busy loop waiting.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604024
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -374,7 +359,6 @@ public void testFailWithWrappedException() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604121
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -441,7 +422,6 @@ public void testFailExternallyDuringInvoke() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222603837
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -178,7 +177,6 @@ public void testRegularExecution() {
 
// verify listener messages
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateListenerMessage(ExecutionState.FINISHED, task, 
false);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604068
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -408,7 +391,6 @@ public void testCancelDuringInvoke() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateCancelingAndCanceledListenerMessage(task);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222603994
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -348,7 +335,6 @@ public void testExecutionFailsInInvoke() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604134
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -468,7 +447,6 @@ public void testCanceledAfterExecutionFailedInInvoke() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222605753
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -181,43 +175,26 @@
 */
@Test
public void testEarlyCanceling() throws Exception {
-   Deadline deadline = Deadline.fromNow(Duration.ofMinutes(2));
-   StreamConfig cfg = new StreamConfig(new Configuration());
+   final StreamConfig cfg = new StreamConfig(new Configuration());
cfg.setOperatorID(new OperatorID(4711L, 42L));
cfg.setStreamOperator(new SlowlyDeserializingOperator());
cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-   Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration());
+   final TaskManagerActions taskManagerActions = 
mock(TaskManagerActions.class);
 
 Review comment:
   Might be orthogonal, but I think it is better to introduce a 
`TestingTaskManagerActions` where you can define the actions similar to 
`TestingResourceManagerGateway` or `TestingTaskExecutorGateway`. They both have 
builders to configure the behaviour. Then we would not need to mock things.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604184
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -505,7 +482,6 @@ public void testExecutionFailesAfterCanceling() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateCancelingAndCanceledListenerMessage(task);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604954
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -181,43 +175,26 @@
 */
@Test
public void testEarlyCanceling() throws Exception {
-   Deadline deadline = Deadline.fromNow(Duration.ofMinutes(2));
-   StreamConfig cfg = new StreamConfig(new Configuration());
+   final StreamConfig cfg = new StreamConfig(new Configuration());
cfg.setOperatorID(new OperatorID(4711L, 42L));
cfg.setStreamOperator(new SlowlyDeserializingOperator());
cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-   Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration());
+   final TaskManagerActions taskManagerActions = 
mock(TaskManagerActions.class);
+   final Task task = createTask(SourceStreamTask.class, cfg, new 
Configuration(), taskManagerActions);
 
-   TestingExecutionStateListener testingExecutionStateListener = 
new TestingExecutionStateListener();
+   final TaskExecutionState state = new TaskExecutionState(
+   task.getJobID(), task.getExecutionId(), 
ExecutionState.RUNNING);
 
-   task.registerExecutionListener(testingExecutionStateListener);
task.startTaskThread();
 
-   Future running = 
testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING);
-
-   // wait until the task thread reached state RUNNING
-   ExecutionState executionState = 
running.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-
-   // make sure the task is really running
-   if (executionState != ExecutionState.RUNNING) {
-   fail("Task entered state " + task.getExecutionState() + 
" with error "
-   + 
ExceptionUtils.stringifyException(task.getFailureCause()));
-   }
+   verify(taskManagerActions, 
timeout(2000L)).updateTaskExecutionState(eq(state));
 
// send a cancel. because the operator takes a long time to 
deserialize, this should
// hit the task before the operator is deserialized
task.cancelExecution();
 
-   Future canceling = 
testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING);
-
-   executionState = canceling.get(deadline.timeLeft().toMillis(), 
TimeUnit.MILLISECONDS);
-
-   // the task should reach state canceled eventually
-   assertTrue(executionState == ExecutionState.CANCELING ||
-   executionState == ExecutionState.CANCELED);
-
-   task.getExecutingThread().join(deadline.timeLeft().toMillis());
 
 Review comment:
   We remove the check that the `Task` transitions into state `CANCELING` or 
`CANCELED`. I think this weakens the test.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222603971
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -322,7 +311,6 @@ public void testInvokableInstantiationFailed() {

assertTrue(task.getFailureCause().getMessage().contains("instantiate"));
 
validateUnregisterTask(task.getExecutionId());
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222603892
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -256,9 +252,6 @@ public void testLibraryCacheRegistrationFailed() {
assertNotNull(task.getFailureCause().getMessage());

assertTrue(task.getFailureCause().getMessage().contains("classloader"));
 
-   // verify listener messages
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-10-04 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


tillrohrmann commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r222604202
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -541,7 +516,6 @@ public void testExecutionFailsAfterTaskMarkedFailed() {
validateUnregisterTask(task.getExecutionId());
 
validateListenerMessage(ExecutionState.RUNNING, task, 
false);
-   validateListenerMessage(ExecutionState.FAILED, task, 
true);
 
 Review comment:
   Why are we removing these assertions?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-26 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220779905
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -429,7 +413,7 @@ public void testFailExternallyDuringInvoke() {
awaitLatch.await();
 
task.failExternally(new Exception("test"));
-   assertTrue(task.getExecutionState() == 
ExecutionState.FAILED);
 
 Review comment:
   Oh I get your point, sorry for the delay. This is a bit about style, and I 
think both of them are OK. If there is a mandatory requirement on "not change 
that", I am ok to revert it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-26 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220779551
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -846,6 +817,23 @@ public void 
testSetsUserCodeClassLoaderForTimerThreadFactory() throws Throwable
//  Test Utilities
// 

 
+   private static void waitUntilExecutionState(Task task, ExecutionState 
exceptedState, Deadline deadline) {
+   while (deadline.hasTimeLeft()) {
+   if (exceptedState == task.getExecutionState()) {
+   return;
+   }
+
+   try {
+   
Thread.sleep(Math.min(deadline.timeLeft().toMillis(), 200));
 
 Review comment:
   Thanks! I change waiting style to notify style, it would be more stable.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-25 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220238017
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -429,7 +413,7 @@ public void testFailExternallyDuringInvoke() {
awaitLatch.await();
 
task.failExternally(new Exception("test"));
-   assertTrue(task.getExecutionState() == 
ExecutionState.FAILED);
 
 Review comment:
   FYI [FLINK-10426](https://issues.apache.org/jira/browse/FLINK-10426), you 
can take over it if have idea of that anytime before I start processing it :-)


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Sub-task
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-25 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220236164
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -429,7 +413,7 @@ public void testFailExternallyDuringInvoke() {
awaitLatch.await();
 
task.failExternally(new Exception("test"));
-   assertTrue(task.getExecutionState() == 
ExecutionState.FAILED);
 
 Review comment:
   @Clark Thanks for your review! Please see also my first comment.
   `TaskTest` is a test based on legacy code, would be ported to FLIP-6 
codebase soon as a sub task of FLINK-10392. All changes under `TaskTest` is 
temporary. Once the porting job done and if this PR doesn't get merged then, I 
would do a rebase.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-25 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220236164
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -429,7 +413,7 @@ public void testFailExternallyDuringInvoke() {
awaitLatch.await();
 
task.failExternally(new Exception("test"));
-   assertTrue(task.getExecutionState() == 
ExecutionState.FAILED);
 
 Review comment:
   @Clark Thanks for your review! Please see also my first comment.
   `TaskTest` is a test based on legacy code, would be ported to FLIP-6 
codebase soon as a sub task of FLINK-10392. All changes under `TaskTest` is 
temporary. Once the porting job done and if this PR doesn't get merged, I would 
do a rebase.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-25 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


Clark commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220221550
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
 ##
 @@ -429,7 +413,7 @@ public void testFailExternallyDuringInvoke() {
awaitLatch.await();
 
task.failExternally(new Exception("test"));
-   assertTrue(task.getExecutionState() == 
ExecutionState.FAILED);
 
 Review comment:
   Is it necessary to change here? Seems like it's works the same for enum.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-25 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


Clark commented on a change in pull request #6729: [FLINK-10386] 
[taskmanager] Remove legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#discussion_r220230614
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ##
 @@ -846,6 +817,23 @@ public void 
testSetsUserCodeClassLoaderForTimerThreadFactory() throws Throwable
//  Test Utilities
// 

 
+   private static void waitUntilExecutionState(Task task, ExecutionState 
exceptedState, Deadline deadline) {
+   while (deadline.hasTimeLeft()) {
+   if (exceptedState == task.getExecutionState()) {
+   return;
+   }
+
+   try {
+   
Thread.sleep(Math.min(deadline.timeLeft().toMillis(), 200));
 
 Review comment:
   Just a little reminder, using Thread.sleep() in Junit Test thread sometimes 
may leads to some strange behavior. I am not sure if it is safe here.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-23 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-423867964
 
 
   cc @Clark  @GJL @tillrohrmann 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-21 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-423611672
 
 
   cc @StefanRRichter @zentol 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: tison
>Assignee: tison
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-21 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun edited a comment on issue #6729: [FLINK-10386] [taskmanager] Remove 
legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-423436864
 
 
   Travis fails on `TaskTest` because it is based on legacy code base. 
Temporarily hack to refine it.
   Note that although it is a hack, the coverage of test is kept -- terminate 
messages are checked by previous assertion.
   
   `TaskTest` based on FLIP-6 would be a follow-up pull request and make this 
one as clean as possible.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: 陈梓立
>Assignee: 陈梓立
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-21 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun commented on issue #6729: [FLINK-10386] [taskmanager] Remove legacy 
class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729#issuecomment-423436864
 
 
   Travis fails on `TaskTest` because it is based on legacy code base. 
Temporarily hack to refine it.
   Note that although it is a hack, the coverage of test is kept -- failure 
messages are checked by previous assertion.
   
   `TaskTest` based on FLIP-6 would be a follow-up pull request and make this 
one as clean as possible.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: 陈梓立
>Assignee: 陈梓立
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10386) Remove legacy class TaskExecutionStateListener

2018-09-20 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on FLINK-10386:


TisonKun opened a new pull request #6729: [FLINK-10386] [taskmanager] Remove 
legacy class TaskExecutionStateListener
URL: https://github.com/apache/flink/pull/6729
 
 
   ## What is the purpose of the change
   
   After a discussion 
[here](https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257)
 with @tillrohrmann . I start to analyze the usage of 
`ActorGatewayTaskExecutionStateListener` and `TaskExecutionStateListener`.
   
   In conclusion, we abort `TaskExecutionStateListener` strategy and no any 
component rely on it. Instead, we introduce `TaskManagerActions` to take the 
role for the communication of `Task` with `TaskManager`. No one except 
`TaskManager` should directly communicate with `Task`. So it can be safely 
remove legacy class `TaskExecutionStateListener`.
   
   ## Brief change log
   
   - Wholly Remove `ActorGatewayTaskExecutionStateListener.java` and 
`TaskExecutionStateListener.java`
   - Graceful remove inactive usage of these class in `Task.java`
   - Refine `TaskTest.java` and `StreamTaskTest.java`
   
   ## Verifying this change
   
   Since it is a removal, make sure existing tests pass is enough.
   
   cc @tillrohrmann @GJL @twalthr 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Remove legacy class TaskExecutionStateListener
> --
>
> Key: FLINK-10386
> URL: https://issues.apache.org/jira/browse/FLINK-10386
> Project: Flink
>  Issue Type: Improvement
>  Components: TaskManager
>Affects Versions: 1.7.0
>Reporter: 陈梓立
>Assignee: 陈梓立
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> After a discussion 
> [here|https://github.com/apache/flink/commit/0735b5b935b0c0757943e2d58047afcfb9949560#commitcomment-30584257]
>  with [~trohrm...@apache.org]. I start to analyze the usage of 
> {{ActorGatewayTaskExecutionStateListener}} and {{TaskExecutionStateListener}}.
> In conclusion, we abort {{TaskExecutionStateListener}} strategy and no any 
> component rely on it. Instead, we introduce {{TaskManagerActions}} to take 
> the role for the communication of {{Task}} with {{TaskManager}}. No one 
> except {{TaskManager}} should directly communicate with {{Task}}. So it can 
> be safely remove legacy class {{TaskExecutionStateListener}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)