-
Notifications
You must be signed in to change notification settings - Fork 9k
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
HADOOP-19531. [ABFS][FnsOverBlob] Streaming List Path Result Should Happen Inside Retry Loop #7582
Conversation
This comment was marked as outdated.
This comment was marked as outdated.
LOG.error("Unable to deserialize list results for Uri {}", uri.toString(), ex); | ||
throw new AbfsDriverException(ERR_DFS_LIST_PARSING, ex); | ||
LOG.error("Unable to deserialize list results for Uri {}", uri != null ? uri.toString(): "NULL", ex); | ||
throw new AbfsDriverException(ex); |
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.
throw new AbfsDriverException(ERR_DFS_LIST_PARSING, ex); add the message here as well
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.
Taken
throw new AbfsDriverException(ERR_BLOB_LIST_PARSING, e); | ||
} catch (AbfsDriverException ex) { | ||
// Throw as it is to avoid multiple wrapping. | ||
LOG.error("Unable to deserialize list results for Uri {}", uri != null ? uri.toString(): "NULL", ex); |
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.
uri should not be null if we are reaching till this part of the code
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.
uri can be null for the cases where this API is called by internal operations like in create flow.
If list call is coming from user, uri won't be null.
@@ -500,6 +505,20 @@ private void parseBlockListResponse(final InputStream stream) throws IOException | |||
blockIdList = client.parseBlockListResponse(stream); | |||
} | |||
|
|||
private void parseListPathResponse(final InputStream stream) throws IOException { |
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.
add javadocs
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.
Taken
This comment was marked as outdated.
This comment was marked as outdated.
@@ -500,6 +505,20 @@ private void parseBlockListResponse(final InputStream stream) throws IOException | |||
blockIdList = client.parseBlockListResponse(stream); | |||
} | |||
|
|||
private void parseListPathResponse(final InputStream stream) throws IOException { | |||
if (stream == null || blockIdList != null) { |
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.
how is blockIdList check relevant here ?
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.
Was buggy, fixed it
This comment was marked as outdated.
This comment was marked as outdated.
This comment was marked as outdated.
This comment was marked as outdated.
throw new AbfsDriverException(e); | ||
return filterDuplicateEntriesAndRenamePendingFiles(listResultSchema, uri); | ||
} catch (SAXException | IOException ex) { | ||
throw new AbfsDriverException(ERR_BLOB_LIST_PARSING, ex); |
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: We can add URI in the error message for better visibility.
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.
URI is already in log messages.
if (stream == null) { | ||
return null; | ||
} | ||
try (InputStream stream = new ByteArrayInputStream(result.getListResultData())) { |
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.
Will there be any issue here in case result.getListResultData() is null?
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.
Yes, it can lead to NPE. Have modified to catch all the exceptions and wrap it around AbfsDriverException and throw back to user
@@ -438,6 +438,11 @@ final void parseResponse(final byte[] buffer, | |||
method, getMaskedUrl(), ex.getMessage()); | |||
log.debug("IO Error: ", ex); | |||
throw ex; | |||
} catch (Exception ex) { | |||
log.warn("Unexpected 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.
Why do we need both log.warn and log.debug here?
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.
We were doing same for other type of exceptions.
@@ -67,6 +68,7 @@ public ITestApacheClientConnectionPool() throws Exception { | |||
public void testKacIsClosed() throws Throwable { | |||
Configuration configuration = new Configuration(getRawConfiguration()); | |||
configuration.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.name()); | |||
configuration.unset(FS_AZURE_METRIC_FORMAT); |
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.
why do we need to unset this config here?
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.
This test was failing when this config was present in test xml files
This comment was marked as outdated.
This comment was marked as outdated.
Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); | ||
Mockito.doReturn(spiedClient).when(spiedStore).getClient(); | ||
|
||
Mockito.doThrow(new SocketException(CONNECTION_RESET_MESSAGE)).when(spiedClient).filterDuplicateEntriesAndRenamePendingFiles(any(), any()); |
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 we reduce the retry count and verify that request was retried ?
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.
This particular test is to make sure retry does not happen as here exception is injected in parsing list result from local stream.
I have modified the test testListPathTracingContext
to assert that exception thrown within httpOperation.processResponse() is retried.
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.
+1 LGTM
This comment was marked as outdated.
This comment was marked as outdated.
This comment was marked as outdated.
This comment was marked as outdated.
💔 -1 overall
This message was automatically generated. |
============================================================
|
🎊 +1 overall
This message was automatically generated. |
…appen Inside Retry Loop (apache#7582) Contributed by Anuj Modi Reviewed by Anmol Asrani, Manish Bhatt, Manika Joshi Signed off by: Anuj Modi<[email protected]>
…appen Inside Retry Loop (#7582) (#7585) Contributed by Anuj Modi Reviewed by Anmol Asrani, Manish Bhatt, Manika Joshi Signed off by: Anuj Modi<[email protected]>
Description of PR
Listing APIs on both DFS and Blob Endpoints return response as part of response body and has to be read from an Socket Input Stream.
Any network error occuring while reading the stream should be retried.
Today, this parsing happens in client and such errors are not retried.
This change fixes this behavior
How was this patch tested?
Added a few more tests and test suite ran for validation