-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-7720] Fix the exception type of InMemoryJobService when job id not found #9347
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
Conversation
Everywhere that we call getInvocation() we should handle the possible StatusException
|
R: @lukecwik |
|
Original discussion about this problem is here: #8977 (comment) |
lukecwik
left a comment
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.
Can you add the try/catch StatusRuntimeException/StatusException to all the other service calls?
| GetJobStateResponse response = GetJobStateResponse.newBuilder().setState(state).build(); | ||
| responseObserver.onNext(response); | ||
| responseObserver.onCompleted(); | ||
| } catch (StatusException e) { |
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.
| } catch (StatusException e) { | |
| } catch (StatusRuntimeException | StatusException e) { |
Here and everywhere else otherwise the catch Exception will also convert the StatusRuntimeException to an INTERNAL status error.
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.
I updated StatusException to StatusRuntimeException | StatusException everywhere, but I'm a bit unclear on what's actually throwing the StatusRuntimeException. I searched the code for StatusRuntimeException and I did not see a clear path from any of those cases to here.
For example, looking at this code I didn't see anything suspicious inside the try/catch block:
@Override
public void getState(
GetJobStateRequest request, StreamObserver<GetJobStateResponse> responseObserver) {
LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
String invocationId = request.getJobId();
try {
JobInvocation invocation = getInvocation(invocationId);
JobState.Enum state = invocation.getState();
GetJobStateResponse response = GetJobStateResponse.newBuilder().setState(state).build();
responseObserver.onNext(response);
responseObserver.onCompleted();
} catch (StatusRuntimeException | StatusException e) {
responseObserver.onError(e);
} catch (Exception e) {
String errMessage =
String.format("Encountered Unexpected Exception for Invocation %s", invocationId);
LOG.error(errMessage, e);
responseObserver.onError(Status.INTERNAL.withCause(e).asException());
}
}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.
Since they are runtime exceptions, nothing is required to declare that they are thrown in their method signatures and can propagate up the call stack from any arbitrary location until they hit an appropriately scoped catch statement.
Without the change to also catch the StatusRuntimeException as part of the StatusException, they would be caught as part of the Exception block and converted to INTERNAL errors.
Everywhere that we call getInvocation() we should handle the possible StatusException
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username).[BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replaceBEAM-XXXwith the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.Post-Commit Tests Status (on master branch)
Pre-Commit Tests Status (on master branch)
See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.