[GitHub] [flink] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-07-02 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r299754527
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SchedulingUtils.java
 ##
 @@ -0,0 +1,218 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import 
org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmaster.slotpool.Scheduler;
+import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This class contains scheduling logic for EAGER and LAZY_FROM_SOURCES.
+ * It is used for normal scheduling and legacy failover strategy re-scheduling.
+ */
+public class SchedulingUtils {
+
+   /**
+* Schedule vertices lazy. That means only vertices satisfying its 
input constraint will be scheduled.
+*
+* @param vertices Topologically sorted vertices to schedule.
+* @param executionGraph The graph the given vertices belongs to.
+*/
+   public static CompletableFuture scheduleLazy(
+   final Iterable vertices,
+   final ExecutionGraph executionGraph) {
+
+   executionGraph.assertRunningInJobMasterMainThread();
+
+   final Set previousAllocations = 
computePriorAllocationIdsIfRequiredByScheduling(
+   vertices, executionGraph.getSlotProvider());
+
+   final ArrayList> schedulingFutures = 
new ArrayList<>();
+   for (ExecutionVertex executionVertex : vertices) {
+   // only schedule vertex when its input constraint is 
satisfied
+   if 
(executionVertex.getJobVertex().getJobVertex().isInputVertex() ||
+   
executionVertex.checkInputDependencyConstraints()) {
 
 Review comment:
   current `executionVertex.checkInputDependencyConstraints` does not proper 
deal with the input vertex case. If the constraint is ANY and the vertex has no 
input, it will return false.
   Maybe we can fix the `checkInputDependencyConstraints` and simplify the 
check 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] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-07-02 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r299754065
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java
 ##
 @@ -0,0 +1,316 @@
+/*
+ * 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.executiongraph.failover;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.Execution;
+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.executiongraph.GlobalModVersionMismatch;
+import org.apache.flink.runtime.executiongraph.SchedulingUtils;
+import 
org.apache.flink.runtime.executiongraph.failover.adapter.DefaultFailoverTopology;
+import 
org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersion;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This failover strategy uses flip1.RestartPipelinedRegionStrategy to make 
task failover decisions.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNG extends FailoverStrategy {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(AdaptedRestartPipelinedRegionStrategyNG.class);
+
+   /** The execution graph on which this FailoverStrategy works. */
+   private final ExecutionGraph executionGraph;
+
+   /** The versioner helps to maintain execution vertex versions. */
+   private final ExecutionVertexVersioner executionVertexVersioner;
+
+   /** The underlying new generation region failover strategy. */
+   private RestartPipelinedRegionStrategy restartPipelinedRegionStrategy;
+
+   public AdaptedRestartPipelinedRegionStrategyNG(final ExecutionGraph 
executionGraph) {
+   this.executionGraph = checkNotNull(executionGraph);
+   this.executionVertexVersioner = new ExecutionVertexVersioner();
+   }
+
+   @Override
+   public void onTaskFailure(final Execution taskExecution, final 
Throwable cause) {
+   if (!executionGraph.getRestartStrategy().canRestart()) {
+   // delegate the failure to a global fail that will 
check the restart strategy and not restart
+   LOG.info("Fail to pass the restart strategy validation 
in region failover. Fallback to fail global.");
+   failGlobal(cause);
+   return;
+   }
+
+   if 
(!isLocalFailoverValid(executionGraph.getGlobalModVersion())) {
+   LOG.info("Skip current region failover as a global 
failover is ongoing.");
+   return;
+   }
+
+   final ExecutionVertexID vertexID = 

[GitHub] [flink] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-30 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298865535
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverStrategyLoader.java
 ##
 @@ -38,9 +38,12 @@
/** Config name for the {@link RestartIndividualStrategy} */
public static final String INDIVIDUAL_RESTART_STRATEGY_NAME = 
"individual";
 
-   /** Config name for the {@link RestartPipelinedRegionStrategy} */
+   /** Config name for the {@link AdaptedRestartPipelinedRegionStrategyNG} 
*/
public static final String PIPELINED_REGION_RESTART_STRATEGY_NAME = 
"region";
 
+   /** Config name for the {@link RestartPipelinedRegionStrategy} */
 
 Review comment:
   A "."  is needed at the end of the statement. 
   Also for the other 3 config name comments.


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] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-28 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298552634
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java
 ##
 @@ -0,0 +1,282 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import 
org.apache.flink.runtime.executiongraph.AdaptedRestartPipelinedRegionStrategyNGFailoverTest.TestAdaptedRestartPipelinedRegionStrategyNG;
+import 
org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AdaptedRestartPipelinedRegionStrategyNG} failover handling 
when concurrent failovers happen.
+ * There can be local+local and local+global concurrent failovers.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest 
extends TestLogger {
+
+   private static final JobID TEST_JOB_ID = new JobID();
+
+   private static final int DEFAULT_PARALLELISM = 2;
+
+   private ManuallyTriggeredScheduledExecutor manualMainThreadExecutor;
+
+   private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+   private TestRestartStrategy manuallyTriggeredRestartStrategy;
+
+   @Before
+   public void setUp() {
+   manualMainThreadExecutor = new 
ManuallyTriggeredScheduledExecutor();
+   componentMainThreadExecutor = new 
ScheduledExecutorToComponentMainThreadExecutorAdapter(manualMainThreadExecutor, 
Thread.currentThread());
+   manuallyTriggeredRestartStrategy = 
TestRestartStrategy.manuallyTriggered();
+   }
+
+   /**
+* Tests that 2 concurrent region failovers can lead to a properly 
vertex state.
+* 
+* (v11) -+-> (v21)
+*x
+* (v12) -+-> (v22)
+*
+*^
+*|
+*   (blocking)
+* 
+*/
+   @Test
+   public void testConcurrentRegionFailovers() throws Exception {
+
+   // the logic in this test is as follows:
+   //  - start a job
+   //  - cause {ev11} failure and delay the local recovery action 
via the manual executor
+   //  - cause {ev12} failure and delay the local recovery action 
via the manual executor
+   //  - resume local recovery actions
+   //  - validate that each task is restarted only once
+
+   final ExecutionGraph eg = createExecutionGraph();
+
+   final TestAdaptedRestartPipelinedRegionStrategyNG 
failoverStrategy =
+   (TestAdaptedRestartPipelinedRegionStrategyNG) 
eg.getFailoverStrategy();
+   failoverStrategy.setBlockerFuture(new CompletableFuture<>());
+
+   final Iterator vertexIterator = 
eg.getAllExecutionVertices().iterator();
+   final ExecutionVertex ev11 = 

[GitHub] [flink] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-28 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298552634
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java
 ##
 @@ -0,0 +1,282 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import 
org.apache.flink.runtime.executiongraph.AdaptedRestartPipelinedRegionStrategyNGFailoverTest.TestAdaptedRestartPipelinedRegionStrategyNG;
+import 
org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AdaptedRestartPipelinedRegionStrategyNG} failover handling 
when concurrent failovers happen.
+ * There can be local+local and local+global concurrent failovers.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest 
extends TestLogger {
+
+   private static final JobID TEST_JOB_ID = new JobID();
+
+   private static final int DEFAULT_PARALLELISM = 2;
+
+   private ManuallyTriggeredScheduledExecutor manualMainThreadExecutor;
+
+   private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+   private TestRestartStrategy manuallyTriggeredRestartStrategy;
+
+   @Before
+   public void setUp() {
+   manualMainThreadExecutor = new 
ManuallyTriggeredScheduledExecutor();
+   componentMainThreadExecutor = new 
ScheduledExecutorToComponentMainThreadExecutorAdapter(manualMainThreadExecutor, 
Thread.currentThread());
+   manuallyTriggeredRestartStrategy = 
TestRestartStrategy.manuallyTriggered();
+   }
+
+   /**
+* Tests that 2 concurrent region failovers can lead to a properly 
vertex state.
+* 
+* (v11) -+-> (v21)
+*x
+* (v12) -+-> (v22)
+*
+*^
+*|
+*   (blocking)
+* 
+*/
+   @Test
+   public void testConcurrentRegionFailovers() throws Exception {
+
+   // the logic in this test is as follows:
+   //  - start a job
+   //  - cause {ev11} failure and delay the local recovery action 
via the manual executor
+   //  - cause {ev12} failure and delay the local recovery action 
via the manual executor
+   //  - resume local recovery actions
+   //  - validate that each task is restarted only once
+
+   final ExecutionGraph eg = createExecutionGraph();
+
+   final TestAdaptedRestartPipelinedRegionStrategyNG 
failoverStrategy =
+   (TestAdaptedRestartPipelinedRegionStrategyNG) 
eg.getFailoverStrategy();
+   failoverStrategy.setBlockerFuture(new CompletableFuture<>());
+
+   final Iterator vertexIterator = 
eg.getAllExecutionVertices().iterator();
+   final ExecutionVertex ev11 = 

[GitHub] [flink] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-28 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298552634
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java
 ##
 @@ -0,0 +1,282 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import 
org.apache.flink.runtime.executiongraph.AdaptedRestartPipelinedRegionStrategyNGFailoverTest.TestAdaptedRestartPipelinedRegionStrategyNG;
+import 
org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AdaptedRestartPipelinedRegionStrategyNG} failover handling 
when concurrent failovers happen.
+ * There can be local+local and local+global concurrent failovers.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest 
extends TestLogger {
+
+   private static final JobID TEST_JOB_ID = new JobID();
+
+   private static final int DEFAULT_PARALLELISM = 2;
+
+   private ManuallyTriggeredScheduledExecutor manualMainThreadExecutor;
+
+   private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+   private TestRestartStrategy manuallyTriggeredRestartStrategy;
+
+   @Before
+   public void setUp() {
+   manualMainThreadExecutor = new 
ManuallyTriggeredScheduledExecutor();
+   componentMainThreadExecutor = new 
ScheduledExecutorToComponentMainThreadExecutorAdapter(manualMainThreadExecutor, 
Thread.currentThread());
+   manuallyTriggeredRestartStrategy = 
TestRestartStrategy.manuallyTriggered();
+   }
+
+   /**
+* Tests that 2 concurrent region failovers can lead to a properly 
vertex state.
+* 
+* (v11) -+-> (v21)
+*x
+* (v12) -+-> (v22)
+*
+*^
+*|
+*   (blocking)
+* 
+*/
+   @Test
+   public void testConcurrentRegionFailovers() throws Exception {
+
+   // the logic in this test is as follows:
+   //  - start a job
+   //  - cause {ev11} failure and delay the local recovery action 
via the manual executor
+   //  - cause {ev12} failure and delay the local recovery action 
via the manual executor
+   //  - resume local recovery actions
+   //  - validate that each task is restarted only once
+
+   final ExecutionGraph eg = createExecutionGraph();
+
+   final TestAdaptedRestartPipelinedRegionStrategyNG 
failoverStrategy =
+   (TestAdaptedRestartPipelinedRegionStrategyNG) 
eg.getFailoverStrategy();
+   failoverStrategy.setBlockerFuture(new CompletableFuture<>());
+
+   final Iterator vertexIterator = 
eg.getAllExecutionVertices().iterator();
+   final ExecutionVertex ev11 = 

[GitHub] [flink] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-28 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298517179
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java
 ##
 @@ -0,0 +1,513 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.types.SlotProfile;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import 
org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG;
+import 
org.apache.flink.runtime.executiongraph.restart.InfiniteDelayRestartStrategy;
+import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.instance.SlotSharingGroupId;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import 
org.apache.flink.runtime.io.network.partition.consumer.PartitionConnectionException;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.ScheduleMode;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.jobmaster.SlotRequestId;
+import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link AdaptedRestartPipelinedRegionStrategyNG} failover handling.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNGFailoverTest extends 
TestLogger {
+
+   private static final JobID TEST_JOB_ID = new JobID();
+
+   private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+   private FailingSlotProviderDecorator slotProvider;
+
+   private ManuallyTriggeredScheduledExecutor manualMainThreadExecutor;
+
+   @Before
+   public void setUp() {
+   manualMainThreadExecutor = new 
ManuallyTriggeredScheduledExecutor();
+   componentMainThreadExecutor = new 
ScheduledExecutorToComponentMainThreadExecutorAdapter(manualMainThreadExecutor, 
Thread.currentThread());
+   slotProvider = new FailingSlotProviderDecorator(new 
SimpleSlotProvider(TEST_JOB_ID, 14));
+   }
+
+   /**
+* Tests for region failover for job in EAGER mode.
+* This applies to streaming job, with no BLOCKING edge.
+* 
+* (v11) ---> (v21)
+*
+* (v12) ---> (v22)
+*
+*^
+ 

[GitHub] [flink] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-28 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298529362
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java
 ##
 @@ -208,7 +211,7 @@ public void testFailurePropagationToUnderlyingStrategy() 
throws Exception {
final ExecutionVertex ev21 = vertexIterator.next();
 
// trigger downstream regions to schedule
-   testingMainThreadExecutor.execute(() -> {
+   componentMainThreadExecutor.execute(() -> {
 
 Review comment:
   can invoke `markFinished` directly.


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] zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] Add an adapter of region failover NG for legacy scheduler

2019-06-28 Thread GitBox
zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298524268
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java
 ##
 @@ -0,0 +1,282 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import 
org.apache.flink.runtime.executiongraph.AdaptedRestartPipelinedRegionStrategyNGFailoverTest.TestAdaptedRestartPipelinedRegionStrategyNG;
+import 
org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AdaptedRestartPipelinedRegionStrategyNG} failover handling 
when concurrent failovers happen.
+ * There can be local+local and local+global concurrent failovers.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest 
extends TestLogger {
+
+   private static final JobID TEST_JOB_ID = new JobID();
+
+   private static final int DEFAULT_PARALLELISM = 2;
+
+   private ManuallyTriggeredScheduledExecutor manualMainThreadExecutor;
+
+   private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+   private TestRestartStrategy manuallyTriggeredRestartStrategy;
+
+   @Before
+   public void setUp() {
+   manualMainThreadExecutor = new 
ManuallyTriggeredScheduledExecutor();
+   componentMainThreadExecutor = new 
ScheduledExecutorToComponentMainThreadExecutorAdapter(manualMainThreadExecutor, 
Thread.currentThread());
+   manuallyTriggeredRestartStrategy = 
TestRestartStrategy.manuallyTriggered();
+   }
+
+   /**
+* Tests that 2 concurrent region failovers can lead to a properly 
vertex state.
+* 
+* (v11) -+-> (v21)
+*x
+* (v12) -+-> (v22)
+*
+*^
+*|
+*   (blocking)
+* 
+*/
+   @Test
+   public void testConcurrentRegionFailovers() throws Exception {
+
+   // the logic in this test is as follows:
+   //  - start a job
+   //  - cause {ev11} failure and delay the local recovery action 
via the manual executor
+   //  - cause {ev12} failure and delay the local recovery action 
via the manual executor
+   //  - resume local recovery actions
+   //  - validate that each task is restarted only once
+
+   final ExecutionGraph eg = createExecutionGraph();
+
+   final TestAdaptedRestartPipelinedRegionStrategyNG 
failoverStrategy =
+   (TestAdaptedRestartPipelinedRegionStrategyNG) 
eg.getFailoverStrategy();
+   failoverStrategy.setBlockerFuture(new CompletableFuture<>());
+
+   final Iterator vertexIterator = 
eg.getAllExecutionVertices().iterator();
+   final ExecutionVertex ev11 =