-
Notifications
You must be signed in to change notification settings - Fork 13k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-19693][runtime] Downstream Failover for Approximate Local Recovery #13880
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,14 +26,19 @@ | |
import org.apache.flink.runtime.jobgraph.ScheduleMode; | ||
import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; | ||
import org.apache.flink.runtime.jobmaster.slotpool.TestingSlotPoolImpl; | ||
import org.apache.flink.runtime.scheduler.strategy.EagerSchedulingStrategy; | ||
import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy; | ||
import org.apache.flink.runtime.scheduler.strategy.PipelinedRegionSchedulingStrategy; | ||
import org.apache.flink.util.TestLogger; | ||
|
||
import org.junit.Test; | ||
|
||
import static org.apache.flink.runtime.jobgraph.ScheduleMode.EAGER; | ||
import static org.apache.flink.runtime.jobgraph.ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST; | ||
import static org.hamcrest.Matchers.instanceOf; | ||
import static org.junit.Assert.assertThat; | ||
import static org.junit.Assert.assertTrue; | ||
import static org.junit.Assert.fail; | ||
|
||
/** | ||
* Tests for the factory method {@link DefaultSchedulerComponents#createSchedulerComponents( | ||
|
@@ -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")); | ||
Comment on lines
+99
to
+101
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same here. |
||
} | ||
} | ||
|
||
@Test | ||
public void testCreatingLegacySchedulingStrategyFactoryWithApproximateLocalRecoveryInEagerMode() { | ||
final Configuration configuration = new Configuration(); | ||
configuration.setString(JobManagerOptions.SCHEDULING_STRATEGY, "legacy"); | ||
|
||
final DefaultSchedulerComponents components = createSchedulerComponents(configuration, true, EAGER); | ||
assertThat(components.getSchedulingStrategyFactory(), instanceOf(EagerSchedulingStrategy.Factory.class)); | ||
} | ||
|
||
private static DefaultSchedulerComponents createSchedulerComponents(final Configuration configuration) { | ||
return createSchedulerComponents(configuration, false, LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST); | ||
} | ||
|
||
private static DefaultSchedulerComponents createSchedulerComponents( | ||
final Configuration configuration, | ||
boolean iApproximateLocalRecoveryEnabled, | ||
ScheduleMode scheduleMode) { | ||
return DefaultSchedulerComponents.createSchedulerComponents( | ||
ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST, | ||
scheduleMode, | ||
iApproximateLocalRecoveryEnabled, | ||
configuration, | ||
new TestingSlotPoolImpl(new JobID()), | ||
Time.milliseconds(10L)); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -287,11 +287,23 @@ private void configureStreamGraph(final StreamGraph graph) { | |
setBatchStateBackendAndTimerService(graph); | ||
} else { | ||
graph.setStateBackend(stateBackend); | ||
graph.setGlobalDataExchangeMode(GlobalDataExchangeMode.ALL_EDGES_PIPELINED); | ||
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"); | ||
Comment on lines
+302
to
+304
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we do a similar check for the scheduler configuration? I think we should fail hard if we see that we are not using the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I've tried out different ways to do the check. It is slightly different from the unaligned checkpoint check, because the PipelinedRegionScheduler is configured in the JobMaster level, and approximate local recovery is configured at the job level. (BTW, I can kind of guessing the reason why it is a job-master level config, but still curious why because I probably would have the same question later when I am constructing a separate failover strategy. We can probably discuss this later). The check should happen in the same place where either PIPELINED_REGION_SCHEDULING or LEGACY_SCHEDULING is chosen, where the type SchedulingStrategyFactory is decided.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will also add a test for this. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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. |
||
} | ||
|
||
private void setBatchStateBackendAndTimerService(StreamGraph graph) { | ||
boolean useStateBackend = configuration.get(ExecutionOptions.USE_BATCH_STATE_BACKEND); | ||
boolean sortInputs = configuration.get(ExecutionOptions.SORT_INPUTS); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: Could use
FlinkMatchers.containsMessage