Skip to content

[FLINK-37730][rest] Add client method for getting JM exception history #26525

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

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
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
Expand Up @@ -69,6 +69,8 @@
import org.apache.flink.runtime.rest.messages.JobClientHeartbeatHeaders;
import org.apache.flink.runtime.rest.messages.JobClientHeartbeatParameters;
import org.apache.flink.runtime.rest.messages.JobClientHeartbeatRequestBody;
import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders;
import org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory;
import org.apache.flink.runtime.rest.messages.JobMessageParameters;
import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
Expand All @@ -92,6 +94,7 @@
import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetListHeaders;
import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders;
import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters;
import org.apache.flink.runtime.rest.messages.job.JobExecutionResultHeaders;
import org.apache.flink.runtime.rest.messages.job.JobResourceRequirementsBody;
import org.apache.flink.runtime.rest.messages.job.JobResourcesRequirementsUpdateHeaders;
Expand Down Expand Up @@ -330,6 +333,20 @@ public CompletableFuture<JobDetailsInfo> getJobDetails(JobID jobId) {
return sendRequest(detailsHeaders, params);
}

/**
* Requests the job exception history.
*
* @param jobID The job id
* @return Job exceptions
*/
public CompletableFuture<JobExceptionsInfoWithHistory> getJobExceptions(JobID jobID) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am curious what happens if there are loops / repeated exceptions - will this cause an issue with the rest call. Is there a case to have a paging API here to avoid large responses?

final JobExceptionsHeaders jobExceptionsHeaders = JobExceptionsHeaders.getInstance();
final JobExceptionsMessageParameters params = new JobExceptionsMessageParameters();
params.jobPathParameter.resolve(jobID);

return sendRequest(jobExceptionsHeaders, params);
}

@Override
public CompletableFuture<JobStatus> getJobStatus(JobID jobId) {
final CheckedSupplier<CompletableFuture<JobStatus>> operation =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,8 @@
import org.apache.flink.runtime.rest.messages.JobAccumulatorsMessageParameters;
import org.apache.flink.runtime.rest.messages.JobCancellationHeaders;
import org.apache.flink.runtime.rest.messages.JobCancellationMessageParameters;
import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders;
import org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory;
import org.apache.flink.runtime.rest.messages.JobMessageParameters;
import org.apache.flink.runtime.rest.messages.JobPlanInfo;
import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders;
Expand All @@ -92,6 +94,7 @@
import org.apache.flink.runtime.rest.messages.dataset.ClusterDataSetListResponseBody;
import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders;
import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo;
import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters;
import org.apache.flink.runtime.rest.messages.job.JobExecutionResultHeaders;
import org.apache.flink.runtime.rest.messages.job.JobExecutionResultResponseBody;
import org.apache.flink.runtime.rest.messages.job.JobStatusInfoHeaders;
Expand Down Expand Up @@ -1298,6 +1301,35 @@ void testJobDetailsContainsSlotSharingGroupId() throws Exception {
}
}

@Test
void testGetJobExceptionsInfoWithHistory() throws Exception {

final TestJobExceptionsHandler jobExceptionsHandler = new TestJobExceptionsHandler();
TestRestServerEndpoint restServerEndpoint = createRestServerEndpoint(jobExceptionsHandler);
final RestClusterClient<?> restClusterClient =
createRestClusterClient(restServerEndpoint.getServerAddress().getPort());
try {
CompletableFuture<JobExceptionsInfoWithHistory> future =
restClusterClient.getJobExceptions(jobId);
JobExceptionsInfoWithHistory result = future.get();
assertThat(result.getExceptionHistory()).isNotNull();
assertThat(result.getExceptionHistory().getEntries()).hasSize(1);
JobExceptionsInfoWithHistory.RootExceptionInfo rootExceptionInfo =
result.getExceptionHistory().getEntries().get(0);
assertThat(rootExceptionInfo.getExceptionName()).isEqualTo("TestRootException");
assertThat(rootExceptionInfo.getStacktrace()).contains("Simulated failure");
assertThat(rootExceptionInfo.getConcurrentExceptions()).hasSize(1);
JobExceptionsInfoWithHistory.ExceptionInfo concurrent =
rootExceptionInfo.getConcurrentExceptions().iterator().next();
assertThat(concurrent.getExceptionName()).isEqualTo("TestException");
assertThat(concurrent.getStacktrace()).contains("Simulated failure");
assertThat(result.getExceptionHistory().isTruncated()).isFalse();
} finally {
restClusterClient.close();
restServerEndpoint.close();
}
}

private class TestClientCoordinationHandler
extends TestHandler<
ClientCoordinationRequestBody,
Expand Down Expand Up @@ -1464,6 +1496,55 @@ protected CompletableFuture<JobStatusInfo> handleRequest(
}
}

private class TestJobExceptionsHandler
extends TestHandler<
EmptyRequestBody,
JobExceptionsInfoWithHistory,
JobExceptionsMessageParameters> {

private TestJobExceptionsHandler() {
super(JobExceptionsHeaders.getInstance());
}

@Override
protected CompletableFuture<JobExceptionsInfoWithHistory> handleRequest(
@Nonnull HandlerRequest<EmptyRequestBody> request,
@Nonnull DispatcherGateway gateway)
throws RestHandlerException {
JobExceptionsInfoWithHistory.ExceptionInfo exceptionInfo =
new JobExceptionsInfoWithHistory.ExceptionInfo(
"TestException",
"java.lang.RuntimeException: Simulated failure\n\tat"
+ " org.test.FakeClass.method(FakeClass.java:42)",
12345L);

Collection<JobExceptionsInfoWithHistory.ExceptionInfo> concurrentExceptions =
Collections.singletonList(exceptionInfo);

Map<String, String> failureLabels = new HashMap<>();
failureLabels.put("flink.operator.failure.label", "simulated");

JobExceptionsInfoWithHistory.RootExceptionInfo rootExceptionInfo =
new JobExceptionsInfoWithHistory.RootExceptionInfo(
"TestRootException",
"java.lang.Exception: Test stack trace\n\tat"
+ " org.test.FakeClass.method(FakeClass.java:123)",
12345L,
failureLabels,
concurrentExceptions);

JobExceptionsInfoWithHistory.JobExceptionHistory exceptionHistory =
new JobExceptionsInfoWithHistory.JobExceptionHistory(
Collections.singletonList(rootExceptionInfo), false // Truncated?
);

JobExceptionsInfoWithHistory jobExceptionsInfoWithHistory =
new JobExceptionsInfoWithHistory(exceptionHistory);

return CompletableFuture.completedFuture(jobExceptionsInfoWithHistory);
}
}

private class TestJobDetailsInfoHandler
extends TestHandler<EmptyRequestBody, JobDetailsInfo, JobMessageParameters> {

Expand Down