[ 
https://issues.apache.org/jira/browse/HADOOP-17290?focusedWorklogId=616211&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-616211
 ]

ASF GitHub Bot logged work on HADOOP-17290:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 29/Jun/21 13:32
            Start Date: 29/Jun/21 13:32
    Worklog Time Spent: 10m 
      Work Description: sumangala-patki commented on a change in pull request 
#2520:
URL: https://github.com/apache/hadoop/pull/2520#discussion_r660083909



##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java
##########
@@ -114,13 +119,15 @@ public AbfsLease(AbfsClient client, String path, int 
acquireMaxRetries,
     LOG.debug("Acquired lease {} on {}", leaseID, path);
   }
 
-  private void acquireLease(RetryPolicy retryPolicy, int numRetries, int 
retryInterval, long delay)
+  private void acquireLease(RetryPolicy retryPolicy, int numRetries,
+      int retryInterval, long delay)
       throws LeaseException {
     LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries);
     if (future != null && !future.isDone()) {
       throw new LeaseException(ERR_LEASE_FUTURE_EXISTS);
     }
-    future = client.schedule(() -> client.acquireLease(path, 
INFINITE_LEASE_DURATION),
+    future = client.schedule(() -> client.acquireLease(path,
+        INFINITE_LEASE_DURATION, new TracingContext(tracingContext)),

Review comment:
       not needed; moved to calling method (constructor) where it is cloned 
only once, instead of per retry

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
##########
@@ -160,6 +170,14 @@ public AbfsOutputStream(
     if (outputStreamStatistics != null) {
       this.ioStatistics = outputStreamStatistics.getIOStatistics();
     }
+    this.outputStreamId = getOutputStreamId();
+    this.tracingContext = new TracingContext(tracingContext);
+    this.tracingContext.setStreamID(outputStreamId);
+    this.tracingContext.setOperation(FSOperationType.WRITE);
+  }
+
+  private String getOutputStreamId() {

Review comment:
       done

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
##########
@@ -385,7 +412,9 @@ private void writeAppendBlobCurrentBufferToService() throws 
IOException {
             "writeCurrentBufferToService", "append")) {
       AppendRequestParameters reqParams = new AppendRequestParameters(offset, 
0,
           bytesLength, APPEND_MODE, true, leaseId);
-      AbfsRestOperation op = client.append(path, bytes, reqParams, 
cachedSasToken.get());
+      AbfsRestOperation op = client
+          .append(path, bytes, reqParams, cachedSasToken.get(),
+              new TracingContext(tracingContext));

Review comment:
       yes

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
##########
@@ -221,17 +226,27 @@ private void completeExecute() throws 
AzureBlobFileSystemException {
     LOG.trace("{} REST operation complete", operationType);
   }
 
+  private void updateClientRequestHeader(AbfsHttpOperation httpOperation,
+      TracingContext tracingContext) {
+    tracingContext.generateClientRequestId();
+    httpOperation.getConnection()
+        .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID,
+            tracingContext.constructHeader());

Review comment:
       - constructHeader() may be called simply to display IDs, in which case 
it should not regenerate new clientReqId each time
   - callee setting it would mean adding code to all methods to call this, and 
retry case would not be handled
   Can be done internally within TC: have a generateClientRequestId function to 
create guid, and call it in constructor as well as the setRetryCount which is 
invoked per retry. However, had avoided this since we are populating ID 
variables only as and when they pass through the corresponding ABFS layers. 
   

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
##########
@@ -221,17 +226,27 @@ private void completeExecute() throws 
AzureBlobFileSystemException {
     LOG.trace("{} REST operation complete", operationType);
   }
 
+  private void updateClientRequestHeader(AbfsHttpOperation httpOperation,
+      TracingContext tracingContext) {
+    tracingContext.generateClientRequestId();
+    httpOperation.getConnection()
+        .setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID,
+            tracingContext.constructHeader());
+  }
+
   /**
    * Executes a single HTTP operation to complete the REST operation.  If it
    * fails, there may be a retry.  The retryCount is incremented with each
    * attempt.
    */
-  private boolean executeHttpOperation(final int retryCount) throws 
AzureBlobFileSystemException {
+  private boolean executeHttpOperation(final int retryCount,
+    TracingContext tracingContext) throws AzureBlobFileSystemException {
     AbfsHttpOperation httpOperation = null;
     try {
       // initialize the HTTP request and open the connection
       httpOperation = new AbfsHttpOperation(url, method, requestHeaders);
       incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1);
+      updateClientRequestHeader(httpOperation, tracingContext);

Review comment:
       done

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java
##########
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+
+/**
+ * Interface for testing identifiers tracked via TracingContext
+ * Implemented in TracingHeaderValidator
+ */
+
+public interface Listener {

Review comment:
       yes, it's an interface to trigger header tests through callback when 
header is constructed. The tests are run only when listener is registered (not 
null), which is done across existing tests for different methods

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
##########
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+
+/**
+ * The TracingContext class to correlate Store requests using unique
+ * identifiers and resources common to requests (e.g. filesystem, stream)
+ *
+ * Implementing new HDFS method:
+ * Create TracingContext instance in method of outer layer of
+ * ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream), to be
+ * passed through ABFS layers up to AbfsRestOperation.
+ *
+ * Add new operations to HdfsOperationConstants file.
+ *
+ * PrimaryRequestId can be enabled for individual HDFS API that invoke
+ * multiple Store calls.
+ *
+ * Testing:
+ * Pass an instance of TracingHeaderValidator to registerListener() of ABFS
+ * filesystem/stream class before calling the API in tests.
+ */
+
+public class TracingContext {
+  private final String clientCorrelationID;
+  private final String fileSystemID;
+  private String clientRequestId = EMPTY_STRING;
+  private String primaryRequestID;

Review comment:
       primaryRequestId is applicable for any method call that triggers more 
than one http request. For example, methods using continuation logic like 
listStatus and rename
   Have added comments explaining use of tracingContext and its members at the 
beginning of file

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
##########
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+
+/**
+ * The TracingContext class to correlate Store requests using unique
+ * identifiers and resources common to requests (e.g. filesystem, stream)
+ *
+ * Implementing new HDFS method:
+ * Create TracingContext instance in method of outer layer of
+ * ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream), to be
+ * passed through ABFS layers up to AbfsRestOperation.
+ *
+ * Add new operations to HdfsOperationConstants file.
+ *
+ * PrimaryRequestId can be enabled for individual HDFS API that invoke
+ * multiple Store calls.
+ *
+ * Testing:
+ * Pass an instance of TracingHeaderValidator to registerListener() of ABFS
+ * filesystem/stream class before calling the API in tests.
+ */
+
+public class TracingContext {
+  private final String clientCorrelationID;
+  private final String fileSystemID;
+  private String clientRequestId = EMPTY_STRING;
+  private String primaryRequestID;
+  private String streamID;
+  private int retryCount;
+  private FSOperationType hadoopOpName;
+  private final TracingHeaderFormat format;
+  private Listener listener = null;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72;
+  public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*";
+
+  /**
+   * Initialize TracingContext
+   * @param clientCorrelationID Provided over config by client
+   * @param fileSystemID Unique guid for AzureBlobFileSystem instance
+   * @param hadoopOpName Code indicating the high-level Hadoop operation that
+   *                    triggered the current Store request
+   * @param tracingHeaderFormat Format of IDs to be printed in header and logs
+   * @param listener Holds instance of TracingHeaderValidator during testing,
+   *                null otherwise
+   */
+  public TracingContext(String clientCorrelationID, String fileSystemID,
+      FSOperationType hadoopOpName, TracingHeaderFormat tracingHeaderFormat,
+      Listener listener) {
+    this.fileSystemID = fileSystemID;
+    this.hadoopOpName = hadoopOpName;

Review comment:
       done

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
##########
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+
+/**
+ * The TracingContext class to correlate Store requests using unique
+ * identifiers and resources common to requests (e.g. filesystem, stream)
+ *
+ * Implementing new HDFS method:
+ * Create TracingContext instance in method of outer layer of
+ * ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream), to be
+ * passed through ABFS layers up to AbfsRestOperation.
+ *
+ * Add new operations to HdfsOperationConstants file.
+ *
+ * PrimaryRequestId can be enabled for individual HDFS API that invoke
+ * multiple Store calls.
+ *
+ * Testing:
+ * Pass an instance of TracingHeaderValidator to registerListener() of ABFS
+ * filesystem/stream class before calling the API in tests.
+ */
+
+public class TracingContext {
+  private final String clientCorrelationID;
+  private final String fileSystemID;
+  private String clientRequestId = EMPTY_STRING;
+  private String primaryRequestID;
+  private String streamID;
+  private int retryCount;
+  private FSOperationType hadoopOpName;
+  private final TracingHeaderFormat format;
+  private Listener listener = null;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72;
+  public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*";
+
+  /**
+   * Initialize TracingContext
+   * @param clientCorrelationID Provided over config by client
+   * @param fileSystemID Unique guid for AzureBlobFileSystem instance
+   * @param hadoopOpName Code indicating the high-level Hadoop operation that
+   *                    triggered the current Store request
+   * @param tracingHeaderFormat Format of IDs to be printed in header and logs
+   * @param listener Holds instance of TracingHeaderValidator during testing,
+   *                null otherwise
+   */
+  public TracingContext(String clientCorrelationID, String fileSystemID,
+      FSOperationType hadoopOpName, TracingHeaderFormat tracingHeaderFormat,
+      Listener listener) {
+    this.fileSystemID = fileSystemID;
+    this.hadoopOpName = hadoopOpName;
+    this.clientCorrelationID = 
validateClientCorrelationID(clientCorrelationID);
+    streamID = EMPTY_STRING;
+    retryCount = 0;
+    primaryRequestID = EMPTY_STRING;
+    format = tracingHeaderFormat;
+    this.listener = listener;
+  }
+
+  public TracingContext(String clientCorrelationID, String fileSystemID,
+      FSOperationType hadoopOpName, boolean needsPrimaryReqId,
+      TracingHeaderFormat tracingHeaderFormat, Listener listener) {
+    this(clientCorrelationID, fileSystemID, hadoopOpName, tracingHeaderFormat,
+        listener);
+    primaryRequestID = needsPrimaryReqId ? UUID.randomUUID().toString() : "";
+    if (listener != null) {
+      listener.updatePrimaryRequestID(primaryRequestID);
+    }
+  }
+
+  public TracingContext(TracingContext originalTracingContext) {
+    this.fileSystemID = originalTracingContext.fileSystemID;
+    this.streamID = originalTracingContext.streamID;
+    this.clientCorrelationID = originalTracingContext.clientCorrelationID;
+    this.hadoopOpName = originalTracingContext.hadoopOpName;
+    this.retryCount = 0;
+    this.primaryRequestID = originalTracingContext.primaryRequestID;
+    this.format = originalTracingContext.format;
+    if (originalTracingContext.listener != null) {
+      this.listener = originalTracingContext.listener.getClone();
+    }
+  }
+
+  public String validateClientCorrelationID(String clientCorrelationID) {
+    if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH)
+        || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) {

Review comment:
       true. Changed the validate method to be static, calling from ABFS 
constructor only now

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
##########
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+
+import static 
org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+
+/**
+ * The TracingContext class to correlate Store requests using unique
+ * identifiers and resources common to requests (e.g. filesystem, stream)
+ *
+ * Implementing new HDFS method:
+ * Create TracingContext instance in method of outer layer of
+ * ABFS driver (AzureBlobFileSystem/AbfsInputStream/AbfsOutputStream), to be
+ * passed through ABFS layers up to AbfsRestOperation.
+ *
+ * Add new operations to HdfsOperationConstants file.
+ *
+ * PrimaryRequestId can be enabled for individual HDFS API that invoke
+ * multiple Store calls.
+ *
+ * Testing:
+ * Pass an instance of TracingHeaderValidator to registerListener() of ABFS
+ * filesystem/stream class before calling the API in tests.
+ */
+
+public class TracingContext {
+  private final String clientCorrelationID;
+  private final String fileSystemID;
+  private String clientRequestId = EMPTY_STRING;
+  private String primaryRequestID;
+  private String streamID;
+  private int retryCount;
+  private FSOperationType hadoopOpName;
+  private final TracingHeaderFormat format;
+  private Listener listener = null;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72;
+  public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*";
+
+  /**
+   * Initialize TracingContext
+   * @param clientCorrelationID Provided over config by client
+   * @param fileSystemID Unique guid for AzureBlobFileSystem instance
+   * @param hadoopOpName Code indicating the high-level Hadoop operation that
+   *                    triggered the current Store request
+   * @param tracingHeaderFormat Format of IDs to be printed in header and logs
+   * @param listener Holds instance of TracingHeaderValidator during testing,
+   *                null otherwise
+   */
+  public TracingContext(String clientCorrelationID, String fileSystemID,
+      FSOperationType hadoopOpName, TracingHeaderFormat tracingHeaderFormat,
+      Listener listener) {
+    this.fileSystemID = fileSystemID;
+    this.hadoopOpName = hadoopOpName;
+    this.clientCorrelationID = 
validateClientCorrelationID(clientCorrelationID);
+    streamID = EMPTY_STRING;
+    retryCount = 0;
+    primaryRequestID = EMPTY_STRING;
+    format = tracingHeaderFormat;
+    this.listener = listener;
+  }
+
+  public TracingContext(String clientCorrelationID, String fileSystemID,
+      FSOperationType hadoopOpName, boolean needsPrimaryReqId,
+      TracingHeaderFormat tracingHeaderFormat, Listener listener) {
+    this(clientCorrelationID, fileSystemID, hadoopOpName, tracingHeaderFormat,
+        listener);
+    primaryRequestID = needsPrimaryReqId ? UUID.randomUUID().toString() : "";
+    if (listener != null) {
+      listener.updatePrimaryRequestID(primaryRequestID);
+    }
+  }
+
+  public TracingContext(TracingContext originalTracingContext) {
+    this.fileSystemID = originalTracingContext.fileSystemID;
+    this.streamID = originalTracingContext.streamID;
+    this.clientCorrelationID = originalTracingContext.clientCorrelationID;
+    this.hadoopOpName = originalTracingContext.hadoopOpName;
+    this.retryCount = 0;
+    this.primaryRequestID = originalTracingContext.primaryRequestID;
+    this.format = originalTracingContext.format;
+    if (originalTracingContext.listener != null) {
+      this.listener = originalTracingContext.listener.getClone();
+    }
+  }
+
+  public String validateClientCorrelationID(String clientCorrelationID) {
+    if ((clientCorrelationID.length() > MAX_CLIENT_CORRELATION_ID_LENGTH)
+        || (!clientCorrelationID.matches(CLIENT_CORRELATION_ID_PATTERN))) {
+      LOG.debug(
+          "Invalid config provided; correlation id not included in header.");
+      return EMPTY_STRING;
+    }
+    return clientCorrelationID;
+  }
+
+  public void generateClientRequestId() {
+    clientRequestId = UUID.randomUUID().toString();
+  }
+
+  public void setPrimaryRequestID() {
+    primaryRequestID = UUID.randomUUID().toString();
+    if (listener != null) {
+      listener.updatePrimaryRequestID(primaryRequestID);
+    }
+  }
+
+  public void setStreamID(String stream) {
+    streamID = stream;
+  }
+
+  public void setOperation(FSOperationType operation) {
+    this.hadoopOpName = operation;
+  }
+
+  public void setRetryCount(int retryCount) {
+    this.retryCount = retryCount;
+  }
+
+  public void setListener(Listener listener) {
+    this.listener = listener;
+  }
+
+  public String constructHeader() {
+    String header;
+    switch (format) {
+    case ALL_ID_FORMAT:
+      header =

Review comment:
       Added description to member declaration and added note in the fn
   Yes the number of separators (:) is kept constant for parsing

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderFormat.java
##########
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+public enum TracingHeaderFormat {

Review comment:
       switched enum, but keeping correlationId at the start would be easier to 
distinguish it as a custom id of variable length




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 616211)
    Time Spent: 7.5h  (was: 7h 20m)

> ABFS: Add Identifiers to Client Request Header
> ----------------------------------------------
>
>                 Key: HADOOP-17290
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17290
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/azure
>    Affects Versions: 3.3.0
>            Reporter: Sumangala Patki
>            Assignee: Sumangala Patki
>            Priority: Major
>              Labels: abfsactive, pull-request-available
>          Time Spent: 7.5h
>  Remaining Estimate: 0h
>
> Adding unique values to the client request header to assist in correlating 
> requests



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to