[ 
https://issues.apache.org/jira/browse/HADOOP-18146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17637418#comment-17637418
 ] 

ASF GitHub Bot commented on HADOOP-18146:
-----------------------------------------

steveloughran commented on code in PR #4039:
URL: https://github.com/apache/hadoop/pull/4039#discussion_r1029705420


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java:
##########
@@ -72,4 +75,12 @@ public boolean isAppendBlob() {
   public String getLeaseId() {
     return this.leaseId;
   }
+
+  public boolean getIsExpectHeaderEnabled() {

Review Comment:
   change to `isExpectHeaderEnabled()` for consistency with the 
AbfsOutputStreamContext property



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -688,6 +692,19 @@ public AbfsRestOperation append(final String path, final 
byte[] buffer,
     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.
+      // 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.
+      int responseStatusCode = ((AbfsRestOperationException) 
e).getStatusCode();
+      if (checkUserError(responseStatusCode) && 
reqParams.getIsExpectHeaderEnabled()) {
+        reqParams.setExpectHeaderEnabled(false);

Review Comment:
   how about logging this at debug so if it happens, there's more of a history 
people can look at



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -320,6 +320,11 @@ public void sendRequest(byte[] buffer, int offset, int 
length) throws IOExceptio
       // accompanying statusCode
       this.bytesSent = length;
       outputStream.write(buffer, offset, length);
+    } catch (IOException e) {
+      // If getOutputStream fails with an exception due to 100-continue
+      // enabled, we update the bytes sent before they are sent
+      // in the catch block.
+      this.bytesSent = length;

Review Comment:
   this is going to swallow all IOEs raised. unless i've misreasd something, 
the IOE *must* be rethrown





> 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]

Reply via email to