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

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

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


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -314,18 +317,29 @@ public void sendRequest(byte[] buffer, int offset, int 
length) throws IOExceptio
     if (this.isTraceEnabled) {
       startTime = System.nanoTime();
     }
-    OutputStream outputStream;
+    OutputStream outputStream = null;
     try {
       try {
         outputStream = this.connection.getOutputStream();
       } catch (IOException e) {
-        // If getOutputStream fails with an exception due to 100-continue
-        // enabled, we return back without throwing an exception.
-        return;
+        // If getOutputStream fails with an exception and 100-continue
+        // is enabled, we return back without throwing an exception
+        // because processResponse will give the correct status code
+        // based on which the retry logic will come into place.
+        String expectHeader = this.connection.getRequestProperty(EXPECT);
+        if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) {
+          return;

Review Comment:
   1. add a log @ debug here, including full stack. ideally, collect some 
iostats on how often it is received so we can understand it more.
   
   2. should we ever expect this if isExpectHeaderEnabled is false? if not, and 
we do get it, then what? same as here?
   
   3. javadocs need updating. sorry



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java:
##########
@@ -30,14 +30,24 @@
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class InvalidAbfsRestOperationException extends 
AbfsRestOperationException {
-  public InvalidAbfsRestOperationException(
-      final Exception innerException) {
-    super(
-        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
-        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
-        innerException != null
-            ? innerException.toString()
-            : "InvalidAbfsRestOperationException",
-        innerException);
-  }
+    public InvalidAbfsRestOperationException(
+            final Exception innerException) {
+        super(
+                AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+                AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+                innerException != null
+                        ? innerException.toString()
+                        : "InvalidAbfsRestOperationException",
+                innerException);
+    }
+
+    public InvalidAbfsRestOperationException(final Exception innerException, 
int retryCount) {
+        super(
+                AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+                AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+                innerException != null
+                        ? innerException.toString()
+                        : "InvalidAbfsRestOperationException" + "RetryCount: " 
+ String.valueOf(retryCount),

Review Comment:
   needs a space. 



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java:
##########
@@ -30,14 +30,24 @@
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class InvalidAbfsRestOperationException extends 
AbfsRestOperationException {
-  public InvalidAbfsRestOperationException(
-      final Exception innerException) {
-    super(
-        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
-        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
-        innerException != null
-            ? innerException.toString()
-            : "InvalidAbfsRestOperationException",
-        innerException);
-  }
+    public InvalidAbfsRestOperationException(
+            final Exception innerException) {
+        super(
+                AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+                AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+                innerException != null
+                        ? innerException.toString()
+                        : "InvalidAbfsRestOperationException",
+                innerException);
+    }
+
+    public InvalidAbfsRestOperationException(final Exception innerException, 
int retryCount) {

Review Comment:
   add some javadoc



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -38,6 +38,7 @@
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.VisibleForTesting;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;

Review Comment:
   put this dowon in the "real" apache imports; things have got a bit messed up 
with the move off guava. putting it below makes cherrypicking a lot easier



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java:
##########
@@ -30,14 +30,24 @@
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class InvalidAbfsRestOperationException extends 
AbfsRestOperationException {
-  public InvalidAbfsRestOperationException(
-      final Exception innerException) {
-    super(
-        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
-        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
-        innerException != null
-            ? innerException.toString()
-            : "InvalidAbfsRestOperationException",
-        innerException);
-  }
+    public InvalidAbfsRestOperationException(
+            final Exception innerException) {
+        super(
+                AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+                AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+                innerException != null
+                        ? innerException.toString()
+                        : "InvalidAbfsRestOperationException",
+                innerException);
+    }
+
+    public InvalidAbfsRestOperationException(final Exception innerException, 
int retryCount) {
+        super(

Review Comment:
   over-indented. use the hadoop 4 space style



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java:
##########
@@ -30,14 +30,24 @@
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class InvalidAbfsRestOperationException extends 
AbfsRestOperationException {
-  public InvalidAbfsRestOperationException(
-      final Exception innerException) {
-    super(
-        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
-        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
-        innerException != null
-            ? innerException.toString()
-            : "InvalidAbfsRestOperationException",
-        innerException);
-  }
+    public InvalidAbfsRestOperationException(

Review Comment:
   the only diff i see in this constructor is indentation. revert.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -314,13 +317,32 @@ 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
-      this.bytesSent = length;
+    OutputStream outputStream = null;
+    try {
+      try {
+        outputStream = this.connection.getOutputStream();
+      } catch (IOException e) {
+        // If getOutputStream fails with an exception and 100-continue
+        // is enabled, we return back without throwing an exception
+        // because processResponse will give the correct status code
+        // based on which the retry logic will come into place.
+        String expectHeader = this.connection.getRequestProperty(EXPECT);
+        if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) {
+          return;
+        } else {
+          throw e;
+        }
+      }
+      // This will normally throw an IOException.

Review Comment:
   explain this comment? normally it should succeed, surely?



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -314,13 +317,32 @@ 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
-      this.bytesSent = length;
+    OutputStream outputStream = null;
+    try {
+      try {
+        outputStream = this.connection.getOutputStream();
+      } catch (IOException e) {
+        // If getOutputStream fails with an exception and 100-continue
+        // is enabled, we return back without throwing an exception
+        // because processResponse will give the correct status code
+        // based on which the retry logic will come into place.
+        String expectHeader = this.connection.getRequestProperty(EXPECT);
+        if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) {
+          return;
+        } else {
+          throw e;
+        }
+      }
+      // This will normally throw an IOException.
       outputStream.write(buffer, offset, length);
     } finally {
+      // Closing the opened output stream
+      if (outputStream != null) {
+        outputStream.close();
+      }
+      // update bytes sent for successful as well as failed attempts via the
+      // accompanying statusCode.
+      this.bytesSent = length;

Review Comment:
   if getOutputStream raises an expect failure, should the values be updated?
   



##########
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md:
##########
@@ -767,9 +767,12 @@ Hflush() being the only documented API that can provide 
persistent data
 transfer, Flush() also attempting to persist buffered data will lead to
 performance issues.
 
+### <a name="100continueconfigoptions"></a> Hundred Continue Options
+`fs.azure.account.expect.header.enabled`: This configuration parameter is used
+to specify whether you wish to send a expect 100 continue header with each
+append request or not. It is configured to true by default.

Review Comment:
   could you add a bit more docs here, as I had to look around to understand 
this.
   
   proposed
   * link to https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Expect
   * statement aboubt when it is used
   * any docs on your store which cover it
   * make clear it is on writing only
   
   ```
   
   This flag configures the client to check with the Azure store before 
uploading a block of data from an output stream. This allows the client to 
throttle back gracefully -before actually attempting to upload the block.
   In experiments this provides significant throughput improvements under heavy 
load.
   
   ```
   



##########
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md:
##########
@@ -767,9 +767,12 @@ Hflush() being the only documented API that can provide 
persistent data
 transfer, Flush() also attempting to persist buffered data will lead to
 performance issues.
 
+### <a name="100continueconfigoptions"></a> Hundred Continue Options
+`fs.azure.account.expect.header.enabled`: This configuration parameter is used
+to specify whether you wish to send a expect 100 continue header with each
+append request or not. It is configured to true by default.
 
 ### <a name="accountlevelthrottlingoptions"></a> Account level throttling 
Options
-

Review Comment:
   reinstate this line; nothing wrong with gap between title and section



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java:
##########
@@ -30,14 +30,24 @@
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class InvalidAbfsRestOperationException extends 
AbfsRestOperationException {
-  public InvalidAbfsRestOperationException(
-      final Exception innerException) {
-    super(
-        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
-        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
-        innerException != null
-            ? innerException.toString()
-            : "InvalidAbfsRestOperationException",
-        innerException);
-  }
+    public InvalidAbfsRestOperationException(
+            final Exception innerException) {
+        super(
+                AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+                AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+                innerException != null
+                        ? innerException.toString()
+                        : "InvalidAbfsRestOperationException",

Review Comment:
   make this a constant so it will be used consistently across contructors



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java:
##########
@@ -103,6 +103,9 @@ public final class AbfsHttpConstants {
   public static final String DEFAULT_SCOPE = "default:";
   public static final String PERMISSION_FORMAT = "%04d";
   public static final String SUPER_USER = "$superuser";
+  //The HTTP 100 Continue informational status response code indicates that 
everything so far

Review Comment:
   nit: a space between the // and the first word is generally expected.





> 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