tillrohrmann commented on a change in pull request #13880:
URL: https://github.com/apache/flink/pull/13880#discussion_r519445776



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerComponentsFactoryTest.java
##########
@@ -67,9 +72,56 @@ public void 
testCreatingPipelinedRegionSchedulingStrategyFactoryByDefault() {
                assertThat(components.getSchedulingStrategyFactory(), 
instanceOf(PipelinedRegionSchedulingStrategy.Factory.class));
        }
 
+       @Test
+       public void 
testCreatingPipelinedRegionSchedulingStrategyFactoryWithApproximateLocalRecovery()
 {
+               final Configuration configuration = new Configuration();
+               configuration.setString(JobManagerOptions.SCHEDULING_STRATEGY, 
"region");
+
+               try {
+                       createSchedulerComponents(configuration, true, EAGER);
+                       fail("expected failure");
+               } catch (IllegalArgumentException e) {
+                       assertTrue(
+                               e.getMessage()
+                                       .contains("Approximate local recovery 
can not be used together with PipelinedRegionScheduler for now"));

Review comment:
       nit: Could use `FlinkMatchers.containsMessage`

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerComponentsFactoryTest.java
##########
@@ -67,9 +72,56 @@ public void 
testCreatingPipelinedRegionSchedulingStrategyFactoryByDefault() {
                assertThat(components.getSchedulingStrategyFactory(), 
instanceOf(PipelinedRegionSchedulingStrategy.Factory.class));
        }
 
+       @Test
+       public void 
testCreatingPipelinedRegionSchedulingStrategyFactoryWithApproximateLocalRecovery()
 {
+               final Configuration configuration = new Configuration();
+               configuration.setString(JobManagerOptions.SCHEDULING_STRATEGY, 
"region");
+
+               try {
+                       createSchedulerComponents(configuration, true, EAGER);
+                       fail("expected failure");
+               } catch (IllegalArgumentException e) {
+                       assertTrue(
+                               e.getMessage()
+                                       .contains("Approximate local recovery 
can not be used together with PipelinedRegionScheduler for now"));
+               }
+       }
+
+       @Test
+       public void 
testCreatingLegacySchedulingStrategyFactoryWithApproximateLocalRecoveryInLazyMode()
 {
+               final Configuration configuration = new Configuration();
+               configuration.setString(JobManagerOptions.SCHEDULING_STRATEGY, 
"legacy");
+
+               try {
+                       createSchedulerComponents(configuration, true, 
LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST);
+                       fail("expected failure");
+               } catch (IllegalArgumentException e) {
+                       assertTrue(
+                               e.getMessage()
+                                       .contains("Approximate local recovery 
can only be used together with EAGER schedule mode"));

Review comment:
       Same here.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
##########
@@ -288,10 +288,24 @@ private void configureStreamGraph(final StreamGraph 
graph) {
                } else {
                        graph.setStateBackend(stateBackend);
                        
graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.ALL_EDGES_PIPELINED);
+                       
graph.setAllVerticesInSameSlotSharingGroupByDefault(true);
                        graph.setScheduleMode(ScheduleMode.EAGER);
+
+                       if 
(checkpointConfig.isApproximateLocalRecoveryEnabled()) {
+                               checkApproximateLocalRecoveryCompatibility();
+                               
graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.ALL_EDGES_PIPELINED_APPROXIMATE);
+                       } else {
+                               
graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.ALL_EDGES_PIPELINED);
+                       }
                }
        }
 
+       private void checkApproximateLocalRecoveryCompatibility() {
+               checkState(
+                       !checkpointConfig.isUnalignedCheckpointsEnabled(),
+                       "Approximate Local Recovery and Unaligned Checkpoint 
can not be used together yet");

Review comment:
       Yes I think that we need to do the check on the cluster/JobMaster-side. 
At the moment we haven't exposed this configuration on a per-job basis because 
ideally we only have a single scheduler which is able to serve all workloads.




----------------------------------------------------------------
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:
[email protected]


Reply via email to