[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2023-03-16 Thread via GitHub


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


##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java:
##
@@ -25,6 +25,7 @@
 import java.net.UnknownHostException;
 import java.util.List;
 
+import org.apache.hadoop.classification.VisibleForTesting;

Review Comment:
   that should go into the org.apache. import block. a lot of existing 
VisibleForTesting imports are in the "other" block, but that's from when we 
moved off guava by patching the source



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2023-03-16 Thread via GitHub


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


##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##
@@ -702,7 +707,7 @@ public AbfsRestOperation append(final String path, final 
byte[] buffer,
   */
   int responseStatusCode = ((AbfsRestOperationException) 
e).getStatusCode();
   if (checkUserError(responseStatusCode) && 
reqParams.isExpectHeaderEnabled()) {
-LOG.debug("User error, retrying without 100 continue enabled");
+LOG.debug("User error, retrying without 100 continue enabled for the 
given path " + path);

Review Comment:
   nit; use {} in the string, path as an argument
   * skips string concat when logging doesn't take place
   * if path is null, triggers error on toString(), logging recovers



##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java:
##
@@ -160,7 +160,7 @@ public void updateMetrics(AbfsRestOperationType 
operationType,
 throttling but there were some expectedBytesToBeSent.
*/
   if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) 
{
-LOG.debug("Updating metrics due to throttling");
+LOG.debug("Updating metrics due to throttling for path " + 
abfsHttpOperation.getConnUrl().getPath());

Review Comment:
   same thing about {}



##
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java:
##
@@ -579,8 +577,6 @@ public void testExpectHundredContinue() throws Exception {
 .isEqualTo(0);
 
 // Verify that the same request was retried with expect header disabled.
-Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled())
-.describedAs("The expect header is not false")
-.isEqualTo(false);
+assertFalse(appendRequestParameters.isExpectHeaderEnabled());

Review Comment:
   sorry, i must have explained it wrong. AssertJ has assert/true false too in 
its type `AbstractBooleanAssert`. you can go
   ```
   Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled())
 .describedAs("The expect header is not false")
 .isFalse()
   ```
   so can you restore what was cut and just change the assert.
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2023-03-15 Thread via GitHub


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


##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##
@@ -681,35 +685,38 @@ 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,
+ the response message might not get set correctly by the server.
+ So, 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.isExpectHeaderEnabled()) {
+LOG.debug("User error, retrying without 100 continue enabled");

Review Comment:
   is there anything which can be added here to help debug problems, e.g path, 
request?
   I'm thinking of the problem "20 GB of log and you want to know why one file 
is having problems"



##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##
@@ -681,35 +685,38 @@ 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,
+ the response message might not get set correctly by the server.
+ So, 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.isExpectHeaderEnabled()) {
+LOG.debug("User error, retrying without 100 continue enabled");
+reqParams.setExpectHeaderEnabled(false);
+return this.append(path, buffer, reqParams, cachedSasToken,
+tracingContext);
+  }
   // If we have no HTTP response, throw the original exception.
   if (!op.hasResult()) {
 throw e;
   }
   if (reqParams.isAppendBlob()
   && appendSuccessCheckOp(op, path,
   (reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
-final AbfsRestOperation successOp = new AbfsRestOperation(
-AbfsRestOperationType.Append,
-this,
-HTTP_METHOD_PUT,
-url,
-requestHeaders,
-buffer,
-reqParams.getoffset(),
-reqParams.getLength(),
+final AbfsRestOperation successOp = getAbfsRestOperationForAppend(
+AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, requestHeaders,

Review Comment:
   again, spread across lines



##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java:
##
@@ -720,6 +727,48 @@ && appendSuccessCheckOp(op, path,
 return op;
   }
 
+  /**
+   * Returns the rest operation for append

Review Comment:
   needs a . 

[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2022-12-12 Thread GitBox


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/except

[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2022-11-30 Thread GitBox


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


##
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##
@@ -314,18 +314,21 @@ 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;
+try {
+  try {
+outputStream = this.connection.getOutputStream();
+  } catch (IOException e) {
+// If getOutputStream fails with an exception due to 100-continue

Review Comment:
   i still dont like this.
   
   there are a lot of reasons for failure, given this is the call which tries 
to set up the entire http request. if it is only expect 100 errors then
   
   1. handler needs to identify those and throw everything else
   2. you need a strategy for downgrading
   as it is, it looks to me that if you have any problem talking to the far end 
then the method will silently swallow it, while updating the bytes sent counter



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2022-11-23 Thread GitBox


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


##
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:
   and it will double count if write() raises an IOE



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2022-11-22 Thread GitBox


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] steveloughran commented on a diff in pull request #4039: HADOOP-18146: ABFS: Added changes for expect hundred continue header

2022-10-26 Thread GitBox


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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

For q