Skip to content
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-28617][SQL Gateway] Support stop job statement in SqlGatewayService #21292

Merged
merged 10 commits into from
Dec 18, 2022
Prev Previous commit
[FLINK-28617][SQL Gateway] Wait for all tasks running in the test
  • Loading branch information
link3280 committed Dec 17, 2022
commit bf18aca792b911a888b01b75d6f9123e3e0dfc70
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.gateway.service;

import org.apache.flink.api.common.JobID;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -431,7 +432,7 @@ public void testStopJobStatementWithSavepoint(
assertThat(results.size()).isEqualTo(1);
String jobId = results.get(0).getString(0).toString();

TestUtils.waitUntilJobIsRunning(restClusterClient);
TestUtils.waitUntilAllTasksAreRunning(restClusterClient, JobID.fromHexString(jobId));

String stopSql = String.format(stopSqlTemplate, jobId, option);
OperationHandle stopOperationHandle =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,19 @@
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.time.Deadline;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.runtime.checkpoint.Checkpoints;
import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata;
import org.apache.flink.runtime.client.JobStatusMessage;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.minicluster.MiniCluster;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.JobMessageParameters;
import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders;
import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess;
import org.apache.flink.runtime.testutils.CommonTestUtils;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.util.ExceptionUtils;
Expand All @@ -42,13 +47,10 @@
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;

import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.CHECKPOINT_DIR_PREFIX;
import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.METADATA_FILE_NAME;
Expand Down Expand Up @@ -188,22 +190,38 @@ public static void waitUntilJobCanceled(JobID jobId, ClusterClient<?> client)
}

/**
* Wait util at least one job turns into RUNNING status in the cluster. Applicable for single
* job scenarios.
* Wait util all task of a job turns into RUNNING state.
*
* @param client ClusterClient which could be {@link
* @param restClusterClient RestClusterClient which could be {@link
* org.apache.flink.test.junit5.InjectClusterClient}.
*/
public static void waitUntilJobIsRunning(ClusterClient<?> client) throws Exception {
List<JobID> runningJobs = new ArrayList<>();
while (runningJobs.isEmpty()) {
Thread.sleep(50);
Collection<JobStatusMessage> statusMessages = client.listJobs().get();
runningJobs =
statusMessages.stream()
.filter(status -> status.getJobState().equals(JobStatus.RUNNING))
.map(JobStatusMessage::getJobId)
.collect(Collectors.toList());
}
public static void waitUntilAllTasksAreRunning(
RestClusterClient<?> restClusterClient, JobID jobId) throws Exception {
// access the REST endpoint of the cluster to determine the state of each ExecutionVertex
final JobDetailsHeaders detailsHeaders = JobDetailsHeaders.getInstance();
final JobMessageParameters params = detailsHeaders.getUnresolvedMessageParameters();
params.jobPathParameter.resolve(jobId);

CommonTestUtils.waitUntilCondition(
() ->
restClusterClient
.sendRequest(detailsHeaders, params, EmptyRequestBody.getInstance())
.thenApply(
detailsInfo ->
allVerticesRunning(
detailsInfo.getJobVerticesPerState()))
.get());
}

private static boolean allVerticesRunning(Map<ExecutionState, Integer> states) {
return states.entrySet().stream()
.allMatch(
entry -> {
if (entry.getKey() == ExecutionState.RUNNING) {
return entry.getValue() > 0;
} else {
return entry.getValue() == 0; // no vertices in non-running state.
}
});
}
}