[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r346110936
 
 

 ##
 File path: docs/monitoring/back_pressure.md
 ##
 @@ -66,7 +64,7 @@ You can find the *Back Pressure* tab next to the job 
overview.
 
 ### Sampling In Progress
 
-This means that the JobManager triggered a stack trace sample of the running 
tasks. With the default configuration, this takes about 5 seconds to complete.
+This means that the JobManager triggered a sample of the running tasks. With 
the default configuration, this takes about 5 seconds to complete.
 
 Review comment:
   triggered a back pressure sample


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r346110747
 
 

 ##
 File path: docs/monitoring/back_pressure.md
 ##
 @@ -34,30 +34,28 @@ If you see a **back pressure warning** (e.g. `High`) for a 
task, this means that
 Take a simple `Source -> Sink` job as an example. If you see a warning for 
`Source`, this means that `Sink` is consuming data slower than `Source` is 
producing. `Sink` is back pressuring the upstream operator `Source`.
 
 
-## Sampling Threads
+## Sampling Tasks
 
-Back pressure monitoring works by repeatedly taking stack trace samples of 
your running tasks. The JobManager triggers repeated calls to 
`Thread.getStackTrace()` for the tasks of your job.
+Back pressure monitoring works by repeatedly taking samples of your running 
tasks. The JobManager triggers repeated calls to `Task.isBackPressured()` for 
the tasks of your job.
 
 
-
+
 
-If the samples show that a task Thread is stuck in a certain internal method 
call (requesting buffers from the network stack), this indicates that there is 
back pressure for the task.
 
 Review comment:
   Add some introductions of how to judge the back pressure state instead.
   
   E.g. Internally we judge the back pressure based on the availability of 
output buffers. If there are no available buffers for at-least one output, then 
it indicates that there is back pressure for the task.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r346110805
 
 

 ##
 File path: docs/monitoring/back_pressure.md
 ##
 @@ -34,30 +34,28 @@ If you see a **back pressure warning** (e.g. `High`) for a 
task, this means that
 Take a simple `Source -> Sink` job as an example. If you see a warning for 
`Source`, this means that `Sink` is consuming data slower than `Source` is 
producing. `Sink` is back pressuring the upstream operator `Source`.
 
 
-## Sampling Threads
+## Sampling Tasks
 
-Back pressure monitoring works by repeatedly taking stack trace samples of 
your running tasks. The JobManager triggers repeated calls to 
`Thread.getStackTrace()` for the tasks of your job.
+Back pressure monitoring works by repeatedly taking samples of your running 
tasks. The JobManager triggers repeated calls to `Task.isBackPressured()` for 
the tasks of your job.
 
 
-
+
 
-If the samples show that a task Thread is stuck in a certain internal method 
call (requesting buffers from the network stack), this indicates that there is 
back pressure for the task.
-
-By default, the job manager triggers 100 stack traces every 50ms for each task 
in order to determine back pressure. The ratio you see in the web interface 
tells you how many of these stack traces were stuck in the internal method 
call, e.g. `0.01` indicates that only 1 in 100 was stuck in that method.
+By default, the job manager triggers 100 samples every 50ms for each task in 
order to determine back pressure. The ratio you see in the web interface tells 
you how many of these sample were indicating back pressure, e.g. `0.01` 
indicates that only 1 in 100 was back pressured.
 
 - **OK**: 0 <= Ratio <= 0.10
 - **LOW**: 0.10 < Ratio <= 0.5
 - **HIGH**: 0.5 < Ratio <= 1
 
-In order to not overload the task managers with stack trace samples, the web 
interface refreshes samples only after 60 seconds.
+In order to not overload the task managers with samples, the web interface 
refreshes samples only after 60 seconds.
 
 Review comment:
   with back pressure samples


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r346109104
 
 

 ##
 File path: docs/monitoring/back_pressure.md
 ##
 @@ -34,30 +34,28 @@ If you see a **back pressure warning** (e.g. `High`) for a 
task, this means that
 Take a simple `Source -> Sink` job as an example. If you see a warning for 
`Source`, this means that `Sink` is consuming data slower than `Source` is 
producing. `Sink` is back pressuring the upstream operator `Source`.
 
 
-## Sampling Threads
+## Sampling Tasks
 
-Back pressure monitoring works by repeatedly taking stack trace samples of 
your running tasks. The JobManager triggers repeated calls to 
`Thread.getStackTrace()` for the tasks of your job.
+Back pressure monitoring works by repeatedly taking samples of your running 
tasks. The JobManager triggers repeated calls to `Task.isBackPressured()` for 
the tasks of your job.
 
 Review comment:
   taking back pressure samples


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r346109051
 
 

 ##
 File path: docs/monitoring/back_pressure.md
 ##
 @@ -34,30 +34,28 @@ If you see a **back pressure warning** (e.g. `High`) for a 
task, this means that
 Take a simple `Source -> Sink` job as an example. If you see a warning for 
`Source`, this means that `Sink` is consuming data slower than `Source` is 
producing. `Sink` is back pressuring the upstream operator `Source`.
 
 
-## Sampling Threads
+## Sampling Tasks
 
 Review comment:
   Sample Back Pressure


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345722380
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345722167
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345721828
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345682575
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345675735
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345665237
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345665154
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345664138
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,187 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.junit.rules.Timeout;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
+   private static final int requestId = 0;
+   private static final double backPressureRatio = 0.1;
+   private static final ExecutionJobVertex executionJobVertex = 
BackPressureTrackerTestUtils.createExecutionJobVertex();
+   private static final ExecutionVertex[] taskVertices = 
executionJobVertex.getTaskVertices();
+   private static final BackPressureStats backPressureStats = 
createBackPressureStats(requestId, 1, backPressureRatio);
+
+   @Rule
+   public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS);
+
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345654926
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -262,9 +274,9 @@ public TestingExecutionVertex(
Runnable::run,
this,
0,
-   1L,
-   System.currentTimeMillis(),
-   Time.seconds(10),
+   initialGlobalModVersion,
+   createTimestamp,
+   timeout,
 
 Review comment:
   We could further remove `maxPriorExecutionHistoryLength` and 
`producedDataSets` arguments at least from the constructor because they can be 
initialized here when needed. No need to pass it from above 
`createExecutionVertex` to make all the relevant method arguments long.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345640075
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##
 @@ -555,138 +555,88 @@ public void testFailingScheduleOrUpdateConsumers() 
throws Exception {
}
 
// 

-   // Stack trace sample
+   // Back pressure request
// 

 
/**
-* Tests sampling of task stack traces.
+* Tests request of task back pressure.
 */
-   @Test(timeout = 1L)
-   @SuppressWarnings("unchecked")
-   public void testRequestStackTraceSample() throws Exception {
-   final ExecutionAttemptID eid = new ExecutionAttemptID();
-   final TaskDeploymentDescriptor tdd = 
createTestTaskDeploymentDescriptor("test task", eid, 
BlockingNoOpInvokable.class);
-
-   final int sampleId1 = 112223;
-   final int sampleId2 = 19230;
-   final int sampleId3 = 1337;
-   final int sampleId4 = 44;
+   @Test(timeout = 2L)
+   public void testRequestTaskBackPressure() throws Exception {
+   final NettyShuffleDescriptor shuffleDescriptor = 
createRemoteWithIdAndLocation(
+   new IntermediateResultPartitionID(), 
ResourceID.generate());
+   final TaskDeploymentDescriptor tdd = 
createSender(shuffleDescriptor, OutputBlockedInvokable.class);
+   final ExecutionAttemptID executionAttemptID = 
tdd.getExecutionAttemptId();
 
final CompletableFuture taskRunningFuture = new 
CompletableFuture<>();
final CompletableFuture taskCanceledFuture = new 
CompletableFuture<>();
 
-   try (TaskSubmissionTestEnvironment env =
-   new TaskSubmissionTestEnvironment.Builder(jobId)
-   .setSlotSize(1)
-   .addTaskManagerActionListener(eid, 
ExecutionState.RUNNING, taskRunningFuture)
-   .addTaskManagerActionListener(eid, 
ExecutionState.CANCELED, taskCanceledFuture)
-   .build()) {
-   TaskExecutorGateway tmGateway = 
env.getTaskExecutorGateway();
-   TaskSlotTable taskSlotTable = env.getTaskSlotTable();
+   final Configuration configuration = new Configuration();
+   configuration.set(WebOptions.BACKPRESSURE_NUM_SAMPLES, 10);
+   configuration.set(WebOptions.BACKPRESSURE_DELAY, 200);
+
+   try (final TaskSubmissionTestEnvironment env = new 
TaskSubmissionTestEnvironment.Builder(jobId)
+   .setSlotSize(1)
+   .setConfiguration(configuration)
+   .useRealNonMockShuffleEnvironment()
+   
.addTaskManagerActionListener(executionAttemptID, ExecutionState.RUNNING, 
taskRunningFuture)
+   
.addTaskManagerActionListener(executionAttemptID, ExecutionState.CANCELED, 
taskCanceledFuture)
+   .build()) {
+   final TaskExecutorGateway tmGateway = 
env.getTaskExecutorGateway();
+   final TaskSlotTable taskSlotTable = 
env.getTaskSlotTable();
 
taskSlotTable.allocateSlot(0, jobId, 
tdd.getAllocationId(), Time.seconds(60));
tmGateway.submitTask(tdd, env.getJobMasterId(), 
timeout).get();
taskRunningFuture.get();
 
-   //
-   // 1) Trigger sample for non-existing task
-   //
-   ExecutionAttemptID nonExistTaskEid = new 
ExecutionAttemptID();
+   // 1) trigger request for non-existing task.
+   final int requestId = 1234;
+   final ExecutionAttemptID nonExistTaskEid = new 
ExecutionAttemptID();
 
-   CompletableFuture 
failedSampleFuture =
-   
tmGateway.requestStackTraceSample(nonExistTaskEid, sampleId1, 100, 
Time.seconds(60L), 0, timeout);
+   final CompletableFuture 
failedRequestFuture =
+   
tmGateway.requestTaskBackPressure(nonExistTaskEid, requestId, timeout);
try {
-   failedSampleFuture.get();
+   failedRequestFuture.get();
} catch (Exception e) {
assertThat(e.getCause(), 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345634076
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureTrackerTestUtils.java
 ##
 @@ -0,0 +1,81 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobInformation;
+import org.apache.flink.runtime.executiongraph.TestingSlotProvider;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Utility methods for {@link BackPressureStatsTrackerImplTest} and {@link 
BackPressureRequestCoordinatorTest}.
+ */
+public class BackPressureTrackerTestUtils {
+
+   public static ExecutionJobVertex createExecutionJobVertex() throws 
Exception {
+   return new ExecutionJobVertex(
+   createExecutionGraph(),
+   new JobVertex("TestingJobVertex", new JobVertexID()),
+   4,
+   
JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE.defaultValue(),
+   Time.milliseconds(1),
+   1L,
+   System.currentTimeMillis());
+   }
+
+   public static ExecutionGraph createExecutionGraph() throws IOException {
 
 Review comment:
   We can change it to public instead because this constructor was introduced 
for tests only for simple, and it should not be limited to use only inside 
package.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-13 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r345633076
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureTrackerTestUtils.java
 ##
 @@ -0,0 +1,81 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobInformation;
+import org.apache.flink.runtime.executiongraph.TestingSlotProvider;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Utility methods for {@link BackPressureStatsTrackerImplTest} and {@link 
BackPressureRequestCoordinatorTest}.
+ */
+public class BackPressureTrackerTestUtils {
+
+   public static ExecutionJobVertex createExecutionJobVertex() throws 
Exception {
 
 Review comment:
   I suggest making `ExecutionJobVertexTest#createExecutionJobVertex` as 
public, because it was defined as static before for further outside usages if 
possible, then it makes sense to do that.  Or we could extract it as 
`ExecutionJobVertexTestUtils` class for using by other places.
   
   Also we could use the real `ExecutionGraph` as you did in this class in 
`ExecutionJobVertexTest#createExecutionJobVertex` and it would benefit also 
previous tests if not changed much.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344608730
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/OutputBlockedInvokable.java
 ##
 @@ -0,0 +1,47 @@
+/*
+ * 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.testtasks;
+
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.types.IntValue;
+
+/**
+ * A simple task that emits int value until the result partition is 
unavailable for output.
+ */
+public class OutputBlockedInvokable extends AbstractInvokable {
 
 Review comment:
   Why we implement to replace the previous `BlockingNoOpInvokable`?
   Now the output would be blocked until the buffers are exhausted, but how 
about the buffers are not used up when the back pressure is sampled?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344606214
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
 ##
 @@ -1919,14 +1919,16 @@ private TaskExecutor 
createTaskExecutor(TaskManagerServices taskManagerServices,
null,
dummyBlobCacheService,
testingFatalErrorHandler,
-   new TaskExecutorPartitionTrackerImpl());
+   new TaskExecutorPartitionTrackerImpl(),
+   
TaskManagerRunner.createBackPressureSampleService(configuration, 
rpc.getScheduledExecutor()));
}
 
private TestingTaskExecutor 
createTestingTaskExecutor(TaskManagerServices taskManagerServices) {
return createTestingTaskExecutor(taskManagerServices, 
HEARTBEAT_SERVICES);
}
 
private TestingTaskExecutor 
createTestingTaskExecutor(TaskManagerServices taskManagerServices, 
HeartbeatServices heartbeatServices) {
+
 
 Review comment:
   remove new line


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344606115
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##
 @@ -555,138 +555,88 @@ public void testFailingScheduleOrUpdateConsumers() 
throws Exception {
}
 
// 

-   // Stack trace sample
+   // Back pressure request
// 

 
/**
-* Tests sampling of task stack traces.
+* Tests request of task back pressure.
 */
-   @Test(timeout = 1L)
-   @SuppressWarnings("unchecked")
-   public void testRequestStackTraceSample() throws Exception {
-   final ExecutionAttemptID eid = new ExecutionAttemptID();
-   final TaskDeploymentDescriptor tdd = 
createTestTaskDeploymentDescriptor("test task", eid, 
BlockingNoOpInvokable.class);
-
-   final int sampleId1 = 112223;
-   final int sampleId2 = 19230;
-   final int sampleId3 = 1337;
-   final int sampleId4 = 44;
+   @Test(timeout = 2L)
+   public void testRequestTaskBackPressure() throws Exception {
+   final NettyShuffleDescriptor shuffleDescriptor = 
createRemoteWithIdAndLocation(
+   new IntermediateResultPartitionID(), 
ResourceID.generate());
+   final TaskDeploymentDescriptor tdd = 
createSender(shuffleDescriptor, OutputBlockedInvokable.class);
+   final ExecutionAttemptID executionAttemptID = 
tdd.getExecutionAttemptId();
 
final CompletableFuture taskRunningFuture = new 
CompletableFuture<>();
final CompletableFuture taskCanceledFuture = new 
CompletableFuture<>();
 
-   try (TaskSubmissionTestEnvironment env =
-   new TaskSubmissionTestEnvironment.Builder(jobId)
-   .setSlotSize(1)
-   .addTaskManagerActionListener(eid, 
ExecutionState.RUNNING, taskRunningFuture)
-   .addTaskManagerActionListener(eid, 
ExecutionState.CANCELED, taskCanceledFuture)
-   .build()) {
-   TaskExecutorGateway tmGateway = 
env.getTaskExecutorGateway();
-   TaskSlotTable taskSlotTable = env.getTaskSlotTable();
+   final Configuration configuration = new Configuration();
+   configuration.set(WebOptions.BACKPRESSURE_NUM_SAMPLES, 10);
+   configuration.set(WebOptions.BACKPRESSURE_DELAY, 200);
+
+   try (final TaskSubmissionTestEnvironment env = new 
TaskSubmissionTestEnvironment.Builder(jobId)
+   .setSlotSize(1)
+   .setConfiguration(configuration)
+   .useRealNonMockShuffleEnvironment()
+   
.addTaskManagerActionListener(executionAttemptID, ExecutionState.RUNNING, 
taskRunningFuture)
+   
.addTaskManagerActionListener(executionAttemptID, ExecutionState.CANCELED, 
taskCanceledFuture)
+   .build()) {
+   final TaskExecutorGateway tmGateway = 
env.getTaskExecutorGateway();
+   final TaskSlotTable taskSlotTable = 
env.getTaskSlotTable();
 
taskSlotTable.allocateSlot(0, jobId, 
tdd.getAllocationId(), Time.seconds(60));
tmGateway.submitTask(tdd, env.getJobMasterId(), 
timeout).get();
taskRunningFuture.get();
 
-   //
-   // 1) Trigger sample for non-existing task
-   //
-   ExecutionAttemptID nonExistTaskEid = new 
ExecutionAttemptID();
+   // 1) trigger request for non-existing task.
+   final int requestId = 1234;
+   final ExecutionAttemptID nonExistTaskEid = new 
ExecutionAttemptID();
 
-   CompletableFuture 
failedSampleFuture =
-   
tmGateway.requestStackTraceSample(nonExistTaskEid, sampleId1, 100, 
Time.seconds(60L), 0, timeout);
+   final CompletableFuture 
failedRequestFuture =
+   
tmGateway.requestTaskBackPressure(nonExistTaskEid, requestId, timeout);
try {
-   failedSampleFuture.get();
+   failedRequestFuture.get();
} catch (Exception e) {
assertThat(e.getCause(), 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344603414
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##
 @@ -555,138 +555,88 @@ public void testFailingScheduleOrUpdateConsumers() 
throws Exception {
}
 
// 

-   // Stack trace sample
+   // Back pressure request
// 

 
/**
-* Tests sampling of task stack traces.
+* Tests request of task back pressure.
 */
-   @Test(timeout = 1L)
-   @SuppressWarnings("unchecked")
-   public void testRequestStackTraceSample() throws Exception {
-   final ExecutionAttemptID eid = new ExecutionAttemptID();
-   final TaskDeploymentDescriptor tdd = 
createTestTaskDeploymentDescriptor("test task", eid, 
BlockingNoOpInvokable.class);
-
-   final int sampleId1 = 112223;
-   final int sampleId2 = 19230;
-   final int sampleId3 = 1337;
-   final int sampleId4 = 44;
+   @Test(timeout = 2L)
+   public void testRequestTaskBackPressure() throws Exception {
+   final NettyShuffleDescriptor shuffleDescriptor = 
createRemoteWithIdAndLocation(
+   new IntermediateResultPartitionID(), 
ResourceID.generate());
+   final TaskDeploymentDescriptor tdd = 
createSender(shuffleDescriptor, OutputBlockedInvokable.class);
+   final ExecutionAttemptID executionAttemptID = 
tdd.getExecutionAttemptId();
 
final CompletableFuture taskRunningFuture = new 
CompletableFuture<>();
final CompletableFuture taskCanceledFuture = new 
CompletableFuture<>();
 
-   try (TaskSubmissionTestEnvironment env =
-   new TaskSubmissionTestEnvironment.Builder(jobId)
-   .setSlotSize(1)
-   .addTaskManagerActionListener(eid, 
ExecutionState.RUNNING, taskRunningFuture)
-   .addTaskManagerActionListener(eid, 
ExecutionState.CANCELED, taskCanceledFuture)
-   .build()) {
-   TaskExecutorGateway tmGateway = 
env.getTaskExecutorGateway();
-   TaskSlotTable taskSlotTable = env.getTaskSlotTable();
+   final Configuration configuration = new Configuration();
+   configuration.set(WebOptions.BACKPRESSURE_NUM_SAMPLES, 10);
+   configuration.set(WebOptions.BACKPRESSURE_DELAY, 200);
 
 Review comment:
   The default delay 50 does not make sense?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344603009
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##
 @@ -555,138 +555,88 @@ public void testFailingScheduleOrUpdateConsumers() 
throws Exception {
}
 
// 

-   // Stack trace sample
+   // Back pressure request
// 

 
/**
-* Tests sampling of task stack traces.
+* Tests request of task back pressure.
 */
-   @Test(timeout = 1L)
-   @SuppressWarnings("unchecked")
-   public void testRequestStackTraceSample() throws Exception {
-   final ExecutionAttemptID eid = new ExecutionAttemptID();
-   final TaskDeploymentDescriptor tdd = 
createTestTaskDeploymentDescriptor("test task", eid, 
BlockingNoOpInvokable.class);
-
-   final int sampleId1 = 112223;
-   final int sampleId2 = 19230;
-   final int sampleId3 = 1337;
-   final int sampleId4 = 44;
+   @Test(timeout = 2L)
+   public void testRequestTaskBackPressure() throws Exception {
+   final NettyShuffleDescriptor shuffleDescriptor = 
createRemoteWithIdAndLocation(
 
 Review comment:
   use `newBuilder().buildLocal()` instead for simple


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344599761
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java
 ##
 @@ -555,138 +555,88 @@ public void testFailingScheduleOrUpdateConsumers() 
throws Exception {
}
 
// 

-   // Stack trace sample
+   // Back pressure request
// 

 
/**
-* Tests sampling of task stack traces.
+* Tests request of task back pressure.
 */
-   @Test(timeout = 1L)
-   @SuppressWarnings("unchecked")
-   public void testRequestStackTraceSample() throws Exception {
-   final ExecutionAttemptID eid = new ExecutionAttemptID();
-   final TaskDeploymentDescriptor tdd = 
createTestTaskDeploymentDescriptor("test task", eid, 
BlockingNoOpInvokable.class);
-
-   final int sampleId1 = 112223;
-   final int sampleId2 = 19230;
-   final int sampleId3 = 1337;
-   final int sampleId4 = 44;
+   @Test(timeout = 2L)
 
 Review comment:
   why we need more timeout here.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-11 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344593616
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleServiceTest.java
 ##
 @@ -0,0 +1,127 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link BackPressureSampleService}.
+ */
+public class BackPressureSampleServiceTest extends TestLogger {
+
+   private ScheduledExecutorService scheduledExecutorService;
+
+   private BackPressureSampleService backPressureSampleService;
+
+   @Before
+   public void setUp() throws Exception {
+   scheduledExecutorService = 
Executors.newSingleThreadScheduledExecutor();
+   final ScheduledExecutor scheduledExecutor = new 
ScheduledExecutorServiceAdapter(scheduledExecutorService);
+
+   backPressureSampleService = new BackPressureSampleService( 10, 
Time.milliseconds(10), scheduledExecutor);
+   }
+
+   @After
+   public void tearDown() throws Exception {
+   if (scheduledExecutorService != null) {
+   scheduledExecutorService.shutdown();
+   }
+   }
+
+   @Test(timeout = 1L)
+   public void testSampleTaskBackPressure() throws Exception {
+   final double backPressureRatio = backPressureSampleService.
+   sampleTaskBackPressure(new TestTask()).get();
+
+   assertEquals(0.5, backPressureRatio, 0.0);
+   }
+
+   @Test(timeout = 1L)
+   public void testTaskStopsWithPartialSampling() throws Exception {
+   final double backPressureRatio = backPressureSampleService.
+   sampleTaskBackPressure(new 
NotRunningAfterFirstSamplingTask()).get();
+
+   assertEquals(1.0, backPressureRatio, 0.0);
+   }
+
+   @Test(expected = IllegalStateException.class)
+   public void testShouldThrowExceptionIfTaskIsNotRunningBeforeSampling() {
+   backPressureSampleService.sampleTaskBackPressure(new 
NeverRunningTask());
+
+   fail("Exception expected.");
+   }
+
+   /**
+* Task that is always running.
+*/
+   private static class TestTask implements BackPressureSampleableTask {
+
+   private final ExecutionAttemptID executionAttemptID = new 
ExecutionAttemptID();
 
 Review comment:
   remove useless code


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344592080
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleServiceTest.java
 ##
 @@ -0,0 +1,127 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link BackPressureSampleService}.
+ */
+public class BackPressureSampleServiceTest extends TestLogger {
+
+   private ScheduledExecutorService scheduledExecutorService;
+
+   private BackPressureSampleService backPressureSampleService;
+
+   @Before
 
 Review comment:
   use @BeforeClass, we can have a class level service instead of creating 
instance for every test.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344591459
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleServiceTest.java
 ##
 @@ -0,0 +1,127 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link BackPressureSampleService}.
+ */
+public class BackPressureSampleServiceTest extends TestLogger {
+
+   private ScheduledExecutorService scheduledExecutorService;
+
+   private BackPressureSampleService backPressureSampleService;
+
+   @Before
+   public void setUp() throws Exception {
+   scheduledExecutorService = 
Executors.newSingleThreadScheduledExecutor();
+   final ScheduledExecutor scheduledExecutor = new 
ScheduledExecutorServiceAdapter(scheduledExecutorService);
+
+   backPressureSampleService = new BackPressureSampleService( 10, 
Time.milliseconds(10), scheduledExecutor);
 
 Review comment:
   remove whitespace before `10`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344590458
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureTrackerTestUtils.java
 ##
 @@ -0,0 +1,81 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobInformation;
+import org.apache.flink.runtime.executiongraph.TestingSlotProvider;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Utility methods for {@link BackPressureStatsTrackerImplTest} and {@link 
BackPressureRequestCoordinatorTest}.
+ */
+public class BackPressureTrackerTestUtils {
+
+   public static ExecutionJobVertex createExecutionJobVertex() throws 
Exception {
+   return new ExecutionJobVertex(
+   createExecutionGraph(),
+   new JobVertex("TestingJobVertex", new JobVertexID()),
+   4,
+   
JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE.defaultValue(),
+   Time.milliseconds(1),
+   1L,
+   System.currentTimeMillis());
+   }
+
+   public static ExecutionGraph createExecutionGraph() throws IOException {
 
 Review comment:
   We could create a more simple `ExecutionGraph` like below: 
   
   ```
   ExecutionGraph eg = new ExecutionGraph(
TestingUtils.defaultExecutor(),
TestingUtils.defaultExecutor(),
new JobID(), 
"test", 
new Configuration(),
new SerializedValue<>(new ExecutionConfig()),
AkkaUtils.getDefaultTimeout(),
new NoRestartStrategy(),
new TestingSlotProvider(ignored -> new 
CompletableFuture<>()));
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344589965
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureTrackerTestUtils.java
 ##
 @@ -0,0 +1,81 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobInformation;
+import org.apache.flink.runtime.executiongraph.TestingSlotProvider;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Utility methods for {@link BackPressureStatsTrackerImplTest} and {@link 
BackPressureRequestCoordinatorTest}.
+ */
+public class BackPressureTrackerTestUtils {
+
+   public static ExecutionJobVertex createExecutionJobVertex() throws 
Exception {
 
 Review comment:
   Actually we could use a simpler constructor for `ExecutionJobVertex`, even 
it is better to use `ExecutionJobVertexTest#createExecutionJobVertex` directly, 
because `BackPressureTrackerTestUtils` actually did not create back pressure 
specific components. 
   
   Although the execution graph is mocked inside 
`ExecutionJobVertexTest#createExecutionJobVertex`, it is not touched by this 
PR, so we can keep it that or make it as a real graph if you like (not 
mandatory).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344589965
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureTrackerTestUtils.java
 ##
 @@ -0,0 +1,81 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobInformation;
+import org.apache.flink.runtime.executiongraph.TestingSlotProvider;
+import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Utility methods for {@link BackPressureStatsTrackerImplTest} and {@link 
BackPressureRequestCoordinatorTest}.
+ */
+public class BackPressureTrackerTestUtils {
+
+   public static ExecutionJobVertex createExecutionJobVertex() throws 
Exception {
 
 Review comment:
   Actually we could use a simpler constructor for `ExecutionJobVertex`, even 
it is better to use `ExecutionJobVertexTest#createExecutionJobVertex` directly, 
because `BackPressureTrackerTestUtils` actually did not create back pressure 
specific components. 
   
   Although the execution graph was mocked inside 
`ExecutionJobVertexTest#createExecutionJobVertex`, it is not touched by this 
PR, so we can keep it that or make it as a real graph if you like (not 
mandatory).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344587521
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344586695
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344586291
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344584416
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344583750
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344583125
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344582415
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344581857
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344581031
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344580223
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -18,166 +18,218 @@
 
 package org.apache.flink.runtime.rest.handler.legacy.backpressure;
 
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
-
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
-
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
-
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
-
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
-
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
-
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
-
-   BackPressureStatsTrackerImpl tracker = new 
BackPressureStatsTrackerImpl(
-   sampleCoordinator, , numSamples, 
Integer.MAX_VALUE, delayBetweenSamples);
-
-   // getOperatorBackPressureStats triggers stack trace sampling
-   
Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
-   Matchers.eq(delayBetweenSamples),
-   
Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH));
-
-   // Request back pressure stats again. This should not trigger 
another sample request
-   
Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent());
-
-   Mockito.verify(sampleCoordinator, 
Mockito.times(1)).triggerStackTraceSample(
-   Matchers.eq(taskVertices),
-   Matchers.eq(numSamples),
- 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344551534
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344550447
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344550465
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344550166
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
 
 Review comment:
   If it is possible to make timeout as class level, no need to tag it for 
every test. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344548191
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344548105
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344546978
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344546570
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
 
 Review comment:
   Migrate this assert into @Aftter 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344546173
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344546013
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344545951
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344545905
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testSuccessfulBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coordinator.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   Map backPressureRatios = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
backPressureRatios.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coordinator.getNumberOfPendingRequests());
+   }
+
+   /** Tests back pressure request of non-running tasks 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344545581
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
 
 Review comment:
   Refactor the format of javadoc for all the methods. Should be
   /**
   *
   */
   Also this description is not very accurate, what is "simple request" mean.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344545581
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,334 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.IntermediateResult;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coordinator;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void initCoordinator() throws Exception {
+   coordinator = new 
BackPressureRequestCoordinator(executorService, requestTimeout);
+   }
+
+   @After
+   public void shutdownCoordinator() throws Exception {
+   if (coordinator != null) {
+   coordinator.shutDown();
+   }
+   }
+
+   /** Tests simple request of task back pressure stats. */
 
 Review comment:
   Refactor the format of javadoc for all the methods. Should be
   /**
   *
   */


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344541629
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
 ##
 @@ -387,7 +389,17 @@ public static TaskExecutor startTaskManager(
metricQueryServiceAddress,
blobCacheService,
fatalErrorHandler,
-   new TaskExecutorPartitionTrackerImpl());
+   new TaskExecutorPartitionTrackerImpl(),
+   createBackPressureSampleService(configuration, 
rpcService.getScheduledExecutor()));
+   }
+
+   static BackPressureSampleService createBackPressureSampleService(
 
 Review comment:
   @VisibleForTesting?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344541629
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
 ##
 @@ -387,7 +389,17 @@ public static TaskExecutor startTaskManager(
metricQueryServiceAddress,
blobCacheService,
fatalErrorHandler,
-   new TaskExecutorPartitionTrackerImpl());
+   new TaskExecutorPartitionTrackerImpl(),
+   createBackPressureSampleService(configuration, 
rpcService.getScheduledExecutor()));
+   }
+
+   static BackPressureSampleService createBackPressureSampleService(
 
 Review comment:
   @VisibleForTesting?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344541099
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -53,64 +53,58 @@
 
this.numSamples = numSamples;
this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
-   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   this.scheduledExecutor = checkNotNull(scheduledExecutor);
}
 
/**
-* Returns a future that completes with the back pressure ratio of a 
task.
+* Schedules to sample the task back pressure and returns a future that 
completes
+* with the back pressure ratio.
 *
 * @param task The task to be sampled.
-* @return A future of the task back pressure ratio.
+* @return A future containing the task back pressure ratio.
 */
public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   if (!task.isRunning()) {
+   throw new IllegalStateException("Cannot sample task. 
Because it is not running.");
 
 Review comment:
   It is better to give some debug info here. E.g we could give 
`task.toString()` in the message, then in the specific task implementation, it 
can provide `executionId` info as before.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344539902
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -82,19 +87,21 @@
/**
 * Creates a back pressure statistics tracker.
 *
-* @param cleanUpInterval Clean up interval for completed stats.
-* @param backPressureStatsRefreshInterval
+* @param coordinator Coordinator for back pressure stats request.
+* @param cleanUpInterval Clean up interval for completed stats.
+* @param refreshInterval Time interval after which the available back 
pressure
+*stats are deprecated and need to be refreshed.
 */
public BackPressureStatsTrackerImpl(
BackPressureRequestCoordinator coordinator,
int cleanUpInterval,
-   int backPressureStatsRefreshInterval) {
+   int refreshInterval) {
 
 Review comment:
   Should also check argument for `cleanUpInterval`. Or add `@Nonnegative` for 
`cleanUpInterval` and `refreshInterval` to replace check?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-10 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344538921
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java
 ##
 @@ -38,26 +40,21 @@
/** Time stamp, when the request was triggered. */
private final long startTime;
 
-   /** Time stamp, when all back pressure stats were collected at the 
JobManager. */
+   /** Time stamp, when all back pressure stats were collected at the 
BackPressureRequestCoordinator. */
private final long endTime;
 
-   /** Map of back pressure ratio by execution ID. */
+   /** Map of back pressure ratios by execution ID. */
private final Map backPressureRatios;
 
public BackPressureStats(
-   int requestId,
-   long startTime,
-   long endTime,
+   @Nonnegative int requestId,
+   @Nonnegative long startTime,
+   @Nonnegative long endTime,
Map backPressureRatios) {
-
-   checkArgument(requestId >= 0, "Negative request ID.");
-   checkArgument(startTime >= 0, "Negative start time.");
-   checkArgument(endTime >= startTime, "End time before start 
time.");
 
 Review comment:
   Keep this check?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344034206
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinator.java
 ##
 @@ -0,0 +1,365 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A coordinator for triggering and collecting back pressure stats
+ * of running tasks.
+ */
+public class BackPressureRequestCoordinator {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(BackPressureRequestCoordinator.class);
+
+   private static final int NUM_GHOST_REQUEST_IDS = 10;
+
+   private final Object lock = new Object();
+
+   /** Executor used to run the futures. */
+   private final Executor executor;
+
+   /** Request time out of a triggered back pressure request. */
+   private final Time requestTimeout;
+
+   /** In progress back pressure requests. */
+   @GuardedBy("lock")
+   private final Map pendingRequests 
= new HashMap<>();
+
+   /** A list of recent request IDs to identify late messages vs. invalid 
ones. */
+   private final ArrayDeque recentPendingRequests = new 
ArrayDeque<>(NUM_GHOST_REQUEST_IDS);
+
+   /** Request ID counter. */
+   @GuardedBy("lock")
+   private int requestIdCounter;
+
+   /** Flag indicating whether the coordinator is still running. */
+   @GuardedBy("lock")
+   private boolean isShutDown;
+
+   /**
+* Creates a new coordinator for the cluster.
+*
+* @param executor Used to execute the futures.
+* @param requestTimeout Request time out of a triggered back pressure 
request.
+*/
+   public BackPressureRequestCoordinator(
+   Executor executor,
+   long requestTimeout) {
+
+   checkArgument(requestTimeout >= 0L, "Illegal request timeout: " 
+ requestTimeout);
+
+   this.executor = checkNotNull(executor);
+   this.requestTimeout = Time.milliseconds(requestTimeout);
+   }
+
+   /**
+* Triggers a task back pressure stats request to all tasks.
+*
+* @param tasks Tasks to request.
+* @return A future of the completed task back pressure stats.
+*/
+   CompletableFuture 
triggerBackPressureRequest(ExecutionVertex[] tasks) {
+
+   checkNotNull(tasks, "Tasks to request must not be null.");
+   checkArgument(tasks.length >= 1, "No tasks to request.");
+
+   // Execution IDs of running tasks
+   ExecutionAttemptID[] triggerIds = new 
ExecutionAttemptID[tasks.length];
+   Execution[] executions = new Execution[tasks.length];
+
+   // Check that all tasks are RUNNING before triggering anything. 
The
+   // triggering can still fail.
+   for (int i = 0; i < triggerIds.length; i++) {
+ 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344029460
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -29,155 +28,242 @@
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.mockito.ArgumentMatchers;
+import org.mockito.stubbing.OngoingStubbing;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
+   public void testGetOperatorBackPressureStats() throws Exception {
+   final ExecutionJobVertex jobVertex = 
createExecutionJobVertex(JobStatus.RUNNING);
 
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
+   final int requestId = 0;
+   final long startTime = System.currentTimeMillis();
+   final long endTime = startTime + 1;
+   final double backPressureRatio = 0.1;
 
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
+   final BackPressureStats backPressureStats = 
createBackPressureStats(
+   jobVertex.getTaskVertices(), requestId, startTime, 
endTime, backPressureRatio);
+   final BackPressureStatsTracker tracker = 
createBackPressureTracker(60, 1, backPressureStats);
 
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
+   // trigger back pressure stats request
+   tracker.getOperatorBackPressureStats(jobVertex);
 
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
+   Optional optionalStats = 
tracker.getOperatorBackPressureStats(jobVertex);
+   assertTrue(optionalStats.isPresent());
+   OperatorBackPressureStats stats = optionalStats.get();
 
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
+   checkOperatorBackPressureStats(jobVertex.getTaskVertices(), 
requestId, endTime, backPressureRatio, stats);
+   }
 
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
+   @Test
+   public void testOperatorBackPressureStatsUpdate() throws Exception {
+   final ExecutionJobVertex jobVertex = 
createExecutionJobVertex(JobStatus.RUNNING);
+   final int backPressureStatsRefreshInterval = 2000;
+   final long waitingTime = backPressureStatsRefreshInterval + 500;
+
+   final int requestId1 = 0;
+   

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344029404
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java
 ##
 @@ -29,155 +28,242 @@
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.TestLogger;
 
-import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mockito;
+import org.mockito.ArgumentMatchers;
+import org.mockito.stubbing.OngoingStubbing;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
- * Tests for the BackPressureStatsTrackerImpl.
+ * Tests for the {@link BackPressureStatsTrackerImpl}.
  */
 public class BackPressureStatsTrackerImplTest extends TestLogger {
 
-   /** Tests simple statistics with fake stack traces. */
@Test
-   @SuppressWarnings("unchecked")
-   public void testTriggerStackTraceSample() throws Exception {
-   CompletableFuture sampleFuture = new 
CompletableFuture<>();
-
-   StackTraceSampleCoordinator sampleCoordinator = 
Mockito.mock(StackTraceSampleCoordinator.class);
-   Mockito.when(sampleCoordinator.triggerStackTraceSample(
-   Matchers.any(ExecutionVertex[].class),
-   Matchers.anyInt(),
-   Matchers.any(Time.class),
-   Matchers.anyInt())).thenReturn(sampleFuture);
+   public void testGetOperatorBackPressureStats() throws Exception {
+   final ExecutionJobVertex jobVertex = 
createExecutionJobVertex(JobStatus.RUNNING);
 
-   ExecutionGraph graph = Mockito.mock(ExecutionGraph.class);
-   Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING);
+   final int requestId = 0;
+   final long startTime = System.currentTimeMillis();
+   final long endTime = startTime + 1;
+   final double backPressureRatio = 0.1;
 
-   // Same Thread execution context
-   Mockito.when(graph.getFutureExecutor()).thenReturn(new 
Executor() {
+   final BackPressureStats backPressureStats = 
createBackPressureStats(
+   jobVertex.getTaskVertices(), requestId, startTime, 
endTime, backPressureRatio);
+   final BackPressureStatsTracker tracker = 
createBackPressureTracker(60, 1, backPressureStats);
 
-   @Override
-   public void execute(Runnable runnable) {
-   runnable.run();
-   }
-   });
+   // trigger back pressure stats request
+   tracker.getOperatorBackPressureStats(jobVertex);
 
-   ExecutionVertex[] taskVertices = new ExecutionVertex[4];
+   Optional optionalStats = 
tracker.getOperatorBackPressureStats(jobVertex);
+   assertTrue(optionalStats.isPresent());
+   OperatorBackPressureStats stats = optionalStats.get();
 
-   ExecutionJobVertex jobVertex = 
Mockito.mock(ExecutionJobVertex.class);
-   Mockito.when(jobVertex.getJobId()).thenReturn(new JobID());
-   Mockito.when(jobVertex.getJobVertexId()).thenReturn(new 
JobVertexID());
-   Mockito.when(jobVertex.getGraph()).thenReturn(graph);
-   
Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices);
-
-   taskVertices[0] = mockExecutionVertex(jobVertex, 0);
-   taskVertices[1] = mockExecutionVertex(jobVertex, 1);
-   taskVertices[2] = mockExecutionVertex(jobVertex, 2);
-   taskVertices[3] = mockExecutionVertex(jobVertex, 3);
+   checkOperatorBackPressureStats(jobVertex.getTaskVertices(), 
requestId, endTime, backPressureRatio, stats);
+   }
 
-   int numSamples = 100;
-   Time delayBetweenSamples = Time.milliseconds(100L);
+   @Test
+   public void testOperatorBackPressureStatsUpdate() throws Exception {
+   final ExecutionJobVertex jobVertex = 
createExecutionJobVertex(JobStatus.RUNNING);
+   final int backPressureStatsRefreshInterval = 2000;
+   final long waitingTime = backPressureStatsRefreshInterval + 500;
+
+   final int requestId1 = 0;
+   

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344029260
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
+
+   Map tracesByTask = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
tracesByTask.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coord.getNumberOfPendingRequests());
+
+   // verify no error on late collect
+   coord.collectTaskBackPressureStat(0, 
vertices[0].getCurrentExecutionAttempt().getAttemptId(), 0.0);
+   }
+
+   /** Tests simple request and collection of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testCollectTaskBackPressureStat() throws Exception {
+ 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344028612
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
+
+   Map tracesByTask = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
tracesByTask.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coord.getNumberOfPendingRequests());
+
+   // verify no error on late collect
+   coord.collectTaskBackPressureStat(0, 
vertices[0].getCurrentExecutionAttempt().getAttemptId(), 0.0);
+   }
+
+   /** Tests simple request and collection of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testCollectTaskBackPressureStat() throws Exception {
 
 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344028612
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
+
+   Map tracesByTask = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
tracesByTask.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coord.getNumberOfPendingRequests());
+
+   // verify no error on late collect
+   coord.collectTaskBackPressureStat(0, 
vertices[0].getCurrentExecutionAttempt().getAttemptId(), 0.0);
+   }
+
+   /** Tests simple request and collection of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testCollectTaskBackPressureStat() throws Exception {
 
 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344027326
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
+
+   Map tracesByTask = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
tracesByTask.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coord.getNumberOfPendingRequests());
+
+   // verify no error on late collect
+   coord.collectTaskBackPressureStat(0, 
vertices[0].getCurrentExecutionAttempt().getAttemptId(), 0.0);
 
 Review comment:
   We should re-arrange the relevant tests in this class. In the 
`createExecutionVertices` we define the different task state and the complete 
type, then we can only verify the 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-08 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344027326
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
+
+   Map tracesByTask = 
backPressureStats.getBackPressureRatios();
+   for (ExecutionVertex executionVertex : vertices) {
+   ExecutionAttemptID executionId = 
executionVertex.getCurrentExecutionAttempt().getAttemptId();
+   assertEquals(backPressureRatio, 
tracesByTask.get(executionId), 0.0);
+   }
+
+   // verify no more pending request
+   assertEquals(0, coord.getNumberOfPendingRequests());
+
+   // verify no error on late collect
+   coord.collectTaskBackPressureStat(0, 
vertices[0].getCurrentExecutionAttempt().getAttemptId(), 0.0);
 
 Review comment:
   We should re-arrange the relevant tests in this class. In the 
`createExecutionVertices` we define the different task state and the complete 
type, then we can only verify the 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344022126
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
 
 Review comment:
   also add `@After` for `coord.shutdown()`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344021494
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
+
+   Map tracesByTask = 
backPressureStats.getBackPressureRatios();
 
 Review comment:
   tracesByTask -> backPressureRatios


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344021592
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
+
+   @BeforeClass
+   public static void setUp() throws Exception {
+   executorService = new ScheduledThreadPoolExecutor(1);
+   }
+
+   @AfterClass
+   public static void tearDown() throws Exception {
+   if (executorService != null) {
+   executorService.shutdown();
+   }
+   }
+
+   @Before
+   public void init() throws Exception {
+   coord = new BackPressureRequestCoordinator(executorService, 
requestTimeout);
+   }
+
+   /** Tests simple request of task back pressure stats. */
+   @Test(timeout = 1L)
+   public void testTriggerBackPressureRequest() throws Exception {
+   ExecutionVertex[] vertices = 
createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY);
+
+   CompletableFuture requestFuture = 
coord.triggerBackPressureRequest(vertices);
+   BackPressureStats backPressureStats = requestFuture.get();
+
+   // verify the request result
+   assertEquals(0, backPressureStats.getRequestId());
+   assertTrue(backPressureStats.getEndTime() >= 
backPressureStats.getStartTime());
 
 Review comment:
   Actually this check is already covered while constructing `BackPressureStats`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344021232
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java
 ##
 @@ -0,0 +1,87 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Back pressure stats for one or more tasks.
+ *
+ * The stats are collected by request triggered in {@link 
BackPressureRequestCoordinator}.
+ */
+public class BackPressureStats {
+
+   /** ID of the request (unique per job). */
+   private final int requestId;
+
+   /** Time stamp, when the request was triggered. */
+   private final long startTime;
+
+   /** Time stamp, when all back pressure stats were collected at the 
JobManager. */
+   private final long endTime;
+
+   /** Map of back pressure ratio by execution ID. */
+   private final Map backPressureRatios;
+
+   public BackPressureStats(
+   int requestId,
+   long startTime,
+   long endTime,
+   Map backPressureRatios) {
+
+   checkArgument(requestId >= 0, "Negative request ID.");
+   checkArgument(startTime >= 0, "Negative start time.");
+   checkArgument(endTime >= startTime, "End time before start 
time.");
 
 Review comment:
   I wonder it exists the case that `endTime = startTime`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344019865
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link BackPressureRequestCoordinator}.
+ */
+public class BackPressureRequestCoordinatorTest extends TestLogger {
+
+   private static final long requestTimeout = 1;
+   private static final double backPressureRatio = 0.5;
+
+   private static ScheduledExecutorService executorService;
+   private BackPressureRequestCoordinator coord;
 
 Review comment:
   coord -> coordinator


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344018872
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleableTask.java
 ##
 @@ -22,13 +22,13 @@
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 
 /**
- * Task interface used by {@link StackTraceSampleService} for back pressure 
tracking.
+ * Task interface used by {@link BackPressureSampleService} for back pressure 
tracking.
  */
-interface StackTraceSampleableTask {
+public interface BackPressureSampleableTask {
 
boolean isRunning();
 
-   StackTraceElement[] getStackTrace();
+   boolean isBackPressured();
 
ExecutionAttemptID getExecutionId();
 
 Review comment:
   This interface method is also not very necessary, and the only usage atm is 
for throwing exception. I guess we can remove it as well.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344015484
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##
 @@ -495,6 +496,11 @@ public boolean isCanceledOrFailed() {
executionState == ExecutionState.FAILED;
}
 
+   @Override
 
 Review comment:
   This can be removed as previously commented. This logic can be covered by 
above `isBackPressured`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344015308
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##
 @@ -461,16 +462,16 @@ AbstractInvokable getInvokable() {
return invokable;
}
 
-   public StackTraceElement[] getStackTraceOfExecutingThread() {
-   final AbstractInvokable invokable = this.invokable;
-
-   if (invokable == null) {
-   return new StackTraceElement[0];
+   @Override
+   public boolean isBackPressured() {
+   if (invokable == null || 
consumableNotifyingPartitionWriters.length == 0) {
+   return true;
}
-
-   return invokable.getExecutingThread()
-   .orElse(executingThread)
-   .getStackTrace();
+   final CompletableFuture[] outputFutures = new 
CompletableFuture[consumableNotifyingPartitionWriters.length];
+   for (int i = 0; i < outputFutures.length; ++i) {
+   outputFutures[i] = 
consumableNotifyingPartitionWriters[i].isAvailable();
+   }
+   return CompletableFuture.allOf(outputFutures).isDone();
 
 Review comment:
   The return value is reverse. If all the outputs are available, then it 
should return false. The return value `true` means back pressured.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344014732
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##
 @@ -461,16 +462,16 @@ AbstractInvokable getInvokable() {
return invokable;
}
 
-   public StackTraceElement[] getStackTraceOfExecutingThread() {
-   final AbstractInvokable invokable = this.invokable;
-
-   if (invokable == null) {
-   return new StackTraceElement[0];
+   @Override
+   public boolean isBackPressured() {
+   if (invokable == null || 
consumableNotifyingPartitionWriters.length == 0) {
+   return true;
 
 Review comment:
   should return false?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344013614
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
 ##
 @@ -273,7 +276,11 @@ public TaskExecutor(
this.resourceManagerConnection = null;
this.currentRegistrationTimeoutId = null;
 
-   this.stackTraceSampleService = new 
StackTraceSampleService(rpcService.getScheduledExecutor());
+   final Configuration config = 
taskManagerConfiguration.getConfiguration();
+   this.taskBackPressureSampleService = new 
BackPressureSampleService(
 
 Review comment:
   I suggest constructing the `BackPressureSampleService` out of the 
constructor to make it clean, also we could avoid 
`taskManagerConfiguration.getConfiguration()` from outside call. It is elegant 
to prepare everything outside and then pass them directly into the constructor. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344011851
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleableTask.java
 ##
 @@ -22,13 +22,13 @@
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 
 /**
- * Task interface used by {@link StackTraceSampleService} for back pressure 
tracking.
+ * Task interface used by {@link BackPressureSampleService} for back pressure 
tracking.
  */
-interface StackTraceSampleableTask {
+public interface BackPressureSampleableTask {
 
boolean isRunning();
 
 Review comment:
   It seems a bit strange to have this method in the interface. This state 
should be covered in the specific implementation of `boolean 
isBackPressured()`. That means if the task is not running, it indicates non 
back pressure. So we do not need to explicitly judge the running state outside.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344011455
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
+   new ArrayList<>(numSamples),
+   new CompletableFuture<>());
+   }
+
+   private CompletableFuture sampleTaskBackPressure(
+   final BackPressureSampleableTask task,
+   final int remainingNumSamples,
+   final Time delayBetweenSamples,
+   final List taskBackPressureSamples,
+   final CompletableFuture resultFuture) {
+
+   if (task.isRunning()) {
 
 Review comment:
   If task state is not running, `task. isBackPressured()` should return false 
in specific implementation. Then we could remove this condition and interface 
method `BackPressureSampleableTask#isRunning`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344010610
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
+   new ArrayList<>(numSamples),
+   new CompletableFuture<>());
+   }
+
+   private CompletableFuture sampleTaskBackPressure(
+   final BackPressureSampleableTask task,
+   final int remainingNumSamples,
+   final Time delayBetweenSamples,
+   final List taskBackPressureSamples,
+   final CompletableFuture resultFuture) {
+
+   if (task.isRunning()) {
+   taskBackPressureSamples.add(task.isBackPressured());
+   } else if (!taskBackPressureSamples.isEmpty()) {
+   
resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples));
+   return resultFuture;
+   } else {
+   throw new IllegalStateException(String.format("Cannot 
sample task %s. " +
+   "Because the task is not running.", 
task.getExecutionId()));
+   }
+
+   if (remainingNumSamples > 1) {
+   scheduledExecutor.schedule(
+   () -> sampleTaskBackPressure(
+   task,
+   remainingNumSamples - 1,
+   delayBetweenSamples,
+   taskBackPressureSamples,
+   resultFuture),
+   delayBetweenSamples.getSize(),
+   delayBetweenSamples.getUnit());
+   } else {
+   
resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples));
+

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344010528
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
+   new ArrayList<>(numSamples),
+   new CompletableFuture<>());
+   }
+
+   private CompletableFuture sampleTaskBackPressure(
+   final BackPressureSampleableTask task,
+   final int remainingNumSamples,
+   final Time delayBetweenSamples,
+   final List taskBackPressureSamples,
+   final CompletableFuture resultFuture) {
+
+   if (task.isRunning()) {
+   taskBackPressureSamples.add(task.isBackPressured());
+   } else if (!taskBackPressureSamples.isEmpty()) {
+   
resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples));
+   return resultFuture;
+   } else {
+   throw new IllegalStateException(String.format("Cannot 
sample task %s. " +
+   "Because the task is not running.", 
task.getExecutionId()));
+   }
+
+   if (remainingNumSamples > 1) {
+   scheduledExecutor.schedule(
+   () -> sampleTaskBackPressure(
+   task,
+   remainingNumSamples - 1,
+   delayBetweenSamples,
+   taskBackPressureSamples,
+   resultFuture),
+   delayBetweenSamples.getSize(),
+   delayBetweenSamples.getUnit());
+   } else {
+   
resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples));
+

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344009866
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
+   new ArrayList<>(numSamples),
+   new CompletableFuture<>());
+   }
+
+   private CompletableFuture sampleTaskBackPressure(
+   final BackPressureSampleableTask task,
+   final int remainingNumSamples,
+   final Time delayBetweenSamples,
+   final List taskBackPressureSamples,
+   final CompletableFuture resultFuture) {
+
+   if (task.isRunning()) {
+   taskBackPressureSamples.add(task.isBackPressured());
+   } else if (!taskBackPressureSamples.isEmpty()) {
+   
resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples));
 
 Review comment:
   There seems some logic bug here. If the `remainingNumSamples` has not 
reached 0, then we should not complete the future, otherwise the results would 
not be accurate?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344009531
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
+   new ArrayList<>(numSamples),
+   new CompletableFuture<>());
+   }
+
+   private CompletableFuture sampleTaskBackPressure(
+   final BackPressureSampleableTask task,
+   final int remainingNumSamples,
+   final Time delayBetweenSamples,
+   final List taskBackPressureSamples,
+   final CompletableFuture resultFuture) {
+
+   if (task.isRunning()) {
+   taskBackPressureSamples.add(task.isBackPressured());
+   } else if (!taskBackPressureSamples.isEmpty()) {
+   
resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples));
+   return resultFuture;
+   } else {
+   throw new IllegalStateException(String.format("Cannot 
sample task %s. " +
+   "Because the task is not running.", 
task.getExecutionId()));
+   }
 
 Review comment:
   These three conditions seem no obvious relationship, then it is better to 
make them separate. 
   Also it should not throw `IllegalStateException` if the task is not running, 
maybe we could regard non-running task as non-backpressure state?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344006383
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
 
 Review comment:
   no need to pass `delayBetweenSamples`, it can be access within class.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344006278
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
+*/
+   public CompletableFuture 
sampleTaskBackPressure(BackPressureSampleableTask task) {
+   return sampleTaskBackPressure(
+   checkNotNull(task),
+   numSamples,
+   delayBetweenSamples,
+   new ArrayList<>(numSamples),
+   new CompletableFuture<>());
+   }
+
+   private CompletableFuture sampleTaskBackPressure(
+   final BackPressureSampleableTask task,
 
 Review comment:
   remove final for all the arguments, also for the below method.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344005130
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
 
 Review comment:
   Schedules to sample the task back pressure and returns a future


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344004540
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
+   }
+
+   /**
+* Returns a future that completes with the back pressure ratio of a 
task.
+*
+* @param task The task to be sampled.
+* @return A future of the task back pressure ratio.
 
 Review comment:
   A future contains the task back pressure ratio


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344004177
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
+   private final ScheduledExecutor scheduledExecutor;
+
+   BackPressureSampleService(
+   int numSamples,
+   Time delayBetweenSamples,
+   ScheduledExecutor scheduledExecutor) {
+
+   checkArgument(numSamples >= 1, "Illegal number of samples: " + 
numSamples);
+
+   this.numSamples = numSamples;
+   this.delayBetweenSamples = checkNotNull(delayBetweenSamples);
+   this.scheduledExecutor = checkNotNull(scheduledExecutor, "The 
scheduledExecutor must not be null.");
 
 Review comment:
   checkNotNull(scheduledExecutor) as above, the comment message seems useless.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344004012
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Samples whether a task is back pressured multi times. The total number of 
samples
+ * divided by the number of back pressure samples reaches the back pressure 
ratio.
+ */
+public class BackPressureSampleService {
+
+   /** Number of samples to take when determining the back pressure of a 
task. */
+   private final int numSamples;
+
+   /** Time to wait between samples when determining the back pressure of 
a task. */
+   private final Time delayBetweenSamples;
+
+   /** Executor to run sample tasks. */
 
 Review comment:
   to run sample back pressures


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344003590
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java
 ##
 @@ -25,40 +25,36 @@
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Back pressure statistics of multiple tasks.
- *
- * Statistics are gathered by sampling stack traces of running tasks. The
- * back pressure ratio denotes the ratio of traces indicating back pressure
- * to the total number of sampled traces.
+ * Back pressure statistics of multiple tasks generated by {@link 
BackPressureStatsTrackerImpl}.
  */
 public class OperatorBackPressureStats implements Serializable {
 
private static final long serialVersionUID = 1L;
 
-   /** ID of the corresponding sample. */
-   private final int sampleId;
+   /** ID of the corresponding request. */
+   private final int requestId;
 
-   /** End time stamp of the corresponding sample. */
+   /** End time stamp when all subtask back pressure ratios were collected 
at the JobManager. */
 
 Review comment:
   when all the responses of back pressure request were collected in 
BackPressureRequestCoordinator


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344001601
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -250,25 +179,18 @@ public void shutDown() {
}
 
/**
-* Invalidates the cache (irrespective of clean up interval).
+* Callback on completed task back pressure request.
 */
-   void invalidateOperatorStatsCache() {
-   operatorStatsCache.invalidateAll();
-   }
-
-   /**
-* Callback on completed stack trace sample.
-*/
-   class StackTraceSampleCompletionCallback implements 
BiFunction {
+   class BackPressureRequestCompletionCallback implements 
BiFunction {
 
 Review comment:
   private class


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344001624
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -250,25 +179,18 @@ public void shutDown() {
}
 
/**
-* Invalidates the cache (irrespective of clean up interval).
+* Callback on completed task back pressure request.
 */
-   void invalidateOperatorStatsCache() {
-   operatorStatsCache.invalidateAll();
-   }
-
-   /**
-* Callback on completed stack trace sample.
-*/
-   class StackTraceSampleCompletionCallback implements 
BiFunction {
+   class BackPressureRequestCompletionCallback implements 
BiFunction {
 
private final ExecutionJobVertex vertex;
 
-   public StackTraceSampleCompletionCallback(ExecutionJobVertex 
vertex) {
+   public BackPressureRequestCompletionCallback(ExecutionJobVertex 
vertex) {
 
 Review comment:
   remove `public`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344000979
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -157,70 +112,44 @@ public long getCleanUpInterval() {
synchronized (lock) {
final OperatorBackPressureStats stats = 
operatorStatsCache.getIfPresent(vertex);
if (stats == null || backPressureStatsRefreshInterval 
<= System.currentTimeMillis() - stats.getEndTimestamp()) {
-   triggerStackTraceSampleInternal(vertex);
+   triggerBackPressureRequestInternal(vertex);
}
return Optional.ofNullable(stats);
}
}
 
/**
-* Triggers a stack trace sample for a operator to gather the back 
pressure
-* statistics. If there is a sample in progress for the operator, the 
call
+* Triggers a back pressure request for a operator to gather the back 
pressure
 
 Review comment:
   operator -> vertex


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r344000581
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -47,108 +47,63 @@
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Back pressure statistics tracker.
- *
- * Back pressure is determined by sampling running tasks. If a task is
- * slowed down by back pressure it will be stuck in memory requests to a
- * {@link org.apache.flink.runtime.io.network.buffer.LocalBufferPool}.
- *
- * The back pressured stack traces look like this:
- *
- * 
- * java.lang.Object.wait(Native Method)
- * o.a.f.[...].LocalBufferPool.requestBuffer(LocalBufferPool.java:163)
- * o.a.f.[...].LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) 
<--- BLOCKING
- * request
- * [...]
- * 
+ * Back pressure statistics tracker. See {@link 
org.apache.flink.runtime.taskexecutor.BackPressureSampleService}
+ * for more details about how back pressure ratio of a task is calculated.
  */
 public class BackPressureStatsTrackerImpl implements BackPressureStatsTracker {
 
private static final Logger LOG = 
LoggerFactory.getLogger(BackPressureStatsTrackerImpl.class);
 
-   /** Maximum stack trace depth for samples. */
-   static final int MAX_STACK_TRACE_DEPTH = 8;
-
-   /** Expected class name for back pressure indicating stack trace 
element. */
-   static final String EXPECTED_CLASS_NAME = 
"org.apache.flink.runtime.io.network.buffer.LocalBufferPool";
-
-   /** Expected method name for back pressure indicating stack trace 
element. */
-   static final String EXPECTED_METHOD_NAME = 
"requestBufferBuilderBlocking";
-
/** Lock guarding trigger operations. */
private final Object lock = new Object();
 
-   /* Stack trace sample coordinator. */
-   private final StackTraceSampleCoordinator coordinator;
+   private final BackPressureRequestCoordinator coordinator;
 
/**
 * Completed stats. Important: Job vertex IDs need to be scoped by job 
ID,
-* because they are potentially constant across runs messing up the 
cached
-* data.
+* because they are potentially constant across runs which may mess up 
the
+* cached data.
 */
private final Cache 
operatorStatsCache;
 
-   /** Pending in progress stats. Important: Job vertex IDs need to be 
scoped
-* by job ID, because they are potentially constant across runs messing 
up
-* the cached data.*/
+   /**
+* Pending in progress stats. Important: Job vertex IDs need to be 
scoped
+* by job ID, because they are potentially constant across runs which 
may
+* mess up the cached data.
+*/
private final Set pendingStats = new HashSet<>();
 
-   /** Cleanup interval for completed stats cache. */
-   private final int cleanUpInterval;
-
-   private final int numSamples;
-
private final int backPressureStatsRefreshInterval;
 
-   private final Time delayBetweenSamples;
-
/** Flag indicating whether the stats tracker has been shut down. */
+   @GuardedBy("lock")
private boolean shutDown;
 
/**
 * Creates a back pressure statistics tracker.
 *
 * @param cleanUpInterval Clean up interval for completed stats.
-* @param numSamples  Number of stack trace samples when 
determining back pressure.
-* @param delayBetweenSamples Delay between samples when determining 
back pressure.
+* @param backPressureStatsRefreshInterval
 */
public BackPressureStatsTrackerImpl(
-   StackTraceSampleCoordinator coordinator,
+   BackPressureRequestCoordinator coordinator,
int cleanUpInterval,
-   int numSamples,
-   int backPressureStatsRefreshInterval,
-   Time delayBetweenSamples) {
-
-   this.coordinator = checkNotNull(coordinator, "Stack trace 
sample coordinator");
-
-   checkArgument(cleanUpInterval >= 0, "Clean up interval");
-   this.cleanUpInterval = cleanUpInterval;
+   int backPressureStatsRefreshInterval) {
 
-   checkArgument(numSamples >= 1, "Number of samples");
-   this.numSamples = numSamples;
+   checkArgument(backPressureStatsRefreshInterval >= 0,
+   "The back pressure stats refresh interval must be 
greater than or equal to 0.");
 
-   checkArgument(
-   backPressureStatsRefreshInterval >= 0,
-   "backPressureStatsRefreshInterval must be greater than 
or equal to 0");
+

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343999496
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -47,108 +47,63 @@
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Back pressure statistics tracker.
- *
- * Back pressure is determined by sampling running tasks. If a task is
- * slowed down by back pressure it will be stuck in memory requests to a
- * {@link org.apache.flink.runtime.io.network.buffer.LocalBufferPool}.
- *
- * The back pressured stack traces look like this:
- *
- * 
- * java.lang.Object.wait(Native Method)
- * o.a.f.[...].LocalBufferPool.requestBuffer(LocalBufferPool.java:163)
- * o.a.f.[...].LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) 
<--- BLOCKING
- * request
- * [...]
- * 
+ * Back pressure statistics tracker. See {@link 
org.apache.flink.runtime.taskexecutor.BackPressureSampleService}
+ * for more details about how back pressure ratio of a task is calculated.
  */
 public class BackPressureStatsTrackerImpl implements BackPressureStatsTracker {
 
private static final Logger LOG = 
LoggerFactory.getLogger(BackPressureStatsTrackerImpl.class);
 
-   /** Maximum stack trace depth for samples. */
-   static final int MAX_STACK_TRACE_DEPTH = 8;
-
-   /** Expected class name for back pressure indicating stack trace 
element. */
-   static final String EXPECTED_CLASS_NAME = 
"org.apache.flink.runtime.io.network.buffer.LocalBufferPool";
-
-   /** Expected method name for back pressure indicating stack trace 
element. */
-   static final String EXPECTED_METHOD_NAME = 
"requestBufferBuilderBlocking";
-
/** Lock guarding trigger operations. */
private final Object lock = new Object();
 
-   /* Stack trace sample coordinator. */
-   private final StackTraceSampleCoordinator coordinator;
+   private final BackPressureRequestCoordinator coordinator;
 
/**
 * Completed stats. Important: Job vertex IDs need to be scoped by job 
ID,
-* because they are potentially constant across runs messing up the 
cached
-* data.
+* because they are potentially constant across runs which may mess up 
the
+* cached data.
 */
private final Cache 
operatorStatsCache;
 
-   /** Pending in progress stats. Important: Job vertex IDs need to be 
scoped
-* by job ID, because they are potentially constant across runs messing 
up
-* the cached data.*/
+   /**
+* Pending in progress stats. Important: Job vertex IDs need to be 
scoped
+* by job ID, because they are potentially constant across runs which 
may
+* mess up the cached data.
+*/
private final Set pendingStats = new HashSet<>();
 
-   /** Cleanup interval for completed stats cache. */
-   private final int cleanUpInterval;
-
-   private final int numSamples;
-
private final int backPressureStatsRefreshInterval;
 
-   private final Time delayBetweenSamples;
-
/** Flag indicating whether the stats tracker has been shut down. */
+   @GuardedBy("lock")
private boolean shutDown;
 
/**
 * Creates a back pressure statistics tracker.
 *
 * @param cleanUpInterval Clean up interval for completed stats.
-* @param numSamples  Number of stack trace samples when 
determining back pressure.
-* @param delayBetweenSamples Delay between samples when determining 
back pressure.
+* @param backPressureStatsRefreshInterval
 
 Review comment:
   add one more param for `coordinator` and give some comments for 
`backPressureStatsRefreshInterval`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343999496
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java
 ##
 @@ -47,108 +47,63 @@
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Back pressure statistics tracker.
- *
- * Back pressure is determined by sampling running tasks. If a task is
- * slowed down by back pressure it will be stuck in memory requests to a
- * {@link org.apache.flink.runtime.io.network.buffer.LocalBufferPool}.
- *
- * The back pressured stack traces look like this:
- *
- * 
- * java.lang.Object.wait(Native Method)
- * o.a.f.[...].LocalBufferPool.requestBuffer(LocalBufferPool.java:163)
- * o.a.f.[...].LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) 
<--- BLOCKING
- * request
- * [...]
- * 
+ * Back pressure statistics tracker. See {@link 
org.apache.flink.runtime.taskexecutor.BackPressureSampleService}
+ * for more details about how back pressure ratio of a task is calculated.
  */
 public class BackPressureStatsTrackerImpl implements BackPressureStatsTracker {
 
private static final Logger LOG = 
LoggerFactory.getLogger(BackPressureStatsTrackerImpl.class);
 
-   /** Maximum stack trace depth for samples. */
-   static final int MAX_STACK_TRACE_DEPTH = 8;
-
-   /** Expected class name for back pressure indicating stack trace 
element. */
-   static final String EXPECTED_CLASS_NAME = 
"org.apache.flink.runtime.io.network.buffer.LocalBufferPool";
-
-   /** Expected method name for back pressure indicating stack trace 
element. */
-   static final String EXPECTED_METHOD_NAME = 
"requestBufferBuilderBlocking";
-
/** Lock guarding trigger operations. */
private final Object lock = new Object();
 
-   /* Stack trace sample coordinator. */
-   private final StackTraceSampleCoordinator coordinator;
+   private final BackPressureRequestCoordinator coordinator;
 
/**
 * Completed stats. Important: Job vertex IDs need to be scoped by job 
ID,
-* because they are potentially constant across runs messing up the 
cached
-* data.
+* because they are potentially constant across runs which may mess up 
the
+* cached data.
 */
private final Cache 
operatorStatsCache;
 
-   /** Pending in progress stats. Important: Job vertex IDs need to be 
scoped
-* by job ID, because they are potentially constant across runs messing 
up
-* the cached data.*/
+   /**
+* Pending in progress stats. Important: Job vertex IDs need to be 
scoped
+* by job ID, because they are potentially constant across runs which 
may
+* mess up the cached data.
+*/
private final Set pendingStats = new HashSet<>();
 
-   /** Cleanup interval for completed stats cache. */
-   private final int cleanUpInterval;
-
-   private final int numSamples;
-
private final int backPressureStatsRefreshInterval;
 
-   private final Time delayBetweenSamples;
-
/** Flag indicating whether the stats tracker has been shut down. */
+   @GuardedBy("lock")
private boolean shutDown;
 
/**
 * Creates a back pressure statistics tracker.
 *
 * @param cleanUpInterval Clean up interval for completed stats.
-* @param numSamples  Number of stack trace samples when 
determining back pressure.
-* @param delayBetweenSamples Delay between samples when determining 
back pressure.
+* @param backPressureStatsRefreshInterval
 
 Review comment:
   wrong param, it should be `coordinator`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343990958
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java
 ##
 @@ -0,0 +1,87 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Back pressure stats for one or more tasks.
+ *
+ * The stats are collected by request triggered in {@link 
BackPressureRequestCoordinator}.
+ */
+public class BackPressureStats {
+
+   /** ID of the request (unique per job). */
+   private final int requestId;
+
+   /** Time stamp, when the request was triggered. */
+   private final long startTime;
+
+   /** Time stamp, when all back pressure stats were collected at the 
JobManager. */
+   private final long endTime;
+
+   /** Map of back pressure ratio by execution ID. */
+   private final Map backPressureRatios;
+
+   public BackPressureStats(
+   int requestId,
+   long startTime,
+   long endTime,
+   Map backPressureRatios) {
+
+   checkArgument(requestId >= 0, "Negative request ID.");
+   checkArgument(startTime >= 0, "Negative start time.");
+   checkArgument(endTime >= startTime, "End time before start 
time.");
+
+   this.requestId = requestId;
+   this.startTime = startTime;
+   this.endTime = endTime;
+   this.backPressureRatios = 
Collections.unmodifiableMap(backPressureRatios);
 
 Review comment:
   Collections.unmodifiableMap(checkNotNull(backPressureRatios))


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343990718
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java
 ##
 @@ -0,0 +1,87 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Back pressure stats for one or more tasks.
+ *
+ * The stats are collected by request triggered in {@link 
BackPressureRequestCoordinator}.
+ */
+public class BackPressureStats {
+
+   /** ID of the request (unique per job). */
+   private final int requestId;
+
+   /** Time stamp, when the request was triggered. */
+   private final long startTime;
+
+   /** Time stamp, when all back pressure stats were collected at the 
JobManager. */
+   private final long endTime;
+
+   /** Map of back pressure ratio by execution ID. */
 
 Review comment:
   ratio -> ratios


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343990575
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java
 ##
 @@ -0,0 +1,87 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Back pressure stats for one or more tasks.
+ *
+ * The stats are collected by request triggered in {@link 
BackPressureRequestCoordinator}.
+ */
+public class BackPressureStats {
+
+   /** ID of the request (unique per job). */
+   private final int requestId;
+
+   /** Time stamp, when the request was triggered. */
+   private final long startTime;
+
+   /** Time stamp, when all back pressure stats were collected at the 
JobManager. */
 
 Review comment:
   JobManager -> BackPressureRequestCoordinator


This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services


[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343990088
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinator.java
 ##
 @@ -0,0 +1,365 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A coordinator for triggering and collecting back pressure stats
+ * of running tasks.
+ */
+public class BackPressureRequestCoordinator {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(BackPressureRequestCoordinator.class);
+
+   private static final int NUM_GHOST_REQUEST_IDS = 10;
+
+   private final Object lock = new Object();
+
+   /** Executor used to run the futures. */
+   private final Executor executor;
+
+   /** Request time out of a triggered back pressure request. */
+   private final Time requestTimeout;
+
+   /** In progress back pressure requests. */
+   @GuardedBy("lock")
+   private final Map pendingRequests 
= new HashMap<>();
+
+   /** A list of recent request IDs to identify late messages vs. invalid 
ones. */
+   private final ArrayDeque recentPendingRequests = new 
ArrayDeque<>(NUM_GHOST_REQUEST_IDS);
+
+   /** Request ID counter. */
+   @GuardedBy("lock")
+   private int requestIdCounter;
+
+   /** Flag indicating whether the coordinator is still running. */
+   @GuardedBy("lock")
+   private boolean isShutDown;
+
+   /**
+* Creates a new coordinator for the cluster.
+*
+* @param executor Used to execute the futures.
+* @param requestTimeout Request time out of a triggered back pressure 
request.
+*/
+   public BackPressureRequestCoordinator(
+   Executor executor,
+   long requestTimeout) {
+
+   checkArgument(requestTimeout >= 0L, "Illegal request timeout: " 
+ requestTimeout);
+
+   this.executor = checkNotNull(executor);
+   this.requestTimeout = Time.milliseconds(requestTimeout);
+   }
+
+   /**
+* Triggers a task back pressure stats request to all tasks.
+*
+* @param tasks Tasks to request.
+* @return A future of the completed task back pressure stats.
+*/
+   CompletableFuture 
triggerBackPressureRequest(ExecutionVertex[] tasks) {
+
+   checkNotNull(tasks, "Tasks to request must not be null.");
+   checkArgument(tasks.length >= 1, "No tasks to request.");
+
+   // Execution IDs of running tasks
+   ExecutionAttemptID[] triggerIds = new 
ExecutionAttemptID[tasks.length];
+   Execution[] executions = new Execution[tasks.length];
+
+   // Check that all tasks are RUNNING before triggering anything. 
The
+   // triggering can still fail.
+   for (int i = 0; i < triggerIds.length; i++) {
+ 

[GitHub] [flink] zhijiangW commented on a change in pull request #10083: [FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.

2019-11-07 Thread GitBox
zhijiangW commented on a change in pull request #10083: 
[FLINK-14472][runtime]Implement back-pressure monitor with non-blocking outputs.
URL: https://github.com/apache/flink/pull/10083#discussion_r343990088
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinator.java
 ##
 @@ -0,0 +1,365 @@
+/*
+ * 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.rest.handler.legacy.backpressure;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.messages.TaskBackPressureResponse;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A coordinator for triggering and collecting back pressure stats
+ * of running tasks.
+ */
+public class BackPressureRequestCoordinator {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(BackPressureRequestCoordinator.class);
+
+   private static final int NUM_GHOST_REQUEST_IDS = 10;
+
+   private final Object lock = new Object();
+
+   /** Executor used to run the futures. */
+   private final Executor executor;
+
+   /** Request time out of a triggered back pressure request. */
+   private final Time requestTimeout;
+
+   /** In progress back pressure requests. */
+   @GuardedBy("lock")
+   private final Map pendingRequests 
= new HashMap<>();
+
+   /** A list of recent request IDs to identify late messages vs. invalid 
ones. */
+   private final ArrayDeque recentPendingRequests = new 
ArrayDeque<>(NUM_GHOST_REQUEST_IDS);
+
+   /** Request ID counter. */
+   @GuardedBy("lock")
+   private int requestIdCounter;
+
+   /** Flag indicating whether the coordinator is still running. */
+   @GuardedBy("lock")
+   private boolean isShutDown;
+
+   /**
+* Creates a new coordinator for the cluster.
+*
+* @param executor Used to execute the futures.
+* @param requestTimeout Request time out of a triggered back pressure 
request.
+*/
+   public BackPressureRequestCoordinator(
+   Executor executor,
+   long requestTimeout) {
+
+   checkArgument(requestTimeout >= 0L, "Illegal request timeout: " 
+ requestTimeout);
+
+   this.executor = checkNotNull(executor);
+   this.requestTimeout = Time.milliseconds(requestTimeout);
+   }
+
+   /**
+* Triggers a task back pressure stats request to all tasks.
+*
+* @param tasks Tasks to request.
+* @return A future of the completed task back pressure stats.
+*/
+   CompletableFuture 
triggerBackPressureRequest(ExecutionVertex[] tasks) {
+
+   checkNotNull(tasks, "Tasks to request must not be null.");
+   checkArgument(tasks.length >= 1, "No tasks to request.");
+
+   // Execution IDs of running tasks
+   ExecutionAttemptID[] triggerIds = new 
ExecutionAttemptID[tasks.length];
+   Execution[] executions = new Execution[tasks.length];
+
+   // Check that all tasks are RUNNING before triggering anything. 
The
+   // triggering can still fail.
+   for (int i = 0; i < triggerIds.length; i++) {
+ 

  1   2   >