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

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

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


##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java:
##########
@@ -22,10 +22,14 @@
 import static 
org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL;
 import static 
org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES;
 import static 
org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL;
+
 import java.util.Random;
+
 import org.junit.Assert;
 import org.junit.Test;
+
 import org.apache.hadoop.conf.Configuration;
+

Review Comment:
   not needed. i'm generally reluctant for any cleanup of imports as they make 
backporting hard, but this is a low-rate-of-change file so i'm happy with the 
other linebreaks



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -652,6 +653,9 @@ public AbfsRestOperation append(final String path, final 
byte[] buffer,
     addCustomerProvidedKeyHeaders(requestHeaders);
     // JDK7 does not support PATCH, so to workaround the issue we will use

Review Comment:
   is this comment still valid? or can the verb be used directly? and would it 
make a difference?
   even if is valid, the new line inserted breaks the comment. keep them 
together



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##########
@@ -730,6 +729,18 @@ && appendSuccessCheckOp(op, path,
     return op;
   }
 
+  /**
+   * Returns true if the status code lies in the range of user error
+   * @param e Exception caught
+   * @return True or False
+   */
+  private boolean checkUserError(AzureBlobFileSystemException e) {
+    return ((AbfsRestOperationException) e).getStatusCode()

Review Comment:
   get the status code into a variable, then the two comparisons can be on 
single lines rather than split



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java:
##########
@@ -32,7 +32,7 @@
 public final class FileSystemConfigurations {
 
   public static final String DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "";
-
+  public static final boolean 
DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = true;

Review Comment:
   why is this the default? isn't it going to add overhead even in light load 
conditions?



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java:
##########
@@ -34,19 +34,21 @@ public enum Mode {
   private final Mode mode;
   private final boolean isAppendBlob;
   private final String leaseId;
+  private boolean isExpectHeaderEnabled;
 
   public AppendRequestParameters(final long position,
       final int offset,
       final int length,
       final Mode mode,
       final boolean isAppendBlob,
-      final String leaseId) {
+      final String leaseId, boolean isExpectHeaderEnabled) {

Review Comment:
   and make final



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java:
##########
@@ -60,6 +60,7 @@ public final class HttpHeaderConfigurations {
   public static final String X_MS_UMASK = "x-ms-umask";
   public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled";
   public static final String X_MS_ABFS_CLIENT_LATENCY = 
"x-ms-abfs-client-latency";
+  public static final String EXPECT = "Expect";

Review Comment:
   add at the end unless there's a good reason for putting it here



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java:
##########
@@ -103,6 +103,7 @@ 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";
+  public static final String HUNDRED_CONTINUE = "100-continue";

Review Comment:
   explain what this is in a javadoc



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java:
##########
@@ -694,6 +694,7 @@ private AbfsOutputStreamContext 
populateAbfsOutputStreamContext(
     return new 
AbfsOutputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds())
             .withWriteBufferSize(bufferSize)
             .enableFlush(abfsConfiguration.isFlushEnabled())
+            .enableExpectHeader(abfsConfiguration.isExpectHeaderEnabled())

Review Comment:
   move up one line for alphabetical ordering of the enable* methods





> 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