[
https://issues.apache.org/jira/browse/HADOOP-18146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17651946#comment-17651946
]
ASF GitHub Bot commented on HADOOP-18146:
-----------------------------------------
pranavsaxena-microsoft commented on code in PR #4039:
URL: https://github.com/apache/hadoop/pull/4039#discussion_r1057102194
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -314,13 +327,46 @@ public void sendRequest(byte[] buffer, int offset, int
length) throws IOExceptio
if (this.isTraceEnabled) {
startTime = System.nanoTime();
}
- try (OutputStream outputStream = this.connection.getOutputStream()) {
- // update bytes sent before they are sent so we may observe
- // attempted sends as well as successful sends via the
- // accompanying statusCode
+ OutputStream outputStream = null;
+ // Updates the expected bytes to be sent based on length.
+ this.expectedBytesToBeSent = length;
+ try {
+ try {
+ /* Without expect header enabled, if getOutputStream() throws
+ an exception, it gets caught by the restOperation. But with
+ expect header enabled we return back without throwing an exception
+ for the correct response code processing.
+ */
+ outputStream = this.connection.getOutputStream();
+ } catch (IOException e) {
+ /* If getOutputStream fails with an exception and expect header
+ is enabled, we return back without throwing an exception to
+ the caller. The caller is responsible for setting the correct
status code.
+ If expect header is not enabled, we throw back the exception.
+ */
+ String expectHeader = this.connection.getRequestProperty(EXPECT);
+ if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) {
+ LOG.debug("Getting output stream failed with expect header enabled,
returning back "
+ + ExceptionUtils.getStackTrace(e));
+ return;
Review Comment:
Are you saying that server in this case would send some status with >400,
and that would be caught in
https://github.com/anmolanmol1234/hadoop/blob/HADOOP-18146/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java#L412-L416?
If yes, what if getOutputStream fails due to openSSL handshake like:
```
java.net.SocketException: Connection reset
at java.net.SocketInputStream.read(SocketInputStream.java:210)
at java.net.SocketInputStream.read(SocketInputStream.java:141)
at
org.wildfly.openssl.OpenSSLSocket.runHandshake(OpenSSLSocket.java:306)
at
org.wildfly.openssl.OpenSSLSocket.startHandshake(OpenSSLSocket.java:210)
at
sun.net.www.protocol.https.HttpsClient.afterConnect(HttpsClient.java:559)
at
sun.net.www.protocol.https.AbstractDelegateHttpsURLConnection.connect(AbstractDelegateHttpsURLConnection.java:197)
at
sun.net.www.protocol.http.HttpURLConnection.getOutputStream0(HttpURLConnection.java:1342)
at
sun.net.www.protocol.http.HttpURLConnection.getOutputStream(HttpURLConnection.java:1317)
at
sun.net.www.protocol.https.HttpsURLConnectionImpl.getOutputStream(HttpsURLConnectionImpl.java:264)
at
org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation.sendRequest(AbfsHttpOperation.java:300)
```
Also, it would be stretch of logic accrosss multiple places: in
sendRequest(), processresponse(), abfsclient.append(). It would be awesome if
we flow an exception from this point to abfsclient. What you feel?
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -681,35 +685,36 @@ public AbfsRestOperation append(final String path, final
byte[] buffer,
abfsUriQueryBuilder, cachedSasToken);
final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.Append,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders,
- buffer,
- reqParams.getoffset(),
- reqParams.getLength(),
- sasTokenForReuse);
+ final AbfsRestOperation op =
getAbfsRestOperationForAppend(AbfsRestOperationType.Append,
+ HTTP_METHOD_PUT, url, requestHeaders, buffer, reqParams.getoffset(),
+ reqParams.getLength(), sasTokenForReuse);
try {
op.execute(tracingContext);
} catch (AzureBlobFileSystemException e) {
+ // If the http response code indicates a user error we retry
+ // the same append request with expect header disabled.
+ // When "100-continue" header is enabled but a non Http 100 response
comes,
+ // JDK fails to provide all response headers.
Review Comment:
From code I can comprehend:
```
// If the http response code indicates a user error we retry
// the same append request with expect header disabled.
```
but I am not able to understand:
```
// When "100-continue" header is enabled but a non Http 100 response
comes,
// JDK fails to provide all response headers.
// This handling is to avoid breaking of backward compatibility
// if someone has taken dependency on the exception message,
// which is created using the error string present in the response
header.
```
> ABFS: Add changes for expect hundred continue header with append requests
> -------------------------------------------------------------------------
>
> Key: HADOOP-18146
> URL: https://issues.apache.org/jira/browse/HADOOP-18146
> Project: Hadoop Common
> Issue Type: Sub-task
> Affects Versions: 3.3.1
> Reporter: Anmol Asrani
> Assignee: Anmol Asrani
> Priority: Major
> Labels: pull-request-available
> Time Spent: 2h 10m
> Remaining Estimate: 0h
>
> Heavy load from a Hadoop cluster lead to high resource utilization at FE
> nodes. Investigations from the server side indicate payload buffering at
> Http.Sys as the cause. Payload of requests that eventually fail due to
> throttling limits are also getting buffered, as its triggered before FE could
> start request processing.
> Approach: Client sends Append Http request with Expect header, but holds back
> on payload transmission until server replies back with HTTP 100. We add this
> header for all append requests so as to reduce.
> We made several workload runs with and without hundred continue enabled and
> the overall observation is that :-
> # The ratio of TCP SYN packet count with and without expect hundred continue
> enabled is 0.32 : 3 on average.
> # The ingress into the machine at TCP level is almost 3 times lesser with
> hundred continue enabled which implies a lot of bandwidth save.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]