Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,133 @@
/*
* 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.jobmaster;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.runtime.execution.Environment;
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.JobVertex;
import org.apache.flink.runtime.jobmanager.Tasks;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
import org.apache.flink.runtime.minicluster.MiniCluster;
import org.apache.flink.runtime.testutils.MiniClusterResource;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.TestLogger;

import org.junit.ClassRule;
import org.junit.Test;

import java.io.IOException;
import java.util.concurrent.CompletableFuture;

import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;

/**
* Tests for the recovery of task failures.
*/
public class JobRecoveryITCase extends TestLogger {

private static final int NUM_TMS = 1;
private static final int SLOTS_PER_TM = 11;
private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM;

@ClassRule
public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource(
new MiniClusterResourceConfiguration.Builder()
.setNumberTaskManagers(NUM_TMS)
.setNumberSlotsPerTaskManager(SLOTS_PER_TM)
.build());

@Test
public void testTaskFailureRecovery() throws Exception {
runTaskFailureRecoveryTest(createjobGraph(false));
}

@Test
public void testTaskFailureWithSlotSharingRecovery() throws Exception {
runTaskFailureRecoveryTest(createjobGraph(true));
}

private void runTaskFailureRecoveryTest(final JobGraph jobGraph) throws Exception {
final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster();

miniCluster.submitJob(jobGraph).get();

final CompletableFuture<JobResult> jobResultFuture = miniCluster.requestJobResult(jobGraph.getJobID());

assertThat(jobResultFuture.get().isSuccess(), is(true));
}

private JobGraph createjobGraph(boolean slotSharingEnabled) throws IOException {
final JobVertex sender = new JobVertex("Sender");
sender.setParallelism(PARALLELISM);
sender.setInvokableClass(Tasks.Sender.class);

final JobVertex receiver = new JobVertex("Receiver");
receiver.setParallelism(PARALLELISM);
receiver.setInvokableClass(FailingOnceReceiver.class);
FailingOnceReceiver.reset();

if (slotSharingEnabled) {
final SlotSharingGroup slotSharingGroup = new SlotSharingGroup();
receiver.setSlotSharingGroup(slotSharingGroup);
sender.setSlotSharingGroup(slotSharingGroup);
}

receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED);

final ExecutionConfig executionConfig = new ExecutionConfig();
executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0L));

final JobGraph jobGraph = new JobGraph(getClass().getSimpleName(), sender, receiver);
jobGraph.setExecutionConfig(executionConfig);

return jobGraph;
}

/**
* Receiver which fails once before successfully completing.
*/
public static final class FailingOnceReceiver extends Tasks.Receiver {

private static volatile boolean failed = false;

public FailingOnceReceiver(Environment environment) {
super(environment);
}

@Override
public void invoke() {
if (!failed && getEnvironment().getTaskInfo().getIndexOfThisSubtask() == 0) {
failed = true;
throw new FlinkRuntimeException(getClass().getSimpleName());
} else {
super.invoke();
}
}

private static void reset() {
failed = false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -151,14 +151,22 @@ public void testReelectionOfJobMaster() throws Exception {

@Test
public void testTaskExecutorsReconnectToClusterWithLeadershipChange() throws Exception {
assertThat(miniCluster.requestClusterOverview().get().getNumTaskManagersConnected(), is(NUM_TMS));
final Deadline deadline = Deadline.fromNow(TESTING_TIMEOUT);
waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline);
highAvailabilityServices.revokeResourceManagerLeadership().get();
highAvailabilityServices.grantResourceManagerLeadership();

// wait for the ResourceManager to confirm the leadership
assertThat(LeaderRetrievalUtils.retrieveLeaderConnectionInfo(highAvailabilityServices.getResourceManagerLeaderRetriever(), Time.minutes(TESTING_TIMEOUT.toMinutes())).getLeaderSessionID(), is(notNullValue()));

CommonTestUtils.waitUntilCondition(() -> miniCluster.requestClusterOverview().get().getNumTaskManagersConnected() == NUM_TMS, Deadline.fromNow(TESTING_TIMEOUT), 10L);
waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline);
}

private void waitUntilTaskExecutorsHaveConnected(int numTaskExecutors, Deadline deadline) throws Exception {
CommonTestUtils.waitUntilCondition(
() -> miniCluster.requestClusterOverview().get().getNumTaskManagersConnected() == numTaskExecutors,
deadline,
10L);
}

private JobGraph createJobGraph(int parallelism) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

package org.apache.flink.runtime.taskexecutor;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.execution.ExecutionState;
Expand All @@ -37,6 +39,7 @@
import org.junit.Before;
import org.junit.Test;

import java.io.IOException;
import java.time.Duration;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
Expand Down Expand Up @@ -78,22 +81,14 @@ public void teardown() throws Exception {
}

/**
* Tests that a job will be re-executed if a new TaskExecutor joins the cluster.
* Tests that a job can be re-executed after the job has failed due
* to a TaskExecutor termination.
*/
@Test
public void testNewTaskExecutorJoinsCluster() throws Exception {
public void testJobReExecutionAfterTaskExecutorTermination() throws Exception {
final JobGraph jobGraph = createJobGraph(PARALLELISM);

miniCluster.submitJob(jobGraph).get();

final CompletableFuture<JobResult> jobResultFuture = miniCluster.requestJobResult(jobGraph.getJobID());

assertThat(jobResultFuture.isDone(), is(false));

CommonTestUtils.waitUntilCondition(
jobIsRunning(() -> miniCluster.getExecutionGraph(jobGraph.getJobID())),
Deadline.fromNow(TESTING_TIMEOUT),
20L);
final CompletableFuture<JobResult> jobResultFuture = submitJobAndWaitUntilRunning(jobGraph);

// kill one TaskExecutor which should fail the job execution
miniCluster.terminateTaskExecutor(0);
Expand All @@ -111,6 +106,40 @@ public void testNewTaskExecutorJoinsCluster() throws Exception {
miniCluster.requestJobResult(jobGraph.getJobID()).get();
}

/**
* Tests that the job can recover from a failing {@link TaskExecutor}.
*/
@Test
public void testJobRecoveryWithFailingTaskExecutor() throws Exception {
final JobGraph jobGraph = createJobGraphWithRestartStrategy(PARALLELISM);

final CompletableFuture<JobResult> jobResultFuture = submitJobAndWaitUntilRunning(jobGraph);

// start an additional TaskExecutor
miniCluster.startTaskExecutor();

miniCluster.terminateTaskExecutor(0).get(); // this should fail the job

BlockingOperator.unblock();

assertThat(jobResultFuture.get().isSuccess(), is(true));
}

private CompletableFuture<JobResult> submitJobAndWaitUntilRunning(JobGraph jobGraph) throws Exception {
miniCluster.submitJob(jobGraph).get();

final CompletableFuture<JobResult> jobResultFuture = miniCluster.requestJobResult(jobGraph.getJobID());

assertThat(jobResultFuture.isDone(), is(false));

CommonTestUtils.waitUntilCondition(
jobIsRunning(() -> miniCluster.getExecutionGraph(jobGraph.getJobID())),
Deadline.fromNow(TESTING_TIMEOUT),
50L);

return jobResultFuture;
}

private SupplierWithException<Boolean, Exception> jobIsRunning(Supplier<CompletableFuture<? extends AccessExecutionGraph>> executionGraphFutureSupplier) {
final Predicate<AccessExecutionGraph> allExecutionsRunning = ExecutionGraphTestUtils.allExecutionsPredicate(ExecutionGraphTestUtils.isInExecutionState(ExecutionState.RUNNING));

Expand All @@ -120,6 +149,15 @@ private SupplierWithException<Boolean, Exception> jobIsRunning(Supplier<Completa
};
}

private JobGraph createJobGraphWithRestartStrategy(int parallelism) throws IOException {
final JobGraph jobGraph = createJobGraph(parallelism);
final ExecutionConfig executionConfig = new ExecutionConfig();
executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0L));
jobGraph.setExecutionConfig(executionConfig);

return jobGraph;
}

private JobGraph createJobGraph(int parallelism) {
final JobVertex vertex = new JobVertex("blocking operator");
vertex.setParallelism(parallelism);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ public static void waitUntilCondition(SupplierWithException<Boolean, Exception>

public static void waitUntilCondition(SupplierWithException<Boolean, Exception> condition, Deadline timeout, long retryIntervalMillis) throws Exception {
while (timeout.hasTimeLeft() && !condition.get()) {
Thread.sleep(Math.min(RETRY_INTERVAL, timeout.timeLeft().toMillis()));
Thread.sleep(Math.min(retryIntervalMillis, timeout.timeLeft().toMillis()));
}

if (!timeout.hasTimeLeft()) {
Expand Down
Loading