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

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

                Author: ASF GitHub Bot
            Created on: 25/Jun/21 11:57
            Start Date: 25/Jun/21 11:57
    Worklog Time Spent: 10m 
      Work Description: anoopsjohn commented on a change in pull request #2520:
URL: https://github.com/apache/hadoop/pull/2520#discussion_r658613610



##########
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:
       This clone of Context needed here?  What gets changed?

##########
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:
       getOutputStreamId() and getStreamID() -> Both create some confusion.  
Normally the Getter just return a already available value.  getStreamID() make 
sense.
   You can use createOutputStreamId() instead? 

##########
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:
       Ok the context been passed here might get changed at least wrt the 
retryCount.  that is why been cloned here?

##########
File path: 
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
##########
@@ -202,9 +206,10 @@ private void completeExecute() throws 
AzureBlobFileSystemException {
 
     retryCount = 0;
     LOG.debug("First execution of REST operation - {}", operationType);
-    while (!executeHttpOperation(retryCount)) {
+    while (!executeHttpOperation(retryCount, tracingContext)) {
       try {
         ++retryCount;
+        tracingContext.setRetryCount(retryCount);

Review comment:
       Yaaa here.

##########
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:
       The same thing in ABFSInputStream also

##########
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 will be used in case of Read ahead?  Worth adding come 
comments about its use here

##########
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:
       Will the below way will be more consistent?
   SINGLE_ID_FORMAT,  // client-req-id
   TWO_ID_FORMAT; // client-req-id:client-correlation-id
   ALL_ID_FORMAT,  // 
client-req-id:client-correlation-id:filesystem-id:primary-req-id:stream-id:hdfs-operation:retry-count

##########
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:
       generateClientRequestId() can be done internally within 
constructHeader()?  Else its upto callee to set it proper before.  Actually the 
UUID clientReqId generation should be an internal responsibility of this 
tracingContext right?
   All other methods are like setter.

##########
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:
       Its actually a fresh AbfsHttpOperation operation and here by we set the 
header.  Can we call it setXXX  than updateXXX?

##########
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:
       This is for testability only?

##########
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:
       call it opType only?

##########
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:
       A regex matching call is not that cheap. We will end up calling this for 
every object creation of this TracingContext.  Can we limit this check only at 
the time of FS instantiation?

##########
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:
       In any kind of op clientCorrelationID , clientRequestId , fileSystemID , 
hadoopOpName and retryCount will be present
   Optional things are primaryRequestID and streamID .  Correct?
   Worth detailing in some comments here.
   When these 2 are not there it will come like <file system ID>:::<op 
type>:...  ?




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

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


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

    Worklog Id:     (was: 614952)
    Time Spent: 6h 10m  (was: 6h)

> 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: 6h 10m
>  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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to