You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2021/05/25 15:00:45 UTC

[hadoop] branch branch-3.3 updated: HADOOP-17511. Add audit/telemetry logging to S3A connector (#2807)

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 464bbd5  HADOOP-17511. Add audit/telemetry logging to S3A connector (#2807)
464bbd5 is described below

commit 464bbd5b7c451bdb493a15ae61a631cf9a2fe2e5
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Tue May 25 10:25:41 2021 +0100

    HADOOP-17511. Add audit/telemetry logging to S3A connector (#2807)
    
    The S3A connector supports
    "an auditor", a plugin which is invoked
    at the start of every filesystem API call,
    and whose issued "audit span" provides a context
    for all REST operations against the S3 object store.
    
    The standard auditor sets the HTTP Referrer header
    on the requests with information about the API call,
    such as process ID, operation name, path,
    and even job ID.
    
    If the S3 bucket is configured to log requests, this
    information will be preserved there and so can be used
    to analyze and troubleshoot storage IO.
    
    Contributed by Steve Loughran.
    
    Change-Id: Ic0a105c194342ed2d529833ecc42608e8ba2f258
---
 .../org/apache/hadoop/fs/audit/AuditConstants.java |  108 ++
 .../hadoop/fs/audit/AuditStatisticNames.java       |   42 +-
 .../apache/hadoop/fs/audit/CommonAuditContext.java |  288 ++++
 .../org/apache/hadoop/fs/audit/package-info.java   |   29 +-
 .../java/org/apache/hadoop/fs/shell/Command.java   |    3 +
 .../java/org/apache/hadoop/fs/shell/PathData.java  |   18 +-
 .../hadoop/fs/statistics/StoreStatisticNames.java  |   10 +
 .../fs/statistics/impl/EmptyIOStatisticsStore.java |  182 +++
 .../fs/statistics/impl/IOStatisticsBinding.java    |   42 +
 .../apache/hadoop/fs/store}/LogExactlyOnce.java    |   12 +-
 .../fs/store/audit/ActiveThreadSpanSource.java     |   34 +-
 .../hadoop/fs/store/audit/AuditEntryPoint.java     |   55 +
 .../apache/hadoop/fs/store/audit/AuditSpan.java    |  109 ++
 .../hadoop/fs/store/audit/AuditSpanSource.java     |   50 +
 .../hadoop/fs/store/audit/AuditingFunctions.java   |  121 ++
 .../fs/store/audit/HttpReferrerAuditHeader.java    |  503 +++++++
 .../apache/hadoop/fs/store/audit/package-info.java |   29 +-
 .../hadoop/service/launcher/ServiceLauncher.java   |    2 +
 .../java/org/apache/hadoop/util/ToolRunner.java    |    5 +
 .../hadoop/util/functional/RemoteIterators.java    |   34 +-
 .../hadoop/fs/audit/TestCommonAuditContext.java    |  161 +++
 .../hadoop/fs/contract/ContractTestUtils.java      |   10 +-
 .../fs/s3a/CredentialInitializationException.java  |    9 +
 .../java/org/apache/hadoop/fs/s3a/Invoker.java     |   38 +-
 .../java/org/apache/hadoop/fs/s3a/Listing.java     |  131 +-
 .../org/apache/hadoop/fs/s3a/MultipartUtils.java   |  110 +-
 .../apache/hadoop/fs/s3a/S3ABlockOutputStream.java |   12 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 1382 ++++++++++++--------
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   |   45 +-
 .../apache/hadoop/fs/s3a/S3AInstrumentation.java   |   72 +-
 .../apache/hadoop/fs/s3a/S3ALocatedFileStatus.java |    1 +
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java |   18 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java    |   26 +-
 .../org/apache/hadoop/fs/s3a/S3ClientFactory.java  |    5 +
 .../java/org/apache/hadoop/fs/s3a/Statistic.java   |   72 +-
 .../apache/hadoop/fs/s3a/WriteOperationHelper.java |  290 ++--
 .../org/apache/hadoop/fs/s3a/WriteOperations.java  |   18 +-
 .../apache/hadoop/fs/s3a/api/RequestFactory.java   |  296 +++++
 .../LogExactlyOnce.java => api/package-info.java}  |   32 +-
 .../fs/s3a/audit/AWSAuditEventCallbacks.java       |  160 +++
 .../hadoop/fs/s3a/audit/AWSRequestAnalyzer.java    |  296 +++++
 .../AuditFailureException.java}                    |   29 +-
 .../hadoop/fs/s3a/audit/AuditIntegration.java      |  137 ++
 .../hadoop/fs/s3a/audit/AuditManagerS3A.java       |   92 ++
 .../AuditSpanS3A.java}                             |   25 +-
 .../hadoop/fs/s3a/audit/OperationAuditor.java      |   73 ++
 .../fs/s3a/audit/OperationAuditorOptions.java      |   74 ++
 .../hadoop/fs/s3a/audit/S3AAuditConstants.java     |  104 ++
 .../apache/hadoop/fs/s3a/audit/S3LogParser.java    |  309 +++++
 .../org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java |   51 +
 .../fs/s3a/audit/impl/AbstractAuditSpanImpl.java   |   99 ++
 .../s3a/audit/impl/AbstractOperationAuditor.java   |  121 ++
 .../fs/s3a/audit/impl/ActiveAuditManagerS3A.java   |  773 +++++++++++
 .../hadoop/fs/s3a/audit/impl/LoggingAuditor.java   |  435 ++++++
 .../fs/s3a/audit/impl/NoopAuditManagerS3A.java     |  185 +++
 .../hadoop/fs/s3a/audit/impl/NoopAuditor.java      |   99 ++
 .../apache/hadoop/fs/s3a/audit/impl/NoopSpan.java  |  109 ++
 .../impl/S3AInternalAuditConstants.java}           |   37 +-
 .../impl/package-info.java}                        |   28 +-
 .../apache/hadoop/fs/s3a/audit/package-info.java   |   41 +
 .../hadoop/fs/s3a/commit/AbstractS3ACommitter.java |   66 +-
 .../hadoop/fs/s3a/commit/CommitOperations.java     |   31 +-
 .../apache/hadoop/fs/s3a/commit/CommitUtils.java   |   24 +
 .../fs/s3a/commit/InternalCommitterConstants.java  |    6 +
 .../fs/s3a/commit/MagicCommitIntegration.java      |   15 +-
 .../fs/s3a/commit/magic/MagicCommitTracker.java    |    2 +-
 .../fs/s3a/commit/staging/StagingCommitter.java    |    2 +
 .../hadoop/fs/s3a/impl/AbstractStoreOperation.java |   40 +-
 .../hadoop/fs/s3a/impl/CallableSupplier.java       |   52 +-
 .../hadoop/fs/s3a/impl/ChangeDetectionPolicy.java  |    1 +
 .../hadoop/fs/s3a/impl/ContextAccessors.java       |   21 +-
 .../apache/hadoop/fs/s3a/impl/DeleteOperation.java |   17 +-
 .../fs/s3a/impl/ExecutingStoreOperation.java       |   40 +-
 .../fs/s3a/impl/GetContentSummaryOperation.java    |  208 +++
 .../hadoop/fs/s3a/impl/HeaderProcessing.java       |   44 +-
 .../hadoop/fs/s3a/impl/InternalConstants.java      |   10 +
 .../fs/s3a/impl/ListingOperationCallbacks.java     |   26 +-
 .../apache/hadoop/fs/s3a/impl/MkdirOperation.java  |  184 +++
 .../apache/hadoop/fs/s3a/impl/RenameOperation.java |   22 +-
 .../hadoop/fs/s3a/impl/RequestFactoryImpl.java     |  695 ++++++++++
 .../apache/hadoop/fs/s3a/impl/StoreContext.java    |   60 +-
 .../hadoop/fs/s3a/impl/StoreContextBuilder.java    |   18 +-
 .../fs/s3a/s3guard/DumpS3GuardDynamoTable.java     |   32 +-
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java      |    4 +-
 .../apache/hadoop/fs/s3a/s3guard/S3GuardTool.java  |   57 +-
 .../apache/hadoop/fs/s3a/select/SelectBinding.java |   29 +-
 .../impl/ForwardingIOStatisticsStore.java          |  186 +++
 .../org/apache/hadoop/fs/s3a/tools/MarkerTool.java |    7 +-
 .../src/site/markdown/tools/hadoop-aws/auditing.md |  389 ++++++
 .../tools/hadoop-aws/auditing_architecture.md      |  323 +++++
 .../src/site/markdown/tools/hadoop-aws/index.md    |    2 +
 .../tools/hadoop-aws/audit-architecture.png        |  Bin 0 -> 55701 bytes
 .../fs/contract/s3a/ITestS3AContractSeek.java      |    2 +-
 .../apache/hadoop/fs/s3a/AbstractS3ATestBase.java  |   46 +
 .../hadoop/fs/s3a/ITestS3AEmptyDirectory.java      |    7 +-
 .../hadoop/fs/s3a/ITestS3AFailureHandling.java     |   13 +-
 .../hadoop/fs/s3a/ITestS3AMiscOperations.java      |   23 +-
 .../hadoop/fs/s3a/ITestS3AMultipartUtils.java      |    4 +-
 .../hadoop/fs/s3a/ITestS3ARemoteFileChanged.java   |    2 +-
 .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java       |   45 +-
 .../apache/hadoop/fs/s3a/MockS3AFileSystem.java    |   44 +-
 .../apache/hadoop/fs/s3a/MultipartTestUtils.java   |   57 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java     |   46 -
 .../hadoop/fs/s3a/TestS3ABlockOutputStream.java    |   11 +-
 .../hadoop/fs/s3a/audit/AbstractAuditingTest.java  |  213 +++
 .../hadoop/fs/s3a/audit/AccessCheckingAuditor.java |   53 +
 .../hadoop/fs/s3a/audit/AuditTestSupport.java      |  123 ++
 .../fs/s3a/audit/ITestAuditAccessChecks.java       |  201 +++
 .../hadoop/fs/s3a/audit/ITestAuditManager.java     |  135 ++
 .../fs/s3a/audit/SimpleAWSRequestHandler.java      |   54 +
 .../hadoop/fs/s3a/audit/TestAuditIntegration.java  |  216 +++
 .../fs/s3a/audit/TestAuditSpanLifecycle.java       |  133 ++
 .../fs/s3a/audit/TestHttpReferrerAuditHeader.java  |  323 +++++
 .../hadoop/fs/s3a/audit/TestLoggingAuditor.java    |  203 +++
 .../apache/hadoop/fs/s3a/auth/ITestAssumeRole.java |    3 +-
 .../hadoop/fs/s3a/commit/AbstractCommitITest.java  |   12 +-
 .../fs/s3a/commit/AbstractITCommitProtocol.java    |    3 +-
 .../fs/s3a/commit/ITestCommitOperations.java       |    3 +-
 .../apache/hadoop/fs/s3a/impl/ITestXAttrCost.java  |    3 +-
 .../hadoop/fs/s3a/impl/TestHeaderProcessing.java   |   27 +-
 .../fs/s3a/impl/TestPartialDeleteFailures.java     |   16 +-
 .../hadoop/fs/s3a/impl/TestRequestFactory.java     |  185 +++
 .../fs/s3a/performance/AbstractS3ACostTest.java    |   33 +-
 .../performance/ITestDirectoryMarkerListing.java   |   13 +-
 .../s3a/performance/ITestS3AMiscOperationCost.java |  143 ++
 .../fs/s3a/performance/ITestS3AMkdirCost.java      |  185 +++
 .../fs/s3a/performance/OperationCostValidator.java |    3 +-
 .../hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java    |    2 -
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java      |   47 +-
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java    |    3 +-
 .../fs/s3a/scale/ITestS3ADirectoryPerformance.java |  137 +-
 .../s3a/test/MinimalListingOperationCallbacks.java |   10 +-
 .../hadoop-aws/src/test/resources/core-site.xml    |   13 +
 .../hadoop-aws/src/test/resources/log4j.properties |    6 +
 134 files changed, 11770 insertions(+), 1325 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java
new file mode 100644
index 0000000..d9629e3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java
@@ -0,0 +1,108 @@
+/*
+ * 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.audit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Constants related to auditing.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class AuditConstants {
+
+  private AuditConstants() {
+  }
+
+  /**
+   * The host from where requests originate: {@value}.
+   * example.org is used as the IETF require that it never resolves.
+   * This isn't always met by some mobile/consumer DNS services, but
+   * we don't worry about that. What is important is that
+   * a scan for "example.org" in the logs will exclusively find
+   * entries from this referrer.
+   */
+  public static final String REFERRER_ORIGIN_HOST = "audit.example.org";
+
+  /**
+   * Header: Command: {@value}.
+   * Set by tool runner.
+   */
+  public static final String PARAM_COMMAND = "cm";
+
+  /**
+   * Header: FileSystem ID: {@value}.
+   */
+  public static final String PARAM_FILESYSTEM_ID = "fs";
+
+  /**
+   * Header: operation ID: {@value}.
+   */
+  public static final String PARAM_ID = "id";
+
+  /**
+   * JobID query header: {@value}.
+   */
+  public static final String PARAM_JOB_ID = "ji";
+
+  /**
+   * Header: operation: {@value}.
+   * These should be from StoreStatisticNames or similar,
+   * and are expected to be at the granularity of FS
+   * API operations.
+   */
+  public static final String PARAM_OP = "op";
+
+  /**
+   * Header: first path of operation: {@value}.
+   */
+  public static final String PARAM_PATH = "p1";
+
+  /**
+   * Header: second path of operation: {@value}.
+   */
+  public static final String PARAM_PATH2 = "p2";
+
+  /**
+   * Header: Principal: {@value}.
+   */
+  public static final String PARAM_PRINCIPAL = "pr";
+
+  /**
+   * Header: Process ID: {@value}.
+   */
+  public static final String PARAM_PROCESS = "ps";
+
+  /**
+   * Thread 0: the thread which created a span {@value}.
+   */
+  public static final String PARAM_THREAD0 = "t0";
+
+  /**
+   * Thread 1: the thread making the S3 request: {@value}.
+   */
+  public static final String PARAM_THREAD1 = "t1";
+
+  /**
+   * Timestamp of span creation: {@value}.
+   */
+  public static final String PARAM_TIMESTAMP = "ts";
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java
similarity index 54%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java
index 54a8836..0ee9d62 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditStatisticNames.java
@@ -16,27 +16,35 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
+package org.apache.hadoop.fs.audit;
 
 /**
- * Log exactly once, even across threads.
+ * Statistic Names for Auditing.
  */
-public class LogExactlyOnce {
+public final class AuditStatisticNames {
 
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
-
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
+  private AuditStatisticNames() {
   }
 
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
-  }
+  /**
+   * Audit failure: {@value}.
+   */
+  public static final String AUDIT_FAILURE = "audit_failure";
+
+  /**
+   * A request was executed and the auditor invoked: {@value}.
+   */
+  public static final String AUDIT_REQUEST_EXECUTION
+      = "audit_request_execution";
+
+  /**
+   * Audit span created: {@value}.
+   */
+  public static final String AUDIT_SPAN_CREATION = "audit_span_creation";
+
+  /**
+   * Access check during audit rejected: {@value}.
+   */
+  public static final String AUDIT_ACCESS_CHECK_FAILURE
+      = "audit_access_check_failure";
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java
new file mode 100644
index 0000000..1168154
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java
@@ -0,0 +1,288 @@
+/*
+ * 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.audit;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_COMMAND;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PROCESS;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1;
+
+/**
+ * The common audit context is a map of common context information
+ * which can be used with any audit span.
+ * This context is shared across all Filesystems within the
+ * thread.
+ * Audit spans will be created with a reference to the current
+ * context of their thread;
+ * That reference is retained even as they are moved across threads, so
+ * context information (including thread ID Java runtime).
+ *
+ * The Global context entries are a set of key-value pairs which span
+ * all threads; the {@code HttpReferrerAuditHeader} picks these
+ * up automatically. It is intended for minimal use of
+ * shared constant values (process ID, entry point).
+ *
+ * An attribute set in {@link #setGlobalContextEntry(String, String)}
+ * will be set across all audit spans in all threads.
+ *
+ * The {@link #noteEntryPoint(Object)} method should be
+ * used in entry points (ToolRunner.run, etc). It extracts
+ * the final element of the classname and attaches that
+ * to the global context with the attribute key
+ * {@link AuditConstants#PARAM_COMMAND}, if not already
+ * set.
+ * This helps identify the application being executued.
+ *
+ * All other values set are specific to this context, which
+ * is thread local.
+ * The attributes which can be added to ths common context include
+ * evaluator methods which will be evaluated in whichever thread
+ * invokes {@link #getEvaluatedEntries()} and then evaluates them.
+ * That map of evaluated options may evaluated later, in a different
+ * thread.
+ *
+ * For setting and clearing thread-level options, use
+ * {@link #currentAuditContext()} to get the thread-local
+ * context for the caller, which can then be manipulated.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class CommonAuditContext {
+
+  /**
+   * Process ID; currently built from UUID and timestamp.
+   */
+  public static final String PROCESS_ID = UUID.randomUUID().toString();
+
+  /**
+   * Context values which are global.
+   * To be used very sparingly.
+   */
+  private static final Map<String, String> GLOBAL_CONTEXT_MAP =
+      new ConcurrentHashMap<>();
+
+  /**
+   * Map of data. Concurrent so when shared across threads
+   * there are no problems.
+   * Supplier operations must themselves be thread safe.
+   */
+  private final Map<String, Supplier<String>> evaluatedEntries =
+      new ConcurrentHashMap<>();
+
+  static {
+    // process ID is fixed.
+    setGlobalContextEntry(PARAM_PROCESS, PROCESS_ID);
+  }
+
+  /**
+   * Thread local context.
+   * Use a weak reference just to keep memory costs down.
+   * The S3A committers all have a strong reference, so if they are
+   * retained, context is retained.
+   * If a span retains the context, then it will also stay valid until
+   * the span is finalized.
+   */
+  private static final ThreadLocal<CommonAuditContext> ACTIVE_CONTEXT =
+      ThreadLocal.withInitial(() -> createInstance());
+
+  private CommonAuditContext() {
+  }
+
+  /**
+   * Put a context entry.
+   * @param key key
+   * @param value new value
+   * @return old value or null
+   */
+  public Supplier<String> put(String key, String value) {
+    return evaluatedEntries.put(key, () -> value);
+  }
+
+  /**
+   * Put a context entry dynamically evaluated on demand.
+   * @param key key
+   * @param value new value
+   * @return old value or null
+   */
+  public Supplier<String> put(String key, Supplier<String> value) {
+    return evaluatedEntries.put(key, value);
+  }
+
+  /**
+   * Remove a context entry.
+   * @param key key
+   */
+  public void remove(String key) {
+    evaluatedEntries.remove(key);
+  }
+
+  /**
+   * Get a context entry.
+   * @param key key
+   * @return value or null
+   */
+  public String get(String key) {
+    Supplier<String> supplier = evaluatedEntries.get(key);
+    return supplier != null
+        ? supplier.get()
+        : null;
+  }
+
+  /**
+   * Rest the context; will set the standard options again.
+   * Primarily for testing.
+   */
+  public void reset() {
+    evaluatedEntries.clear();
+    init();
+  }
+
+  /**
+   * Initialize.
+   */
+  private void init() {
+
+    // thread 1 is dynamic
+    put(PARAM_THREAD1, () -> currentThreadID());
+  }
+
+  /**
+   * Does the context contain a specific key?
+   * @param key key
+   * @return true if it is in the context.
+   */
+  public boolean containsKey(String key) {
+    return evaluatedEntries.containsKey(key);
+  }
+
+  /**
+   * Demand invoked to create the instance for this thread.
+   * @return an instance.
+   */
+  private static CommonAuditContext createInstance() {
+    CommonAuditContext context = new CommonAuditContext();
+    context.init();
+    return context;
+  }
+
+  /**
+   * Get the current common audit context. Thread local.
+   * @return the audit context of this thread.
+   */
+  public static CommonAuditContext currentAuditContext() {
+    return ACTIVE_CONTEXT.get();
+  }
+
+  /**
+   * A thread ID which is unique for this process and shared across all
+   * S3A clients on the same thread, even those using different FS instances.
+   * @return a thread ID for reporting.
+   */
+  public static String currentThreadID() {
+    return Long.toString(Thread.currentThread().getId());
+  }
+
+  /**
+   * Get the evaluated operations.
+   * This is the map unique to this context.
+   * @return the operations map.
+   */
+  public Map<String, Supplier<String>> getEvaluatedEntries() {
+    return evaluatedEntries;
+  }
+
+  /**
+   * Set a global entry.
+   * @param key key
+   * @param value value
+   */
+  public static void setGlobalContextEntry(String key, String value) {
+    GLOBAL_CONTEXT_MAP.put(key, value);
+  }
+
+  /**
+   * Get a global entry.
+   * @param key key
+   * @return value or null
+   */
+  public static String getGlobalContextEntry(String key) {
+    return GLOBAL_CONTEXT_MAP.get(key);
+  }
+
+  /**
+   * Remove a global entry.
+   * @param key key to clear.
+   */
+  public static void removeGlobalContextEntry(String key) {
+    GLOBAL_CONTEXT_MAP.remove(key);
+  }
+
+  /**
+   * Add the entry point as a context entry with the key
+   * {@link AuditConstants#PARAM_COMMAND}
+   * if it has not  already been recorded.
+   * This is called via ToolRunner but may be used at any
+   * other entry point.
+   * @param tool object loaded/being launched.
+   */
+  public static void noteEntryPoint(Object tool) {
+    if (tool != null && !GLOBAL_CONTEXT_MAP.containsKey(PARAM_COMMAND)) {
+      String classname = tool.getClass().toString();
+      int lastDot = classname.lastIndexOf('.');
+      int l = classname.length();
+      if (lastDot > 0 && lastDot < (l - 1)) {
+        String name = classname.substring(lastDot + 1, l);
+        setGlobalContextEntry(PARAM_COMMAND, name);
+      }
+    }
+  }
+
+  /**
+   * Get an iterator over the global entries.
+   * Thread safe.
+   * @return an iterable to enumerate the values.
+   */
+  public static Iterable<Map.Entry<String, String>>
+      getGlobalContextEntries() {
+    return new GlobalIterable();
+  }
+
+  /**
+   * Iterable to the global iterator. Avoids serving
+   * up full access to the map.
+   */
+  private static final class GlobalIterable
+      implements Iterable<Map.Entry<String, String>> {
+
+    @Override
+    public Iterator<Map.Entry<String, String>> iterator() {
+      return GLOBAL_CONTEXT_MAP.entrySet().iterator();
+    }
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java
similarity index 62%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java
index 54a8836..16c2249 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/package-info.java
@@ -16,27 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
-
 /**
- * Log exactly once, even across threads.
+ * Public classes for adding information to any auditing information
+ * picked up by filesystem clients.
+ *
  */
-public class LogExactlyOnce {
-
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
-
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
-  }
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.audit;
 
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
-  }
-}
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java
index c818257..0bdb477 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator;
+
 /**
  * An abstract class for the execution of a file system command
  */
@@ -361,6 +363,7 @@ abstract public class Command extends Configured {
         }
       }
     }
+    cleanupRemoteIterator(itemsIterator);
   }
 
   private void processPathInternal(PathData item) throws IOException {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
index dad54ea..1ff8d8f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.fs.RemoteIterator;
 
+import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator;
+
 /**
  * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs).
  * PathData ensures that the returned path string will be the same as the
@@ -287,20 +289,8 @@ public class PathData implements Comparable<PathData> {
       throws IOException {
     checkIfExists(FileTypeRequirement.SHOULD_BE_DIRECTORY);
     final RemoteIterator<FileStatus> stats = this.fs.listStatusIterator(path);
-    return new RemoteIterator<PathData>() {
-
-      @Override
-      public boolean hasNext() throws IOException {
-        return stats.hasNext();
-      }
-
-      @Override
-      public PathData next() throws IOException {
-        FileStatus file = stats.next();
-        String child = getStringForChildPath(file.getPath());
-        return new PathData(fs, child, file);
-      }
-    };
+    return mappingRemoteIterator(stats,
+        file -> new PathData(fs, getStringForChildPath(file.getPath()), file));
   }
 
   /**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
index 9514439..9ec8dcd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
@@ -40,6 +40,9 @@ public final class StoreStatisticNames {
   /** {@value}. */
   public static final String OP_ABORT = "op_abort";
 
+  /** access() API call {@value}. */
+  public static final String OP_ACCESS = "op_access";
+
   /** {@value}. */
   public static final String OP_APPEND = "op_append";
 
@@ -161,6 +164,10 @@ public final class StoreStatisticNames {
   public static final String DELEGATION_TOKENS_ISSUED
       = "delegation_tokens_issued";
 
+  /** Probe for store existing: {@value}. */
+  public static final String STORE_EXISTS_PROBE
+      = "store_exists_probe";
+
   /** Requests throttled and retried: {@value}. */
   public static final String STORE_IO_THROTTLED
       = "store_io_throttled";
@@ -379,6 +386,9 @@ public final class StoreStatisticNames {
   public static final String MULTIPART_UPLOAD_STARTED
       = "multipart_upload_started";
 
+  public static final String MULTIPART_UPLOAD_LIST
+      = "multipart_upload_list";
+
   private StoreStatisticNames() {
   }
 
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java
new file mode 100644
index 0000000..c970546
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/EmptyIOStatisticsStore.java
@@ -0,0 +1,182 @@
+/*
+ * 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.statistics.impl;
+
+import javax.annotation.Nullable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.MeanStatistic;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * An Empty IOStatisticsStore implementation.
+ */
+final class EmptyIOStatisticsStore implements IOStatisticsStore {
+
+  /**
+   * The sole instance of this class.
+   */
+  private static final EmptyIOStatisticsStore INSTANCE =
+      new EmptyIOStatisticsStore();
+  /**
+   * Get the single instance of this class.
+   * @return a shared, empty instance.
+   */
+  static IOStatisticsStore getInstance() {
+    return INSTANCE;
+  }
+
+  private EmptyIOStatisticsStore() {
+  }
+
+  @Override
+  public Map<String, Long> counters() {
+    return emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> gauges() {
+    return emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> minimums() {
+    return emptyMap();
+  }
+
+  @Override
+  public Map<String, Long> maximums() {
+    return emptyMap();
+  }
+
+  @Override
+  public Map<String, MeanStatistic> meanStatistics() {
+    return emptyMap();
+  }
+
+  @Override
+  public boolean aggregate(@Nullable final IOStatistics statistics) {
+    return false;
+  }
+
+  @Override
+  public long incrementCounter(final String key, final long value) {
+    return 0;
+  }
+
+  @Override
+  public void setCounter(final String key, final long value) {
+
+  }
+
+  @Override
+  public void setGauge(final String key, final long value) {
+
+  }
+
+  @Override
+  public long incrementGauge(final String key, final long value) {
+    return 0;
+  }
+
+  @Override
+  public void setMaximum(final String key, final long value) {
+
+  }
+
+  @Override
+  public long incrementMaximum(final String key, final long value) {
+    return 0;
+  }
+
+  @Override
+  public void setMinimum(final String key, final long value) {
+
+  }
+
+  @Override
+  public long incrementMinimum(final String key, final long value) {
+    return 0;
+  }
+
+  @Override
+  public void addMinimumSample(final String key, final long value) {
+
+  }
+
+  @Override
+  public void addMaximumSample(final String key, final long value) {
+
+  }
+
+  @Override
+  public void setMeanStatistic(final String key, final MeanStatistic value) {
+
+  }
+
+  @Override
+  public void addMeanStatisticSample(final String key, final long value) {
+
+  }
+
+  @Override
+  public void reset() {
+
+  }
+
+  @Override
+  public AtomicLong getCounterReference(final String key) {
+    return null;
+  }
+
+  @Override
+  public AtomicLong getMaximumReference(final String key) {
+    return null;
+  }
+
+  @Override
+  public AtomicLong getMinimumReference(final String key) {
+    return null;
+  }
+
+  @Override
+  public AtomicLong getGaugeReference(final String key) {
+    return null;
+  }
+
+  @Override
+  public MeanStatistic getMeanStatistic(final String key) {
+    return null;
+  }
+
+  @Override
+  public void addTimedOperation(final String prefix,
+      final long durationMillis) {
+
+  }
+
+  @Override
+  public void addTimedOperation(final String prefix, final Duration duration) {
+
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
index c3507db..1aece01 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiFunction;
 import java.util.function.Function;
+import java.util.function.Supplier;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 
@@ -98,6 +99,15 @@ public final class IOStatisticsBinding {
   }
 
   /**
+   * Get the shared instance of the immutable empty statistics
+   * store.
+   * @return an empty statistics object.
+   */
+  public static IOStatisticsStore emptyStatisticsStore() {
+    return EmptyIOStatisticsStore.getInstance();
+  }
+
+  /**
    * Take an IOStatistics instance and wrap it in a source.
    * @param statistics statistics.
    * @return a source which will return the values
@@ -574,6 +584,38 @@ public final class IOStatisticsBinding {
   }
 
   /**
+   * Given a Java supplier, evaluate it while
+   * tracking the duration of the operation and success/failure.
+   * @param factory factory of duration trackers
+   * @param statistic statistic key
+   * @param input input callable.
+   * @param <B> return type.
+   * @return the output of the supplier.
+   */
+  public static <B> B trackDurationOfSupplier(
+      @Nullable DurationTrackerFactory factory,
+      String statistic,
+      Supplier<B> input) {
+    // create the tracker outside try-with-resources so
+    // that failures can be set in the catcher.
+    DurationTracker tracker = createTracker(factory, statistic);
+    try {
+      // exec the input function and return its value
+      return input.get();
+    } catch (RuntimeException e) {
+      // input function failed: note it
+      tracker.failed();
+      // and rethrow
+      throw e;
+    } finally {
+      // update the tracker.
+      // this is called after any catch() call will have
+      // set the failed flag.
+      tracker.close();
+    }
+  }
+
+  /**
    * Create the tracker. If the factory is null, a stub
    * tracker is returned.
    * @param factory tracker factory
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/LogExactlyOnce.java
similarity index 81%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/LogExactlyOnce.java
index 54a8836..04cd511 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/LogExactlyOnce.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
+package org.apache.hadoop.fs.store;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -39,4 +39,14 @@ public class LogExactlyOnce {
       log.warn(format, args);
     }
   }
+  public void info(String format, Object...args) {
+    if (!logged.getAndSet(true)) {
+      log.info(format, args);
+    }
+  }
+  public void error(String format, Object...args) {
+    if (!logged.getAndSet(true)) {
+      log.error(format, args);
+    }
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java
similarity index 57%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java
index 54a8836..4ddb8e1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/ActiveThreadSpanSource.java
@@ -16,27 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
+package org.apache.hadoop.fs.store.audit;
 
 /**
- * Log exactly once, even across threads.
+ * Interface to get the active thread span.
+ * This can be used to collect the active span to
+ * propagate it into other threads.
+ *
+ * FileSystems which track their active span may implement
+ * this and offer their active span.
  */
-public class LogExactlyOnce {
-
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
-
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
-  }
+public interface ActiveThreadSpanSource<T extends AuditSpan> {
 
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
-  }
+  /**
+   * The active span. This may not be a valid span, i.e. there is no guarantee
+   * that {@code getActiveAuditSpan().isValidSpan()} is true, but
+   * implementations MUST always return a non-null span.
+   * @return the currently active span.
+   */
+  T getActiveAuditSpan();
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java
new file mode 100644
index 0000000..6210dd0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditEntryPoint.java
@@ -0,0 +1,55 @@
+/*
+ * 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.store.audit;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * A marker attribute simply to highlight which of the methods
+ * in a FileSystem why are audit entry points.
+ * <ol>
+ *   <li>
+ *     A FS method is an AuditEntryPoint if, on invocation it
+ *     creates and activates an Audit Span for that FS.
+ *   </li>
+ *   <li>
+ *     The audit span SHOULD be deactivated before returning,
+ *   </li>
+ *   <li>
+ *     Objects returned by the API call which go on
+ *     to make calls of the filesystem MUST perform
+ *     all IO within the same audit span.
+ *   </li>
+ *   <li>
+ *     Audit Entry points SHOULD NOT invoke other Audit Entry Points.
+ *     This is to ensure the original audit span information
+ *     is not replaced.
+ *   </li>
+ * </ol>
+ * FileSystem methods the entry point then invokes
+ * SHOULD NOT invoke audit entry points internally.
+ *
+ * All external methods MUST be audit entry points.
+ */
+@Documented
+@Retention(RetentionPolicy.SOURCE)
+public @interface AuditEntryPoint {
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java
new file mode 100644
index 0000000..ecdaf71
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpan.java
@@ -0,0 +1,109 @@
+/*
+ * 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.store.audit;
+
+import java.io.Closeable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This is a span created by an {@link AuditSpanSource}.
+ * An implementation of a span may carry context which can be picked
+ * up by the filesystem when activated.
+ * Each FS can have one active span per thread.
+ * Different filesystem instances SHALL have different active
+ * spans (if they support them)
+ * A span is activated in a thread when {@link #activate()}
+ * is called.
+ * The span stays active in that thread until {@link #deactivate()}
+ * is called.
+ * When deactivated in one thread, it MAY still be active in others.
+ * There's no explicit "end of span"; this is too hard to manage in
+ * terms of API lifecycle.
+ * Similarly, there's no stack of spans. Once a span is activated,
+ * the previous span is forgotten about.
+ * Therefore each FS will need a fallback "inactive span" which
+ * will be reverted to on deactivation of any other span.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface AuditSpan extends Closeable {
+
+  /**
+   * Return a span ID which must be unique for all spans within
+   * everywhere. That effectively means part of the
+   * span SHOULD be derived from a UUID.
+   * Callers MUST NOT make any assumptions about the actual
+   * contents or structure of this string other than the
+   * uniqueness.
+   * @return a non-empty string
+   */
+  String getSpanId();
+
+  /**
+   * Get the name of the operation.
+   * @return the operation name.
+   */
+  String getOperationName();
+
+  /**
+   * Timestamp in UTC of span creation.
+   * @return timestamp.
+   */
+  long getTimestamp();
+
+  /**
+   * Make this span active in the current thread.
+   * @return the activated span.
+   * This is makes it easy to use in try with resources
+   */
+  AuditSpan activate();
+
+  /**
+   * Deactivate the span in the current thread.
+   */
+  void deactivate();
+
+  /**
+   * Close calls {@link #deactivate()}; subclasses may override
+   * but the audit manager's wrapping span will always relay to
+   * {@link #deactivate()} rather
+   * than call this method on the wrapped span.
+   */
+  default void close() {
+    deactivate();
+  }
+
+  /**
+   * Is the span valid? False == this is a span to indicate unbonded.
+   * @return true if this span represents a real operation.
+   */
+  default boolean isValidSpan() {
+    return true;
+  }
+
+  /**
+   * Set an attribute.
+   * This may or may not be propagated to audit logs.
+   * @param key attribute name
+   * @param value value
+   */
+  default void set(String key, String value) { }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java
new file mode 100644
index 0000000..4f9f5a6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditSpanSource.java
@@ -0,0 +1,50 @@
+/*
+ * 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.store.audit;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A source of audit spans.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface AuditSpanSource<T extends AuditSpan> {
+
+  /**
+   * Create a span for an operation.
+   *
+   * All operation names <i>SHOULD</i> come from
+   * {@code StoreStatisticNames} or
+   * {@code StreamStatisticNames}.
+   * @param operation operation name.
+   * @param path1 first path of operation
+   * @param path2 second path of operation
+   * @return a span for the audit
+   * @throws IOException failure
+   */
+  T createSpan(String operation,
+      @Nullable String path1,
+      @Nullable String path2)
+      throws IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java
new file mode 100644
index 0000000..acc8276
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java
@@ -0,0 +1,121 @@
+/*
+ * 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.store.audit;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.FunctionRaisingIOE;
+import org.apache.hadoop.util.functional.InvocationRaisingIOE;
+
+/**
+ * Static methods to assist in working with Audit Spans.
+ * the {@code withinX} calls take a span and a closure/function etc.
+ * and return a new function of the same types but which will
+ * activate and the span.
+ * They do not deactivate it afterwards to avoid accidentally deactivating
+ * the already-active span during a chain of operations in the same thread.
+ * All they do is ensure that the given span is guaranteed to be
+ * active when the passed in callable/function/invokable is evaluated.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class AuditingFunctions {
+
+  private AuditingFunctions() {
+  }
+
+  /**
+   * Given a callable, return a new callable which
+   * activates and deactivates the span around the inner invocation.
+   * @param auditSpan audit span
+   * @param operation operation
+   * @param <T> type of result
+   * @return a new invocation.
+   */
+  public static <T> CallableRaisingIOE<T> withinAuditSpan(
+      @Nullable AuditSpan auditSpan,
+      CallableRaisingIOE<T> operation) {
+    return auditSpan == null
+        ? operation
+        : () -> {
+          auditSpan.activate();
+          return operation.apply();
+        };
+  }
+
+  /**
+   * Given an invocation, return a new invocation which
+   * activates and deactivates the span around the inner invocation.
+   * @param auditSpan audit span
+   * @param operation operation
+   * @return a new invocation.
+   */
+  public static InvocationRaisingIOE withinAuditSpan(
+      @Nullable AuditSpan auditSpan,
+      InvocationRaisingIOE operation) {
+    return auditSpan == null
+        ? operation
+        : () -> {
+          auditSpan.activate();
+          operation.apply();
+        };
+  }
+
+  /**
+   * Given a function, return a new function which
+   * activates and deactivates the span around the inner one.
+   * @param auditSpan audit span
+   * @param operation operation
+   * @return a new invocation.
+   */
+  public static <T, R> FunctionRaisingIOE<T, R> withinAuditSpan(
+      @Nullable AuditSpan auditSpan,
+      FunctionRaisingIOE<T, R> operation) {
+    return auditSpan == null
+        ? operation
+        : (x) -> {
+          auditSpan.activate();
+          return operation.apply(x);
+        };
+  }
+
+  /**
+   * Given a callable, return a new callable which
+   * activates and deactivates the span around the inner invocation.
+   * @param auditSpan audit span
+   * @param operation operation
+   * @param <T> type of result
+   * @return a new invocation.
+   */
+  public static <T> Callable<T> callableWithinAuditSpan(
+      @Nullable AuditSpan auditSpan,
+      Callable<T> operation) {
+    return auditSpan == null
+        ? operation
+        : () -> {
+          auditSpan.activate();
+          return operation.call();
+        };
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java
new file mode 100644
index 0000000..b2684e7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java
@@ -0,0 +1,503 @@
+/*
+ * 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.store.audit;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.http.NameValuePair;
+import org.apache.http.client.utils.URLEncodedUtils;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_ID;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_OP;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH2;
+import static org.apache.hadoop.fs.audit.AuditConstants.REFERRER_ORIGIN_HOST;
+
+/**
+ * Contains all the logic for generating an HTTP "Referer"
+ * entry; includes escaping query params.
+ * Tests for this are in
+ * {@code org.apache.hadoop.fs.s3a.audit.TestHttpReferrerAuditHeader}
+ * so as to verify that header generation in the S3A auditors, and
+ * S3 log parsing, all work.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class HttpReferrerAuditHeader {
+
+  /**
+   * Format of path to build: {@value}.
+   * the params passed in are (context ID, span ID, op).
+   * Update
+   * {@code TestHttpReferrerAuditHeader.SAMPLE_LOG_ENTRY} on changes
+   */
+  public static final String REFERRER_PATH_FORMAT = "/hadoop/1/%3$s/%2$s/";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HttpReferrerAuditHeader.class);
+
+  /**
+   * Log for warning of problems creating headers will only log of
+   * a problem once per process instance.
+   * This is to avoid logs being flooded with errors.
+   */
+  private static final LogExactlyOnce WARN_OF_URL_CREATION =
+      new LogExactlyOnce(LOG);
+
+  /** Context ID. */
+  private final String contextId;
+
+  /** operation name. */
+  private final String operationName;
+
+  /** Span ID. */
+  private final String spanId;
+
+  /** optional first path. */
+  private final String path1;
+
+  /** optional second path. */
+  private final String path2;
+
+  /**
+   * The header as created in the constructor; used in toString().
+   * A new header is built on demand in {@link #buildHttpReferrer()}
+   * so that evaluated attributes are dynamically evaluated
+   * in the correct thread/place.
+   */
+  private final String initialHeader;
+
+  /**
+   * Map of simple attributes.
+   */
+  private final Map<String, String> attributes;
+
+  /**
+   * Parameters dynamically evaluated on the thread just before
+   * the request is made.
+   */
+  private final Map<String, Supplier<String>> evaluated;
+
+  /**
+   * Elements to filter from the final header.
+   */
+  private final Set<String> filter;
+
+  /**
+   * Instantiate.
+   *
+   * Context and operationId are expected to be well formed
+   * numeric/hex strings, at least adequate to be
+   * used as individual path elements in a URL.
+   */
+  private HttpReferrerAuditHeader(
+      final Builder builder) {
+    this.contextId = requireNonNull(builder.contextId);
+    this.evaluated = builder.evaluated;
+    this.filter = builder.filter;
+    this.operationName = requireNonNull(builder.operationName);
+    this.path1 = builder.path1;
+    this.path2 = builder.path2;
+    this.spanId = requireNonNull(builder.spanId);
+
+    // copy the parameters from the builder and extend
+    attributes = builder.attributes;
+
+    addAttribute(PARAM_OP, operationName);
+    addAttribute(PARAM_PATH, path1);
+    addAttribute(PARAM_PATH2, path2);
+    addAttribute(PARAM_ID, spanId);
+
+    // patch in global context values where not set
+    Iterable<Map.Entry<String, String>> globalContextValues
+        = builder.globalContextValues;
+    if (globalContextValues != null) {
+      for (Map.Entry<String, String> entry : globalContextValues) {
+        attributes.putIfAbsent(entry.getKey(), entry.getValue());
+      }
+    }
+
+    // build the referrer up. so as to find/report problems early
+    initialHeader = buildHttpReferrer();
+  }
+
+  /**
+   * Build the referrer string.
+   * This includes dynamically evaluating all of the evaluated
+   * attributes.
+   * If there is an error creating the string it will be logged once
+   * per entry, and "" returned.
+   * @return a referrer string or ""
+   */
+  public String buildHttpReferrer() {
+
+    String header;
+    try {
+      String queries;
+      // Update any params which are dynamically evaluated
+      evaluated.forEach((key, eval) ->
+          addAttribute(key, eval.get()));
+      // now build the query parameters from all attributes, static and
+      // evaluated, stripping out any from the filter
+      queries = attributes.entrySet().stream()
+          .filter(e -> !filter.contains(e.getKey()))
+          .map(e -> e.getKey() + "=" + e.getValue())
+          .collect(Collectors.joining("&"));
+      final URI uri = new URI("https", REFERRER_ORIGIN_HOST,
+          String.format(Locale.ENGLISH, REFERRER_PATH_FORMAT,
+              contextId, spanId, operationName),
+          queries,
+          null);
+      header = uri.toASCIIString();
+    } catch (URISyntaxException e) {
+      WARN_OF_URL_CREATION.warn("Failed to build URI for auditor: " + e, e);
+      header = "";
+    }
+    return header;
+  }
+
+  /**
+   * Add a query parameter if not null/empty
+   * There's no need to escape here as it is done in the URI
+   * constructor.
+   * @param key query key
+   * @param value query value
+   */
+  private void addAttribute(String key,
+      String value) {
+    if (StringUtils.isNotEmpty(value)) {
+      attributes.put(key, value);
+    }
+  }
+
+  /**
+   * Set an attribute. If the value is non-null/empty,
+   * it will be used as a query parameter.
+   *
+   * @param key key to set
+   * @param value value.
+   */
+  public void set(final String key, final String value) {
+    addAttribute(requireNonNull(key), value);
+  }
+
+  public String getContextId() {
+    return contextId;
+  }
+
+  public String getOperationName() {
+    return operationName;
+  }
+
+  public String getSpanId() {
+    return spanId;
+  }
+
+  public String getPath1() {
+    return path1;
+  }
+
+  public String getPath2() {
+    return path2;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ",
+        HttpReferrerAuditHeader.class.getSimpleName() + "[", "]")
+        .add(initialHeader)
+        .toString();
+  }
+
+  /**
+   * Perform any escaping to valid path elements in advance of
+   * new URI() doing this itself. Only path separators need to
+   * be escaped/converted at this point.
+   * @param source source string
+   * @return an escaped path element.
+   */
+  public static String escapeToPathElement(CharSequence source) {
+    int len = source.length();
+    StringBuilder r = new StringBuilder(len);
+    for (int i = 0; i < len; i++) {
+      char c = source.charAt(i);
+      String s = Character.toString(c);
+      switch (c) {
+      case '/':
+      case '@':
+        s = "+";
+        break;
+      default:
+        break;
+      }
+      r.append(s);
+    }
+    return r.toString();
+
+  }
+
+  /**
+   * Strip any quotes from around a header.
+   * This is needed when processing log entries.
+   * @param header field.
+   * @return field without quotes.
+   */
+  public static String maybeStripWrappedQuotes(String header) {
+    String h = header;
+    // remove quotes if needed.
+    while (h.startsWith("\"")) {
+      h = h.substring(1);
+    }
+    while (h.endsWith("\"")) {
+      h = h.substring(0, h.length() - 1);
+    }
+    return h;
+  }
+
+  /**
+   * Split up the string. Uses httpClient: make sure it is on the classpath.
+   * Any query param with a name but no value, e.g ?something is
+   * returned in the map with an empty string as the value.
+   * @param header URI to parse
+   * @return a map of parameters.
+   * @throws URISyntaxException failure to build URI from header.
+   */
+  public static Map<String, String> extractQueryParameters(String header)
+      throws URISyntaxException {
+    URI uri = new URI(maybeStripWrappedQuotes(header));
+    // get the decoded query
+    List<NameValuePair> params = URLEncodedUtils.parse(uri,
+        StandardCharsets.UTF_8);
+    Map<String, String> result = new HashMap<>(params.size());
+    for (NameValuePair param : params) {
+      String name = param.getName();
+      String value = param.getValue();
+      if (value == null) {
+        value = "";
+      }
+      result.put(name, value);
+    }
+    return result;
+  }
+
+  /**
+   * Get a builder.
+   * @return a new builder.
+   */
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder.
+   *
+   * Context and operationId are expected to be well formed
+   * numeric/hex strings, at least adequate to be
+   * used as individual path elements in a URL.
+   */
+  public static final class Builder {
+
+    /** Context ID. */
+    private String contextId;
+
+    /** operation name. */
+    private String operationName;
+
+    /** operation ID. */
+    private String spanId;
+
+    /** optional first path. */
+    private String path1;
+
+    /** optional second path. */
+    private String path2;
+
+    /** Map of attributes to add as query parameters. */
+    private final Map<String, String> attributes = new HashMap<>();
+
+    /**
+     * Parameters dynamically evaluated on the thread just before
+     * the request is made.
+     */
+    private final Map<String, Supplier<String>> evaluated =
+        new HashMap<>();
+
+    /**
+     * Global context values; defaults to that of
+     * {@link CommonAuditContext#getGlobalContextEntries()} and
+     * should not need to be changed.
+     */
+    private Iterable<Map.Entry<String, String>> globalContextValues =
+        CommonAuditContext.getGlobalContextEntries();
+
+    /**
+     * Elements to filter from the final header.
+     */
+    private Set<String> filter = new HashSet<>();
+
+    private Builder() {
+    }
+
+    /**
+     * Build.
+     * @return an HttpReferrerAuditHeader
+     */
+    public HttpReferrerAuditHeader build() {
+      return new HttpReferrerAuditHeader(this);
+    }
+
+    /**
+     * Set context ID.
+     * @param value context
+     * @return the builder
+     */
+    public Builder withContextId(final String value) {
+      contextId = value;
+      return this;
+    }
+
+    /**
+     * Set Operation name.
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withOperationName(final String value) {
+      operationName = value;
+      return this;
+    }
+
+    /**
+     * Set ID.
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withSpanId(final String value) {
+      spanId = value;
+      return this;
+    }
+
+    /**
+     * Set Path1 of operation.
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withPath1(final String value) {
+      path1 = value;
+      return this;
+    }
+
+    /**
+     * Set Path2 of operation.
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withPath2(final String value) {
+      path2 = value;
+      return this;
+    }
+
+    /**
+     * Add all attributes to the current map.
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withAttributes(final Map<String, String> value) {
+      attributes.putAll(value);
+      return this;
+    }
+
+    /**
+     * Add an attribute to the current map.
+     * Replaces any with the existing key.
+     * @param key key to set/update
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withAttribute(String key, String value) {
+      attributes.put(key, value);
+      return this;
+    }
+
+    /**
+     * Add all evaluated attributes to the current map.
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withEvaluated(final Map<String, Supplier<String>> value) {
+      evaluated.putAll(value);
+      return this;
+    }
+
+    /**
+     * Add an evaluated attribute to the current map.
+     * Replaces any with the existing key.
+     * Set evaluated methods.
+     * @param key key
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withEvaluated(String key, Supplier<String> value) {
+      evaluated.put(key, value);
+      return this;
+    }
+
+    /**
+     * Set the global context values (replaces the default binding
+     * to {@link CommonAuditContext#getGlobalContextEntries()}).
+     * @param value new value
+     * @return the builder
+     */
+    public Builder withGlobalContextValues(
+        final Iterable<Map.Entry<String, String>> value) {
+      globalContextValues = value;
+      return this;
+    }
+
+    /**
+     * Declare the fields to filter.
+     * @param fields iterable of field names.
+     * @return the builder
+     */
+    public Builder withFilter(final Collection<String> fields) {
+      this.filter = new HashSet<>(fields);
+      return this;
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java
similarity index 62%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java
index 54a8836..98fb5b5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/package-info.java
@@ -16,27 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
-
 /**
- * Log exactly once, even across threads.
+ * Auditing classes for internal
+ * use within the hadoop-* modules only. No stability guarantees.
+ * The public/evolving API is in {@code org.apache.hadoop.fs.audit}.
  */
-public class LogExactlyOnce {
-
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
-
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
-  }
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.store.audit;
 
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
-  }
-}
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java
index 903e6ba..208a7fd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java
@@ -35,6 +35,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.ExitCodeProvider;
@@ -590,6 +591,7 @@ public class ServiceLauncher<S extends Service>
     }
     String name = getServiceName();
     LOG.debug("Launched service {}", name);
+    CommonAuditContext.noteEntryPoint(service);
     LaunchableService launchableService = null;
 
     if (service instanceof LaunchableService) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java
index 8740be4..336700a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java
@@ -23,6 +23,7 @@ import java.io.PrintStream;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
 import org.apache.hadoop.ipc.CallerContext;
 
 /**
@@ -63,6 +64,10 @@ public class ToolRunner {
       CallerContext ctx = new CallerContext.Builder("CLI").build();
       CallerContext.setCurrent(ctx);
     }
+    // Note the entry point in the audit context; this
+    // may be used in audit events set to cloud store logs
+    // or elsewhere.
+    CommonAuditContext.noteEntryPoint(tool);
     
     if(conf == null) {
       conf = new Configuration();
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java
index 3ac0fce..5fdea4f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java
@@ -189,6 +189,7 @@ public final class RemoteIterators {
 
   /**
    * Build a list from a RemoteIterator.
+   * @param source source iterator
    * @param <T> type
    * @return a list of the values.
    * @throws IOException if the source RemoteIterator raises it.
@@ -202,12 +203,17 @@ public final class RemoteIterators {
 
   /**
    * Build an array from a RemoteIterator.
+   * @param source source iterator
+   * @param a destination array; if too small a new array
+   * of the same type is created
    * @param <T> type
    * @return an array of the values.
    * @throws IOException if the source RemoteIterator raises it.
    */
-  public static <T> T[] toArray(RemoteIterator<T> source) throws IOException {
-    return (T[]) toList(source).toArray();
+  public static <T> T[] toArray(RemoteIterator<T> source,
+      T[] a) throws IOException {
+    List<T> list = toList(source);
+    return list.toArray(a);
   }
 
   /**
@@ -240,19 +246,29 @@ public final class RemoteIterators {
         consumer.accept(source.next());
       }
 
-      // maybe log the results
-      logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source);
     } finally {
-      if (source instanceof Closeable) {
-        // source is closeable, so close.
-        IOUtils.cleanupWithLogger(LOG, (Closeable) source);
-      }
+      cleanupRemoteIterator(source);
     }
-
     return count;
   }
 
   /**
+   * Clean up after an iteration.
+   * If the log is at debug, calculate and log the IOStatistics.
+   * If the iterator is closeable, cast and then cleanup the iterator
+   * @param source iterator source
+   * @param <T> type of source
+   */
+  public static <T> void cleanupRemoteIterator(RemoteIterator<T> source) {
+    // maybe log the results
+    logIOStatisticsAtDebug(LOG, "RemoteIterator Statistics: {}", source);
+    if (source instanceof Closeable) {
+      /* source is closeable, so close.*/
+      IOUtils.cleanupWithLogger(LOG, (Closeable) source);
+    }
+  }
+
+  /**
    * A remote iterator from a singleton. It has a single next()
    * value, after which hasNext() returns false and next() fails.
    * <p></p>
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java
new file mode 100644
index 0000000..798841a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/audit/TestCommonAuditContext.java
@@ -0,0 +1,161 @@
+/*
+ * 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.audit;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.assertj.core.api.AbstractStringAssert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_COMMAND;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PROCESS;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.PROCESS_ID;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.removeGlobalContextEntry;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.getGlobalContextEntry;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.getGlobalContextEntries;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.noteEntryPoint;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.setGlobalContextEntry;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests of the common audit context.
+ */
+public class TestCommonAuditContext extends AbstractHadoopTestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestCommonAuditContext.class);
+
+  private final CommonAuditContext context = currentAuditContext();
+  /**
+   * We can set, get and enumerate global context values.
+   */
+  @Test
+  public void testGlobalSetGetEnum() throws Throwable {
+
+    String s = "command";
+    setGlobalContextEntry(PARAM_COMMAND, s);
+    assertGlobalEntry(PARAM_COMMAND)
+        .isEqualTo(s);
+    // and the iterators.
+    List<Map.Entry<String, String>> list = StreamSupport
+        .stream(getGlobalContextEntries().spliterator(),
+            false)
+        .filter(e -> e.getKey().equals(PARAM_COMMAND))
+        .collect(Collectors.toList());
+    assertThat(list)
+        .hasSize(1)
+        .allMatch(e -> e.getValue().equals(s));
+  }
+
+  @Test
+  public void testVerifyProcessID() throws Throwable {
+    assertThat(
+        getGlobalContextEntry(PARAM_PROCESS))
+        .describedAs("global context value of %s", PARAM_PROCESS)
+        .isEqualTo(PROCESS_ID);
+  }
+
+
+  @Test
+  public void testNullValue() throws Throwable {
+    assertThat(context.get(PARAM_PROCESS))
+        .describedAs("Value of context element %s", PARAM_PROCESS)
+        .isNull();
+  }
+
+  @Test
+  public void testThreadId() throws Throwable {
+    String t1 = getContextValue(PARAM_THREAD1);
+    Long tid = Long.valueOf(t1);
+    assertThat(tid).describedAs("thread ID")
+        .isEqualTo(Thread.currentThread().getId());
+  }
+
+  /**
+   * Verify functions are dynamically evaluated.
+   */
+  @Test
+  public void testDynamicEval() throws Throwable {
+    context.reset();
+    final AtomicBoolean ab = new AtomicBoolean(false);
+    context.put("key", () ->
+        Boolean.toString(ab.get()));
+    assertContextValue("key")
+        .isEqualTo("false");
+    // update the reference and the next get call will
+    // pick up the new value.
+    ab.set(true);
+    assertContextValue("key")
+        .isEqualTo("true");
+  }
+
+  private String getContextValue(final String key) {
+    String val = context.get(key);
+    assertThat(val).isNotBlank();
+    return val;
+  }
+
+  /**
+   * Start an assertion on a context value.
+   * @param key key to look up
+   * @return an assert which can be extended call
+   */
+  private AbstractStringAssert<?> assertContextValue(final String key) {
+    String val = context.get(key);
+    return assertThat(val)
+        .describedAs("Value of context element %s", key)
+        .isNotBlank();
+  }
+
+  @Test
+  public void testNoteEntryPoint() throws Throwable {
+    setAndAssertEntryPoint(this).isEqualTo("TestCommonAuditContext");
+
+  }
+
+  @Test
+  public void testNoteNullEntryPoint() throws Throwable {
+    setAndAssertEntryPoint(null).isNull();
+  }
+
+  private AbstractStringAssert<?> setAndAssertEntryPoint(final Object tool) {
+    removeGlobalContextEntry(PARAM_COMMAND);
+    noteEntryPoint(tool);
+    AbstractStringAssert<?> anAssert = assertGlobalEntry(
+        PARAM_COMMAND);
+    return anAssert;
+  }
+
+  private AbstractStringAssert<?> assertGlobalEntry(final String key) {
+    AbstractStringAssert<?> anAssert = assertThat(getGlobalContextEntry(key))
+        .describedAs("Global context value %s", key);
+    return anAssert;
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 35193fa..e13a49c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -399,9 +399,7 @@ public class ContractTestUtils extends Assert {
       IOException {
     if (fileSystem != null) {
       rejectRootOperation(path, allowRootDelete);
-      if (fileSystem.exists(path)) {
-        return fileSystem.delete(path, recursive);
-      }
+      return fileSystem.delete(path, recursive);
     }
     return false;
 
@@ -728,8 +726,10 @@ public class ContractTestUtils extends Assert {
       assertPathExists(fs, "about to be deleted file", file);
     }
     boolean deleted = fs.delete(file, recursive);
-    String dir = ls(fs, file.getParent());
-    assertTrue("Delete failed on " + file + ": " + dir, deleted);
+    if (!deleted) {
+      String dir = ls(fs, file.getParent());
+      assertTrue("Delete failed on " + file + ": " + dir, deleted);
+    }
     assertPathDoesNotExist(fs, "Deleted file", file);
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
index 46655bc..2f0cfd3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
@@ -20,12 +20,21 @@ package org.apache.hadoop.fs.s3a;
 
 import com.amazonaws.AmazonClientException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 /**
  * Exception which Hadoop's AWSCredentialsProvider implementations should
  * throw when there is a problem with the credential setup. This
  * is a subclass of {@link AmazonClientException} which sets
  * {@link #isRetryable()} to false, so as to fail fast.
+ * This is used in credential providers and elsewhere.
+ * When passed through {@code S3AUtils.translateException()} it
+ * is mapped to an AccessDeniedException. As a result, the Invoker
+ * code will automatically translate
  */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
 public class CredentialInitializationException extends AmazonClientException {
   public CredentialInitializationException(String message, Throwable t) {
     super(message, t);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
index 19cd6c9..a851f0f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.InvocationRaisingIOE;
 
 /**
  * Class to provide lambda expression invocation of AWS operations.
@@ -43,7 +44,8 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
  * the other {@code retry() and retryUntranslated()} calls are wrappers.
  *
  * The static {@link #once(String, String, CallableRaisingIOE)} and
- * {@link #once(String, String, VoidOperation)} calls take an operation and
+ * {@link #once(String, String, InvocationRaisingIOE)} calls take an
+ * operation and
  * return it with AWS exceptions translated to IOEs of some form.
  *
  * The retry logic on a failure is defined by the retry policy passed in
@@ -57,7 +59,7 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
  * but before the sleep.
  * These callbacks can be used for reporting and incrementing statistics.
  *
- * The static {@link #quietly(String, String, VoidOperation)} and
+ * The static {@link #quietly(String, String, InvocationRaisingIOE)} and
  * {@link #quietlyEval(String, String, CallableRaisingIOE)} calls exist to
  * take any operation and quietly catch and log at debug.
  * The return value of {@link #quietlyEval(String, String, CallableRaisingIOE)}
@@ -126,11 +128,11 @@ public class Invoker {
    * @throws IOException any IOE raised, or translated exception
    */
   @Retries.OnceTranslated
-  public static void once(String action, String path, VoidOperation operation)
-      throws IOException {
+  public static void once(String action, String path,
+      InvocationRaisingIOE operation) throws IOException {
     once(action, path,
         () -> {
-          operation.execute();
+          operation.apply();
           return null;
         });
   }
@@ -171,10 +173,10 @@ public class Invoker {
       Logger log,
       String action,
       String path,
-      VoidOperation operation) {
+      InvocationRaisingIOE operation) {
     ignoreIOExceptions(log, action, path,
         () -> {
-          operation.execute();
+          operation.apply();
           return null;
         });
   }
@@ -194,11 +196,11 @@ public class Invoker {
       String path,
       boolean idempotent,
       Retried retrying,
-      VoidOperation operation)
+      InvocationRaisingIOE operation)
       throws IOException {
     retry(action, path, idempotent, retrying,
         () -> {
-          operation.execute();
+          operation.apply();
           return null;
         });
   }
@@ -221,11 +223,11 @@ public class Invoker {
       String path,
       boolean idempotent,
       Retried retrying,
-      VoidOperation operation)
+      InvocationRaisingIOE operation)
       throws IOException {
     maybeRetry(doRetry, action, path, idempotent, retrying,
         () -> {
-          operation.execute();
+          operation.apply();
           return null;
         });
   }
@@ -243,7 +245,7 @@ public class Invoker {
   public void retry(String action,
       String path,
       boolean idempotent,
-      VoidOperation operation)
+      InvocationRaisingIOE operation)
       throws IOException {
     retry(action, path, idempotent, retryCallback, operation);
   }
@@ -265,7 +267,7 @@ public class Invoker {
       String action,
       String path,
       boolean idempotent,
-      VoidOperation operation)
+      InvocationRaisingIOE operation)
       throws IOException {
     maybeRetry(doRetry, action, path, idempotent, retryCallback, operation);
   }
@@ -475,7 +477,7 @@ public class Invoker {
    */
   public static void quietly(String action,
       String path,
-      VoidOperation operation) {
+      InvocationRaisingIOE operation) {
     try {
       once(action, path, operation);
     } catch (Exception e) {
@@ -516,14 +518,6 @@ public class Invoker {
   }
 
   /**
-   * Void operation which may raise an IOException.
-   */
-  @FunctionalInterface
-  public interface VoidOperation {
-    void execute() throws IOException;
-  }
-
-  /**
    * Callback for retry and notification operations.
    * Even if the interface is throwing up "raw" exceptions, this handler
    * gets the translated one.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
index 3cb3d5d..113e6f4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java
@@ -22,6 +22,7 @@ import javax.annotation.Nullable;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 import org.apache.commons.lang3.tuple.Triple;
@@ -41,10 +42,12 @@ import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
 import org.apache.hadoop.util.functional.RemoteIterators;
 
 import org.slf4j.Logger;
 
+import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.time.Instant;
@@ -79,6 +82,9 @@ import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFr
 
 /**
  * Place for the S3A listing classes; keeps all the small classes under control.
+ *
+ * Spans passed in are attached to the listing iterators returned, but are not
+ * closed at the end of the iteration. This is because the same span
  */
 @InterfaceAudience.Private
 public class Listing extends AbstractStoreOperation {
@@ -137,16 +143,19 @@ public class Listing extends AbstractStoreOperation {
    * @param filter the filter on which paths to accept
    * @param acceptor the class/predicate to decide which entries to accept
    * in the listing based on the full file status.
+   * @param span audit span for this iterator
    * @return the iterator
    * @throws IOException IO Problems
    */
+  @Retries.RetryRaw
   public FileStatusListingIterator createFileStatusListingIterator(
       Path listPath,
       S3ListRequest request,
       PathFilter filter,
-      Listing.FileStatusAcceptor acceptor) throws IOException {
+      Listing.FileStatusAcceptor acceptor,
+      AuditSpan span) throws IOException {
     return createFileStatusListingIterator(listPath, request, filter, acceptor,
-        null);
+        null, span);
   }
 
   /**
@@ -159,6 +168,7 @@ public class Listing extends AbstractStoreOperation {
    * in the listing based on the full file status.
    * @param providedStatus the provided list of file status, which may contain
    *                       items that are not listed from source.
+   * @param span audit span for this iterator
    * @return the iterator
    * @throws IOException IO Problems
    */
@@ -168,9 +178,10 @@ public class Listing extends AbstractStoreOperation {
       S3ListRequest request,
       PathFilter filter,
       Listing.FileStatusAcceptor acceptor,
-      RemoteIterator<S3AFileStatus> providedStatus) throws IOException {
+      RemoteIterator<S3AFileStatus> providedStatus,
+      AuditSpan span) throws IOException {
     return new FileStatusListingIterator(
-        createObjectListingIterator(listPath, request),
+        createObjectListingIterator(listPath, request, span),
         filter,
         acceptor,
         providedStatus);
@@ -181,14 +192,16 @@ public class Listing extends AbstractStoreOperation {
    * list object request.
    * @param listPath path of the listing
    * @param request initial request to make
+   * @param span audit span for this iterator
    * @return the iterator
    * @throws IOException IO Problems
    */
   @Retries.RetryRaw
-  public ObjectListingIterator createObjectListingIterator(
+  private ObjectListingIterator createObjectListingIterator(
       final Path listPath,
-      final S3ListRequest request) throws IOException {
-    return new ObjectListingIterator(listPath, request);
+      final S3ListRequest request,
+      final AuditSpan span) throws IOException {
+    return new ObjectListingIterator(listPath, request, span);
   }
 
   /**
@@ -245,6 +258,7 @@ public class Listing extends AbstractStoreOperation {
    * @param forceNonAuthoritativeMS forces metadata store to act like non
    *                                authoritative. This is useful when
    *                                listFiles output is used by import tool.
+   * @param span audit span for this iterator
    * @return an iterator over listing.
    * @throws IOException any exception.
    */
@@ -252,7 +266,8 @@ public class Listing extends AbstractStoreOperation {
           Path path,
           boolean recursive, Listing.FileStatusAcceptor acceptor,
           boolean collectTombstones,
-          boolean forceNonAuthoritativeMS) throws IOException {
+          boolean forceNonAuthoritativeMS,
+          AuditSpan span) throws IOException {
 
     String key = maybeAddTrailingSlash(pathToKey(path));
     String delimiter = recursive ? null : "/";
@@ -325,10 +340,13 @@ public class Listing extends AbstractStoreOperation {
             createLocatedFileStatusIterator(
                     createFileStatusListingIterator(path,
                                     listingOperationCallbacks
-                                    .createListObjectsRequest(key, delimiter),
+                                    .createListObjectsRequest(key,
+                                        delimiter,
+                                        span),
                             ACCEPT_ALL,
                             acceptor,
-                            cachedFilesIterator)),
+                            cachedFilesIterator,
+                            span)),
             collectTombstones ? tombstones : null);
   }
 
@@ -337,11 +355,13 @@ public class Listing extends AbstractStoreOperation {
    * Also performing tombstone reconciliation for guarded directories.
    * @param dir directory to check.
    * @param filter a path filter.
+   * @param span audit span for this iterator
    * @return an iterator that traverses statuses of the given dir.
    * @throws IOException in case of failure.
    */
   public RemoteIterator<S3ALocatedFileStatus> getLocatedFileStatusIteratorForDir(
-          Path dir, PathFilter filter) throws IOException {
+          Path dir, PathFilter filter, AuditSpan span) throws IOException {
+    span.activate();
     final String key = maybeAddTrailingSlash(pathToKey(dir));
     final Listing.FileStatusAcceptor acceptor =
             new Listing.AcceptAllButSelfAndS3nDirs(dir);
@@ -353,39 +373,55 @@ public class Listing extends AbstractStoreOperation {
                     listingOperationCallbacks
                             .getUpdatedTtlTimeProvider(),
                     allowAuthoritative);
-    Set<Path> tombstones = meta != null
-            ? meta.listTombstones()
-            : null;
-    final RemoteIterator<S3AFileStatus> cachedFileStatusIterator =
-            createProvidedFileStatusIterator(
-                    S3Guard.dirMetaToStatuses(meta), filter, acceptor);
-    return (allowAuthoritative && meta != null
-            && meta.isAuthoritative())
-            ? createLocatedFileStatusIterator(
-            cachedFileStatusIterator)
-            : createTombstoneReconcilingIterator(
+    if (meta != null) {
+      // there's metadata
+      // convert to an iterator
+      final RemoteIterator<S3AFileStatus> cachedFileStatusIterator =
+          createProvidedFileStatusIterator(
+              S3Guard.dirMetaToStatuses(meta), filter, acceptor);
+
+      // if the dir is authoritative and the data considers itself
+      // to be authorititative.
+      if (allowAuthoritative && meta.isAuthoritative()) {
+        // return the list
+        return createLocatedFileStatusIterator(cachedFileStatusIterator);
+      } else {
+        // merge the datasets
+        return createTombstoneReconcilingIterator(
             createLocatedFileStatusIterator(
-                    createFileStatusListingIterator(dir,
-                            listingOperationCallbacks
-                                    .createListObjectsRequest(key, "/"),
-                            filter,
-                            acceptor,
-                            cachedFileStatusIterator)),
-            tombstones);
+                createFileStatusListingIterator(dir,
+                    listingOperationCallbacks
+                        .createListObjectsRequest(key, "/", span),
+                    filter,
+                    acceptor,
+                    cachedFileStatusIterator,
+                    span)),
+            meta.listTombstones());
+      }
+    } else {
+      // Unguarded
+      return createLocatedFileStatusIterator(
+          createFileStatusListingIterator(dir,
+              listingOperationCallbacks
+                  .createListObjectsRequest(key, "/", span),
+              filter,
+              acceptor,
+              span));
+    }
   }
 
   /**
    * Calculate list of file statuses assuming path
    * to be a non-empty directory.
    * @param path input path.
+   * @param span audit span for this iterator
    * @return Triple of file statuses, metaData, auth flag.
    * @throws IOException Any IO problems.
    */
   public Triple<RemoteIterator<S3AFileStatus>, DirListingMetadata, Boolean>
-        getFileStatusesAssumingNonEmptyDir(Path path)
+        getFileStatusesAssumingNonEmptyDir(Path path, final AuditSpan span)
           throws IOException {
     String key = pathToKey(path);
-    List<S3AFileStatus> result;
     if (!key.isEmpty()) {
       key = key + '/';
     }
@@ -408,14 +444,15 @@ public class Listing extends AbstractStoreOperation {
               dirMeta, Boolean.TRUE);
     }
 
-    S3ListRequest request = createListObjectsRequest(key, "/");
+    S3ListRequest request = createListObjectsRequest(key, "/", span);
     LOG.debug("listStatus: doing listObjects for directory {}", key);
 
     FileStatusListingIterator filesItr = createFileStatusListingIterator(
             path,
             request,
             ACCEPT_ALL,
-            new Listing.AcceptAllButSelfAndS3nDirs(path));
+            new Listing.AcceptAllButSelfAndS3nDirs(path),
+            span);
 
     // return the results obtained from s3.
     return Triple.of(
@@ -424,8 +461,11 @@ public class Listing extends AbstractStoreOperation {
             Boolean.FALSE);
   }
 
-  public S3ListRequest createListObjectsRequest(String key, String delimiter) {
-    return listingOperationCallbacks.createListObjectsRequest(key, delimiter);
+  public S3ListRequest createListObjectsRequest(String key,
+      String delimiter,
+      final AuditSpan span) {
+    return listingOperationCallbacks.createListObjectsRequest(key, delimiter,
+        span);
   }
 
   /**
@@ -730,11 +770,13 @@ public class Listing extends AbstractStoreOperation {
    * Thread safety: none.
    */
   class ObjectListingIterator implements RemoteIterator<S3ListResult>,
-      IOStatisticsSource {
+      IOStatisticsSource, Closeable {
 
     /** The path listed. */
     private final Path listPath;
 
+    private final AuditSpan span;
+
     /** The most recent listing results. */
     private S3ListResult objects;
 
@@ -772,12 +814,14 @@ public class Listing extends AbstractStoreOperation {
      * initial set of results/fail if there was a problem talking to the bucket.
      * @param listPath path of the listing
      * @param request initial request to make
+     * @param span audit span for this iterator.
      * @throws IOException if listObjects raises one.
      */
     @Retries.RetryRaw
     ObjectListingIterator(
         Path listPath,
-        S3ListRequest request) throws IOException {
+        S3ListRequest request,
+        AuditSpan span) throws IOException {
       this.listPath = listPath;
       this.maxKeys = listingOperationCallbacks.getMaxKeys();
       this.request = request;
@@ -786,8 +830,9 @@ public class Listing extends AbstractStoreOperation {
           .withDurationTracking(OBJECT_LIST_REQUEST)
           .withDurationTracking(OBJECT_CONTINUE_LIST_REQUEST)
           .build();
+      this.span = span;
       this.s3ListResultFuture = listingOperationCallbacks
-          .listObjectsAsync(request, iostats);
+          .listObjectsAsync(request, iostats, span);
     }
 
     /**
@@ -851,7 +896,7 @@ public class Listing extends AbstractStoreOperation {
         LOG.debug("[{}], Requesting next {} objects under {}",
                 listingCount, maxKeys, listPath);
         s3ListResultFuture = listingOperationCallbacks
-                .continueListObjectsAsync(request, objects, iostats);
+                .continueListObjectsAsync(request, objects, iostats, span);
       }
     }
 
@@ -883,6 +928,14 @@ public class Listing extends AbstractStoreOperation {
     public int getListingCount() {
       return listingCount;
     }
+
+    /**
+     * Close, if actually called, will close the span
+     * this listing was created with.
+     */
+    @Override
+    public void close() {
+    }
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
index 6eb490f..d8c820c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
@@ -31,11 +31,20 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.api.RequestFactory;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_LIST;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
 
 
 /**
  * MultipartUtils upload-specific functions for use by S3AFileSystem and Hadoop
  * CLI.
+ * The Audit span active when
+ * {@link #listMultipartUploads(StoreContext, AmazonS3, String, int)}
+ * was invoked is retained for all subsequent operations.
  */
 public final class MultipartUtils {
 
@@ -48,33 +57,47 @@ public final class MultipartUtils {
   /**
    * List outstanding multipart uploads.
    * Package private: S3AFileSystem and tests are the users of this.
+   *
+   * @param storeContext store context
    * @param s3 AmazonS3 client to use.
-   * @param bucketName name of S3 bucket to use.
-   * @param maxKeys maximum batch size to request at a time from S3.
    * @param prefix optional key prefix to narrow search.  If null then whole
    *               bucket will be searched.
+   * @param maxKeys maximum batch size to request at a time from S3.
    * @return an iterator of matching uploads
    */
-  static MultipartUtils.UploadIterator listMultipartUploads(AmazonS3 s3,
-      Invoker invoker, String bucketName, int maxKeys, @Nullable String prefix)
+  static MultipartUtils.UploadIterator listMultipartUploads(
+      final StoreContext storeContext,
+      AmazonS3 s3,
+      @Nullable String prefix,
+      int maxKeys)
       throws IOException {
-    return new MultipartUtils.UploadIterator(s3, invoker, bucketName, maxKeys,
+    return new MultipartUtils.UploadIterator(storeContext,
+        s3,
+        maxKeys,
         prefix);
   }
 
   /**
    * Simple RemoteIterator wrapper for AWS `listMultipartUpload` API.
    * Iterates over batches of multipart upload metadata listings.
+   * All requests are in the StoreContext's active span
+   * at the time the iterator was constructed.
    */
   static class ListingIterator implements
       RemoteIterator<MultipartUploadListing> {
 
-    private final String bucketName;
     private final String prefix;
+
+    private final RequestFactory requestFactory;
+
     private final int maxKeys;
     private final AmazonS3 s3;
     private final Invoker invoker;
 
+    private final AuditSpan auditSpan;
+
+    private final StoreContext storeContext;
+
     /**
      * Most recent listing results.
      */
@@ -85,16 +108,24 @@ public final class MultipartUtils {
      */
     private boolean firstListing = true;
 
-    private int listCount = 1;
+    /**
+     * Count of list calls made.
+     */
+    private int listCount = 0;
 
-    ListingIterator(AmazonS3 s3, Invoker invoker, String bucketName,
-        int maxKeys, @Nullable String prefix) throws IOException {
+    ListingIterator(final StoreContext storeContext,
+        AmazonS3 s3,
+        @Nullable String prefix,
+        int maxKeys) throws IOException {
+      this.storeContext = storeContext;
       this.s3 = s3;
-      this.bucketName = bucketName;
+      this.requestFactory = storeContext.getRequestFactory();
       this.maxKeys = maxKeys;
       this.prefix = prefix;
-      this.invoker = invoker;
+      this.invoker = storeContext.getInvoker();
+      this.auditSpan = storeContext.getActiveAuditSpan();
 
+      // request the first listing.
       requestNextBatch();
     }
 
@@ -138,31 +169,36 @@ public final class MultipartUtils {
 
     @Override
     public String toString() {
-      return "Upload iterator: prefix " + prefix + "; list count " +
-          listCount + "; isTruncated=" + listing.isTruncated();
+      return "Upload iterator: prefix " + prefix
+          + "; list count " + listCount
+          + "; upload count " + listing.getMultipartUploads().size()
+          + "; isTruncated=" + listing.isTruncated();
     }
 
     @Retries.RetryTranslated
     private void requestNextBatch() throws IOException {
-      ListMultipartUploadsRequest req =
-          new ListMultipartUploadsRequest(bucketName);
-      if (prefix != null) {
-        req.setPrefix(prefix);
-      }
-      if (!firstListing) {
-        req.setKeyMarker(listing.getNextKeyMarker());
-        req.setUploadIdMarker(listing.getNextUploadIdMarker());
-      }
-      req.setMaxUploads(listCount);
+      try (AuditSpan span = auditSpan.activate()) {
+        ListMultipartUploadsRequest req = requestFactory
+            .newListMultipartUploadsRequest(prefix);
+        if (!firstListing) {
+          req.setKeyMarker(listing.getNextKeyMarker());
+          req.setUploadIdMarker(listing.getNextUploadIdMarker());
+        }
+        req.setMaxUploads(maxKeys);
 
-      LOG.debug("[{}], Requesting next {} uploads prefix {}, " +
-          "next key {}, next upload id {}", listCount, maxKeys, prefix,
-          req.getKeyMarker(), req.getUploadIdMarker());
-      listCount++;
+        LOG.debug("[{}], Requesting next {} uploads prefix {}, " +
+            "next key {}, next upload id {}", listCount, maxKeys, prefix,
+            req.getKeyMarker(), req.getUploadIdMarker());
+        listCount++;
 
-      listing = invoker.retry("listMultipartUploads", prefix, true,
-          () -> s3.listMultipartUploads(req));
-      LOG.debug("New listing state: {}", this);
+        listing = invoker.retry("listMultipartUploads", prefix, true,
+            trackDurationOfOperation(storeContext.getInstrumentation(),
+                MULTIPART_UPLOAD_LIST.getSymbol(),
+                () -> s3.listMultipartUploads(req)));
+        LOG.debug("Listing found {} upload(s)",
+            listing.getMultipartUploads().size());
+        LOG.debug("New listing state: {}", this);
+      }
     }
   }
 
@@ -174,6 +210,10 @@ public final class MultipartUtils {
   public static class UploadIterator
       implements RemoteIterator<MultipartUpload> {
 
+    /**
+     * Iterator for issuing new upload list requests from
+     * where the previous one ended.
+     */
     private ListingIterator lister;
     /** Current listing: the last upload listing we fetched. */
     private MultipartUploadListing listing;
@@ -181,11 +221,15 @@ public final class MultipartUtils {
     private ListIterator<MultipartUpload> batchIterator;
 
     @Retries.RetryTranslated
-    public UploadIterator(AmazonS3 s3, Invoker invoker, String bucketName,
-        int maxKeys, @Nullable String prefix)
+    public UploadIterator(
+        final StoreContext storeContext,
+        AmazonS3 s3,
+        int maxKeys,
+        @Nullable String prefix)
         throws IOException {
 
-      lister = new ListingIterator(s3, invoker, bucketName, maxKeys, prefix);
+      lister = new ListingIterator(storeContext, s3, prefix,
+          maxKeys);
       requestNextBatch();
     }
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 65b9535..5ba39aa 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.amazonaws.SdkBaseException;
 import com.amazonaws.event.ProgressEvent;
 import com.amazonaws.event.ProgressEventType;
 import com.amazonaws.event.ProgressListener;
@@ -55,12 +56,12 @@ import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.PutTracker;
-import org.apache.hadoop.fs.s3a.impl.LogExactlyOnce;
 import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
 import org.apache.hadoop.util.Progressable;
 
 import static java.util.Objects.requireNonNull;
@@ -134,6 +135,8 @@ class S3ABlockOutputStream extends OutputStream implements
 
   /**
    * Write operation helper; encapsulation of the filesystem operations.
+   * This contains the audit span for the operation, and activates/deactivates
+   * it within calls.
    */
   private final WriteOperations writeOperationHelper;
 
@@ -393,6 +396,7 @@ class S3ABlockOutputStream extends OutputStream implements
         final List<PartETag> partETags =
             multiPartUpload.waitForAllPartUploads();
         bytes = bytesSubmitted;
+
         // then complete the operation
         if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
             partETags,
@@ -777,6 +781,12 @@ class S3ABlockOutputStream extends OutputStream implements
             uploadData.getUploadStream(),
             uploadData.getFile(),
             0L);
+      } catch (SdkBaseException aws) {
+        // catch and translate
+        IOException e = translateException("upload", key, aws);
+        // failure to start the upload.
+        noteUploadFailure(e);
+        throw e;
       } catch (IOException e) {
         // failure to start the upload.
         noteUploadFailure(e);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index e2402de..d9e9fd6 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.fs.s3a;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.net.URI;
 import java.nio.file.AccessDeniedException;
@@ -53,12 +52,12 @@ import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.SdkBaseException;
 import com.amazonaws.services.s3.AmazonS3;
-import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
 import com.amazonaws.services.s3.model.CannedAccessControlList;
 import com.amazonaws.services.s3.model.CopyObjectRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsRequest;
 import com.amazonaws.services.s3.model.DeleteObjectsResult;
 import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
 import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
@@ -69,8 +68,7 @@ import com.amazonaws.services.s3.model.MultipartUpload;
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectResult;
-import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
-import com.amazonaws.services.s3.model.SSECustomerKey;
+import com.amazonaws.services.s3.model.S3Object;
 import com.amazonaws.services.s3.model.UploadPartRequest;
 import com.amazonaws.services.s3.model.UploadPartResult;
 import com.amazonaws.services.s3.transfer.Copy;
@@ -80,6 +78,9 @@ import com.amazonaws.services.s3.transfer.Upload;
 import com.amazonaws.services.s3.transfer.model.CopyResult;
 import com.amazonaws.services.s3.transfer.model.UploadResult;
 import com.amazonaws.event.ProgressListener;
+
+import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
@@ -91,11 +92,13 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonPathCapabilities;
+import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Globber;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.s3a.auth.SignerManager;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
@@ -106,12 +109,15 @@ import org.apache.hadoop.fs.s3a.impl.CopyOutcome;
 import org.apache.hadoop.fs.s3a.impl.DeleteOperation;
 import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy;
 import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl;
+import org.apache.hadoop.fs.s3a.impl.GetContentSummaryOperation;
 import org.apache.hadoop.fs.s3a.impl.HeaderProcessing;
 import org.apache.hadoop.fs.s3a.impl.InternalConstants;
 import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks;
+import org.apache.hadoop.fs.s3a.impl.MkdirOperation;
 import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
 import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
 import org.apache.hadoop.fs.s3a.impl.RenameOperation;
+import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl;
 import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder;
 import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
 import org.apache.hadoop.fs.s3a.impl.StoreContext;
@@ -123,9 +129,14 @@ import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
 import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.store.audit.AuditEntryPoint;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.fs.store.audit.AuditSpanSource;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.DelegationTokenIssuer;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.DurationInfo;
@@ -144,9 +155,12 @@ import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.s3a.api.RequestFactory;
+import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A;
+import org.apache.hadoop.fs.s3a.audit.AuditIntegration;
+import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
 import org.apache.hadoop.fs.s3a.auth.RoleModel;
 import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider;
-import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations;
 import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
 import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
 import org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier;
@@ -175,6 +189,7 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
@@ -186,7 +201,7 @@ import static org.apache.hadoop.fs.s3a.Invoker.*;
 import static org.apache.hadoop.fs.s3a.Listing.toLocatedFileStatusIterator;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
-import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.INITIALIZE_SPAN;
 import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW;
 import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
 import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
@@ -197,7 +212,10 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
 import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT;
 import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
 import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup;
 import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.dirMetaToStatuses;
@@ -205,8 +223,10 @@ import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatistic
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator;
 
@@ -226,22 +246,14 @@ import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemot
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AFileSystem extends FileSystem implements StreamCapabilities,
-    AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource {
+    AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource,
+    AuditSpanSource<AuditSpanS3A>, ActiveThreadSpanSource<AuditSpanS3A> {
+
   /**
    * Default blocksize as used in blocksize and FS status queries.
    */
   public static final int DEFAULT_BLOCKSIZE = 32 * 1024 * 1024;
 
-  /**
-   * This declared delete as idempotent.
-   * This is an "interesting" topic in past Hadoop FS work.
-   * Essentially: with a single caller, DELETE is idempotent
-   * but in a shared filesystem, it is is very much not so.
-   * Here, on the basis that isn't a filesystem with consistency guarantees,
-   * retryable results in files being deleted.
-  */
-  public static final boolean DELETE_CONSIDERED_IDEMPOTENT = true;
-
   private URI uri;
   private Path workingDir;
   private String username;
@@ -304,7 +316,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private S3ADataBlocks.BlockFactory blockFactory;
   private int blockOutputActiveBlocks;
   private WriteOperationHelper writeHelper;
-  private SelectBinding selectBinding;
   private boolean useListV1;
   private MagicCommitIntegration committerIntegration;
 
@@ -318,12 +329,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   private int pageSize;
 
-  /**
-   * Specific operations used by rename and delete operations.
-   */
-  private final S3AFileSystem.OperationCallbacksImpl
-      operationCallbacks = new OperationCallbacksImpl();
-
   private final ListingOperationCallbacks listingOperationCallbacks =
           new ListingOperationCallbacksImpl();
   /**
@@ -332,14 +337,24 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private DirectoryPolicy directoryPolicy;
 
   /**
-   * Header processing for XAttr.
+   * Context accessors for re-use.
    */
-  private HeaderProcessing headerProcessing;
+  private final ContextAccessors contextAccessors = new ContextAccessorsImpl();
 
   /**
-   * Context accessors for re-use.
+   * Factory for AWS requests.
    */
-  private final ContextAccessors contextAccessors = new ContextAccessorsImpl();
+  private RequestFactory requestFactory;
+
+  /**
+   * Audit manager (service lifecycle).
+   * Creates the audit service and manages the binding of different audit spans
+   * to different threads.
+   * Initially this is a no-op manager; once the service is initialized it will
+   * be replaced with a configured one.
+   */
+  private AuditManagerS3A auditManager =
+      AuditIntegration.stubAuditManager();
 
   /** Add any deprecated keys. */
   @SuppressWarnings("deprecation")
@@ -371,6 +386,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       throws IOException {
     // get the host; this is guaranteed to be non-null, non-empty
     bucket = name.getHost();
+    AuditSpan span = null;
     try {
       LOG.debug("Initializing S3AFileSystem for {}", bucket);
       // clone the configuration into one with propagated bucket options
@@ -411,8 +427,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
       s3guardInvoker = new Invoker(new S3GuardExistsRetryPolicy(getConf()),
           onRetry);
-      writeHelper = new WriteOperationHelper(this, getConf(),
-          statisticsContext);
 
       failOnMetadataWriteError = conf.getBoolean(FAIL_ON_METADATA_WRITE_ERROR,
           FAIL_ON_METADATA_WRITE_ERROR_DEFAULT);
@@ -442,6 +456,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       signerManager = new SignerManager(bucket, this, conf, owner);
       signerManager.initCustomSigners();
 
+      // start auditing
+      initializeAuditService();
+
+      // create the requestFactory.
+      // requires the audit manager to be initialized.
+      requestFactory = createRequestFactory();
+
+      // create the static write operation helper.
+      // this doesn't have a short-lived span; auditors which
+      // require one may reject usages.
+      writeHelper = createWriteOperationHelper(getActiveAuditSpan());
+
+      // create an initial span for all other operations.
+      span = createSpan(INITIALIZE_SPAN, bucket, null);
+
       // creates the AWS client, including overriding auth chain if
       // the FS came with a DT
       // this may do some patching of the configuration (e.g. setting
@@ -467,11 +496,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           magicCommitterEnabled ? "is" : "is not");
       committerIntegration = new MagicCommitIntegration(
           this, magicCommitterEnabled);
-      // header processing for rename and magic committer
-      headerProcessing = new HeaderProcessing(createStoreContext());
-
-      // instantiate S3 Select support
-      selectBinding = new SelectBinding(writeHelper);
 
       boolean blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, true);
 
@@ -519,10 +543,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       listing = new Listing(listingOperationCallbacks, createStoreContext());
     } catch (AmazonClientException e) {
       // amazon client exception: stop all services then throw the translation
+      cleanupWithLogger(LOG, span);
       stopAllServices();
       throw translateException("initializing ", new Path(name), e);
     } catch (IOException | RuntimeException e) {
       // other exceptions: stop the services.
+      cleanupWithLogger(LOG, span);
       stopAllServices();
       throw e;
     }
@@ -602,6 +628,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param conf configuration.
    */
   private void initThreadPools(Configuration conf) {
+    final String name = "s3a-transfer-" + getBucket();
     int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
     if (maxThreads < 2) {
       LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2.");
@@ -615,13 +642,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         maxThreads,
         maxThreads + totalTasks,
         keepAliveTime, TimeUnit.SECONDS,
-        "s3a-transfer-shared");
+        name + "-bounded");
     unboundedThreadPool = new ThreadPoolExecutor(
         maxThreads, Integer.MAX_VALUE,
         keepAliveTime, TimeUnit.SECONDS,
         new LinkedBlockingQueue<>(),
         BlockingThreadPoolExecutorService.newDaemonThreadFactory(
-            "s3a-transfer-unbounded"));
+            name + "-unbounded"));
     unboundedThreadPool.allowCoreThreadTimeOut(true);
     executorCapacity = intOption(conf,
         EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1);
@@ -651,7 +678,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   protected void verifyBucketExists()
       throws UnknownStoreException, IOException {
     if (!invoker.retry("doesBucketExist", bucket, true,
-        () -> s3.doesBucketExist(bucket))) {
+        trackDurationOfOperation(getDurationTrackerFactory(),
+            STORE_EXISTS_PROBE.getSymbol(),
+            () -> s3.doesBucketExist(bucket)))) {
       throw new UnknownStoreException("Bucket " + bucket + " does not exist");
     }
   }
@@ -667,7 +696,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   protected void verifyBucketExistsV2()
           throws UnknownStoreException, IOException {
     if (!invoker.retry("doesBucketExistV2", bucket, true,
-        () -> s3.doesBucketExistV2(bucket))) {
+        trackDurationOfOperation(getDurationTrackerFactory(),
+            STORE_EXISTS_PROBE.getSymbol(),
+            () -> s3.doesBucketExistV2(bucket)))) {
       throw new UnknownStoreException("Bucket " + bucket + " does not exist");
     }
   }
@@ -750,7 +781,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         .withEndpoint(conf.getTrimmed(ENDPOINT, DEFAULT_ENDPOINT))
         .withMetrics(statisticsContext.newStatisticsFromAwsSdk())
         .withPathStyleAccess(conf.getBoolean(PATH_STYLE_ACCESS, false))
-        .withUserAgentSuffix(uaSuffix);
+        .withUserAgentSuffix(uaSuffix)
+        .withRequestHandlers(auditManager.createRequestHandlers());
 
     s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
         .createS3Client(getUri(),
@@ -758,6 +790,109 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
+   * Initialize and launch the audit manager and service.
+   * As this takes the FS IOStatistics store, it must be invoked
+   * after instrumentation is initialized.
+   * @throws IOException failure to instantiate/initialize.
+   */
+  protected void initializeAuditService() throws IOException {
+    auditManager = AuditIntegration.createAndStartAuditManager(
+        getConf(),
+        instrumentation.createMetricsUpdatingStore());
+  }
+
+  /**
+   * The audit manager.
+   * @return the audit manager
+   */
+  @InterfaceAudience.Private
+  public AuditManagerS3A getAuditManager() {
+    return auditManager;
+  }
+
+  /**
+   * Get the auditor; valid once initialized.
+   * @return the auditor.
+   */
+  @InterfaceAudience.Private
+  public OperationAuditor getAuditor() {
+    return getAuditManager().getAuditor();
+  }
+
+  /**
+   * Get the active audit span.
+   * @return the span.
+   */
+  @InterfaceAudience.Private
+  @Override
+  public AuditSpanS3A getActiveAuditSpan() {
+    return getAuditManager().getActiveAuditSpan();
+  }
+
+  /**
+   * Get the audit span source; allows for components like the committers
+   * to have a source of spans without being hard coded to the FS only.
+   * @return the source of spans -base implementation is this instance.
+   */
+  @InterfaceAudience.Private
+  public AuditSpanSource getAuditSpanSource() {
+    return this;
+  }
+
+  /**
+   * Start an operation; this informs the audit service of the event
+   * and then sets it as the active span.
+   * @param operation operation name.
+   * @param path1 first path of operation
+   * @param path2 second path of operation
+   * @return a span for the audit
+   * @throws IOException failure
+   */
+  public AuditSpanS3A createSpan(String operation,
+      @Nullable String path1,
+      @Nullable String path2)
+      throws IOException {
+
+    return getAuditManager().createSpan(operation, path1, path2);
+  }
+
+  /**
+   * Build the request factory.
+   * MUST be called after reading encryption secrets from settings/
+   * delegation token.
+   * Protected, in case test/mock classes want to implement their
+   * own variants.
+   * @return request factory.
+   */
+  protected RequestFactory createRequestFactory() {
+    long partCountLimit = longOption(getConf(),
+        UPLOAD_PART_COUNT_LIMIT,
+        DEFAULT_UPLOAD_PART_COUNT_LIMIT,
+        1);
+    if (partCountLimit != DEFAULT_UPLOAD_PART_COUNT_LIMIT) {
+      LOG.warn("Configuration property {} shouldn't be overridden by client",
+          UPLOAD_PART_COUNT_LIMIT);
+    }
+
+    return RequestFactoryImpl.builder()
+        .withBucket(requireNonNull(bucket))
+        .withCannedACL(getCannedACL())
+        .withEncryptionSecrets(requireNonNull(encryptionSecrets))
+        .withMultipartPartCountLimit(partCountLimit)
+        .withRequestPreparer(getAuditManager()::requestCreated)
+        .build();
+  }
+
+  /**
+   * Get the request factory which uses this store's audit span.
+   * @return the request factory.
+   */
+  @VisibleForTesting
+  public RequestFactory getRequestFactory() {
+    return requestFactory;
+  }
+
+  /**
    * Implementation of all operations used by delegation tokens.
    */
   private class DelegationOperationsImpl implements DelegationOperations {
@@ -783,6 +918,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   protected void setEncryptionSecrets(final EncryptionSecrets secrets) {
     this.encryptionSecrets = secrets;
+    if (requestFactory != null) {
+      requestFactory.setEncryptionSecrets(secrets);
+    }
   }
 
   /**
@@ -936,11 +1074,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Get the region of a bucket.
+   * Invoked from StoreContext; consider an entry point.
    * @return the region in which a bucket is located
    * @throws AccessDeniedException if the caller lacks permission.
    * @throws IOException on any failure.
    */
   @Retries.RetryTranslated
+  @InterfaceAudience.LimitedPrivate("diagnostics")
   public String getBucketLocation() throws IOException {
     return getBucketLocation(bucket);
   }
@@ -957,10 +1097,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws IOException on any failure.
    */
   @VisibleForTesting
+  @AuditEntryPoint
   @Retries.RetryTranslated
   public String getBucketLocation(String bucketName) throws IOException {
-    final String region = invoker.retry("getBucketLocation()", bucketName, true,
-        () -> s3.getBucketLocation(bucketName));
+    final String region = trackDurationAndSpan(
+        STORE_EXISTS_PROBE, bucketName, null, () ->
+            invoker.retry("getBucketLocation()", bucketName, true, () ->
+                s3.getBucketLocation(bucketName)));
     return fixBucketRegion(region);
   }
 
@@ -1192,14 +1335,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws IOException IO failure.
    */
   @Retries.RetryTranslated
+  @AuditEntryPoint
   private FSDataInputStream open(
       final Path file,
       final Optional<Configuration> options,
       final Optional<S3AFileStatus> providedStatus)
       throws IOException {
 
-    entryPoint(INVOCATION_OPEN);
     final Path path = qualify(file);
+    // this span is passed into the stream.
+    final AuditSpan auditSpan = entryPoint(INVOCATION_OPEN, path);
     S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path,
         providedStatus);
 
@@ -1217,13 +1362,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           fileStatus,
           policy,
           changeDetectionPolicy,
-          readAheadRange2);
+          readAheadRange2,
+          auditSpan);
     } else {
       readContext = createReadContext(
           fileStatus,
           inputPolicy,
           changeDetectionPolicy,
-          readAhead);
+          readAhead,
+          auditSpan);
     }
     LOG.debug("Opening '{}'", readContext);
 
@@ -1231,7 +1378,60 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         new S3AInputStream(
             readContext,
             createObjectAttributes(fileStatus),
-            s3));
+            createInputStreamCallbacks(auditSpan)));
+  }
+
+  /**
+   * Override point: create the callbacks for S3AInputStream.
+   * @return an implementation of the InputStreamCallbacks,
+   */
+  private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks(
+      final AuditSpan auditSpan) {
+    return new InputStreamCallbacksImpl(auditSpan);
+  }
+
+  /**
+   * Operations needed by S3AInputStream to read data.
+   */
+  private final class InputStreamCallbacksImpl implements
+      S3AInputStream.InputStreamCallbacks {
+
+    /**
+     * Audit span to activate before each call.
+     */
+    private final AuditSpan auditSpan;
+
+    /**
+     * Create.
+     * @param auditSpan Audit span to activate before each call.
+     */
+    private InputStreamCallbacksImpl(final AuditSpan auditSpan) {
+      this.auditSpan = requireNonNull(auditSpan);
+    }
+
+    /**
+     * Closes the audit span.
+     */
+    @Override
+    public void close()  {
+      auditSpan.close();
+    }
+
+    @Override
+    public GetObjectRequest newGetRequest(final String key) {
+      // active the audit span used for the operation
+      try (AuditSpan span = auditSpan.activate()) {
+        return getRequestFactory().newGetObjectRequest(key);
+      }
+    }
+
+    @Override
+    public S3Object getObject(GetObjectRequest request) {
+      // active the audit span used for the operation
+      try (AuditSpan span = auditSpan.activate()) {
+        return s3.getObject(request);
+      }
+    }
   }
 
   /**
@@ -1246,7 +1446,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       final FileStatus fileStatus,
       final S3AInputPolicy seekPolicy,
       final ChangeDetectionPolicy changePolicy,
-      final long readAheadRange) {
+      final long readAheadRange,
+      final AuditSpan auditSpan) {
     return new S3AReadOpContext(fileStatus.getPath(),
         hasMetadataStore(),
         invoker,
@@ -1256,7 +1457,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         fileStatus,
         seekPolicy,
         changePolicy,
-        readAheadRange);
+        readAheadRange,
+        auditSpan);
   }
 
   /**
@@ -1313,12 +1515,43 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @see #setPermission(Path, FsPermission)
    */
   @Override
+  @AuditEntryPoint
   @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
   public FSDataOutputStream create(Path f, FsPermission permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
-    entryPoint(INVOCATION_CREATE);
     final Path path = qualify(f);
+    // the span will be picked up inside the output stream
+    return trackDurationAndSpan(INVOCATION_CREATE, path, () ->
+        innerCreateFile(path, permission, overwrite, bufferSize, replication,
+            blockSize, progress));
+  }
+
+  /**
+   * Create an FSDataOutputStream at the indicated Path with write-progress
+   * reporting; in the active span.
+   * Retry policy: retrying, translated on the getFileStatus() probe.
+   * No data is uploaded to S3 in this call, so no retry issues related to that.
+   * @param path the file name to open
+   * @param permission the permission to set.
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @param bufferSize the size of the buffer to be used.
+   * @param replication required block replication for the file.
+   * @param blockSize the requested block size.
+   * @param progress the progress reporter.
+   * @throws IOException in the event of IO related errors.
+   * @see #setPermission(Path, FsPermission)
+   */
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+  @Retries.RetryTranslated
+  private FSDataOutputStream innerCreateFile(Path path,
+      FsPermission permission,
+      boolean overwrite,
+      int bufferSize,
+      short replication,
+      long blockSize,
+      Progressable progress) throws IOException {
     String key = pathToKey(path);
     FileStatus status = null;
     try {
@@ -1359,7 +1592,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         .withStatistics(outputStreamStatistics)
         .withProgress(progress)
         .withPutTracker(putTracker)
-        .withWriteOperations(getWriteOperationHelper())
+        .withWriteOperations(
+            createWriteOperationHelper(getActiveAuditSpan()))
         .withExecutorService(
             new SemaphoredDelegatingExecutor(
                 boundedThreadPool,
@@ -1375,7 +1609,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Get a {@code WriteOperationHelper} instance.
+   * Create a Write Operation Helper with the current active span.
+   * All operations made through this helper will activate the
+   * span before execution.
    *
    * This class permits other low-level operations against the store.
    * It is unstable and
@@ -1385,7 +1621,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   @InterfaceAudience.Private
   public WriteOperationHelper getWriteOperationHelper() {
-    return writeHelper;
+    return createWriteOperationHelper(getActiveAuditSpan());
+  }
+
+  /**
+   * Create a Write Operation Helper with the given span.
+   * All operations made through this helper will activate the
+   * span before execution.
+   * @param auditSpan audit span
+   * @return a new helper.
+   */
+  @InterfaceAudience.Private
+  public WriteOperationHelper createWriteOperationHelper(AuditSpan auditSpan) {
+    return new WriteOperationHelper(this,
+        getConf(),
+        statisticsContext,
+        getAuditSpanSource(),
+        auditSpan);
   }
 
   /**
@@ -1394,6 +1646,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * is not a directory.
    */
   @Override
+  @AuditEntryPoint
   public FSDataOutputStream createNonRecursive(Path p,
       FsPermission permission,
       EnumSet<CreateFlag> flags,
@@ -1401,29 +1654,31 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       short replication,
       long blockSize,
       Progressable progress) throws IOException {
-    entryPoint(INVOCATION_CREATE_NON_RECURSIVE);
     final Path path = makeQualified(p);
-    Path parent = path.getParent();
-    // expect this to raise an exception if there is no parent dir
-    if (parent != null && !parent.isRoot()) {
-      S3AFileStatus status;
-      try {
-        // optimize for the directory existing: Call list first
-        status = innerGetFileStatus(parent, false,
-            StatusProbeEnum.DIRECTORIES);
-      } catch (FileNotFoundException e) {
-        // no dir, fall back to looking for a file
-        // (failure condition if true)
-        status = innerGetFileStatus(parent, false,
-            StatusProbeEnum.HEAD_ONLY);
-      }
-      if (!status.isDirectory()) {
-        throw new FileAlreadyExistsException("Not a directory: " + parent);
+    // this span is passed into the stream.
+    try (AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, path)) {
+      Path parent = path.getParent();
+      // expect this to raise an exception if there is no parent dir
+      if (parent != null && !parent.isRoot()) {
+        S3AFileStatus status;
+        try {
+          // optimize for the directory existing: Call list first
+          status = innerGetFileStatus(parent, false,
+              StatusProbeEnum.DIRECTORIES);
+        } catch (FileNotFoundException e) {
+          // no dir, fall back to looking for a file
+          // (failure condition if true)
+          status = innerGetFileStatus(parent, false,
+              StatusProbeEnum.HEAD_ONLY);
+        }
+        if (!status.isDirectory()) {
+          throw new FileAlreadyExistsException("Not a directory: " + parent);
+        }
       }
+      return innerCreateFile(path, permission,
+          flags.contains(CreateFlag.OVERWRITE), bufferSize,
+          replication, blockSize, progress);
     }
-    return create(path, permission,
-        flags.contains(CreateFlag.OVERWRITE), bufferSize,
-        replication, blockSize, progress);
   }
 
   /**
@@ -1461,11 +1716,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws IOException on IO failure
    * @return true if rename is successful
    */
+  @AuditEntryPoint
   @Retries.RetryTranslated
   public boolean rename(Path src, Path dst) throws IOException {
-    try (DurationInfo ignored = new DurationInfo(LOG, false,
-        "rename(%s, %s", src, dst)) {
-      long bytesCopied = innerRename(src, dst);
+    try {
+      long bytesCopied = trackDurationAndSpan(
+          INVOCATION_RENAME, src.toString(), dst.toString(), () ->
+          innerRename(src, dst));
       LOG.debug("Copied {} bytes", bytesCopied);
       return true;
     } catch (AmazonClientException e) {
@@ -1603,7 +1860,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     Path dst = qualify(dest);
 
     LOG.debug("Rename path {} to {}", src, dst);
-    entryPoint(INVOCATION_RENAME);
 
     String srcKey = pathToKey(src);
     String dstKey = pathToKey(dst);
@@ -1617,7 +1873,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         createStoreContext(),
         src, srcKey, p.getLeft(),
         dst, dstKey, p.getRight(),
-        operationCallbacks,
+        new OperationCallbacksImpl(),
         pageSize);
     return renameOperation.execute();
   }
@@ -1631,8 +1887,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * The callbacks made by the rename and delete operations.
    * This separation allows the operation to be factored out and
    * still avoid knowledge of the S3AFilesystem implementation.
+   * The Audit span active at the time of creation is cached and activated
+   * before every call.
    */
-  private class OperationCallbacksImpl implements OperationCallbacks {
+  private final class OperationCallbacksImpl implements OperationCallbacks {
+
+    /** Audit Span at time of creation. */
+    private final AuditSpan auditSpan;
+
+    private OperationCallbacksImpl() {
+      auditSpan = getActiveAuditSpan();
+    }
 
     @Override
     public S3ObjectAttributes createObjectAttributes(final Path path,
@@ -1653,7 +1918,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     public S3AReadOpContext createReadContext(final FileStatus fileStatus) {
       return S3AFileSystem.this.createReadContext(fileStatus,
           inputPolicy,
-          changeDetectionPolicy, readAhead);
+          changeDetectionPolicy, readAhead,
+          auditSpan);
     }
 
     @Override
@@ -1663,6 +1929,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         final boolean isFile,
         final BulkOperationState operationState)
         throws IOException {
+      auditSpan.activate();
       once("delete", path.toString(), () ->
           S3AFileSystem.this.deleteObjectAtPath(path, key, isFile,
               operationState));
@@ -1675,6 +1942,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         final S3AFileStatus status,
         final boolean collectTombstones,
         final boolean includeSelf) throws IOException {
+      auditSpan.activate();
       return innerListFiles(
           path,
           true,
@@ -1691,6 +1959,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         final String destKey,
         final S3ObjectAttributes srcAttributes,
         final S3AReadOpContext readContext) throws IOException {
+      auditSpan.activate();
       return S3AFileSystem.this.copyFile(srcKey, destKey,
           srcAttributes.getLen(), srcAttributes, readContext);
     }
@@ -1703,6 +1972,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         final BulkOperationState operationState,
         final boolean quiet)
         throws MultiObjectDeleteException, AmazonClientException, IOException {
+      auditSpan.activate();
       return S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir,
           undeletedObjectsOnFailure, operationState, quiet);
     }
@@ -1710,6 +1980,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     @Override
     public void finishRename(final Path sourceRenamed, final Path destCreated)
         throws IOException {
+      auditSpan.activate();
       Path destParent = destCreated.getParent();
       if (!sourceRenamed.getParent().equals(destParent)) {
         LOG.debug("source & dest parents are different; fix up dir markers");
@@ -1736,10 +2007,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
               createListObjectsRequest(key, null),
               ACCEPT_ALL,
               Listing.ACCEPT_ALL_BUT_S3N,
-              null));
+              null,
+              auditSpan));
     }
   }
 
+  /**
+   * Callbacks from {@link Listing}.
+   * Auditing: the listing object is long-lived; the audit span
+   * for a single listing is passed in from the listing
+   * method calls and then down to the callbacks.
+   */
   protected class ListingOperationCallbacksImpl implements
           ListingOperationCallbacks {
 
@@ -1747,9 +2025,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     @Retries.RetryRaw
     public CompletableFuture<S3ListResult> listObjectsAsync(
         S3ListRequest request,
-        DurationTrackerFactory trackerFactory)
+        DurationTrackerFactory trackerFactory,
+        AuditSpan span)
             throws IOException {
-      return submit(unboundedThreadPool, () ->
+      return submit(unboundedThreadPool, span, () ->
           listObjects(request,
               pairedTrackerFactory(trackerFactory,
                   getDurationTrackerFactory())));
@@ -1760,9 +2039,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     public CompletableFuture<S3ListResult> continueListObjectsAsync(
         S3ListRequest request,
         S3ListResult prevResult,
-        DurationTrackerFactory trackerFactory)
+        DurationTrackerFactory trackerFactory,
+        AuditSpan span)
             throws IOException {
-      return submit(unboundedThreadPool,
+      return submit(unboundedThreadPool, span,
           () -> continueListObjects(request, prevResult,
               pairedTrackerFactory(trackerFactory,
                   getDurationTrackerFactory())));
@@ -1777,8 +2057,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
     @Override
     public S3ListRequest createListObjectsRequest(
-            String key,
-            String delimiter) {
+        String key,
+        String delimiter,
+        AuditSpan span) {
+      span.activate();
       return S3AFileSystem.this.createListObjectsRequest(key, delimiter);
     }
 
@@ -1805,15 +2087,22 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Low-level call to get at the object metadata.
+   * This method is used in some external applications and so
+   * must be viewed as a public entry point.
+   * Auditing: An audit entry point.
    * @param path path to the object. This will be qualified.
    * @return metadata
    * @throws IOException IO and object access problems.
    */
   @VisibleForTesting
+  @AuditEntryPoint
+  @InterfaceAudience.LimitedPrivate("utilities")
   @Retries.RetryTranslated
+  @InterfaceStability.Evolving
   public ObjectMetadata getObjectMetadata(Path path) throws IOException {
-    return getObjectMetadata(makeQualified(path), null, invoker,
-        "getObjectMetadata");
+    return trackDurationAndSpan(INVOCATION_GET_FILE_STATUS, path, () ->
+        getObjectMetadata(makeQualified(path), null, invoker,
+            "getObjectMetadata"));
   }
 
   /**
@@ -1829,7 +2118,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private ObjectMetadata getObjectMetadata(Path path,
       ChangeTracker changeTracker, Invoker changeInvoker, String operation)
       throws IOException {
-    checkNotClosed();
     String key = pathToKey(path);
     return once(operation, path.toString(),
         () ->
@@ -1877,12 +2165,76 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /**
    * Entry point to an operation.
    * Increments the statistic; verifies the FS is active.
-   * @param operation The operation to increment
-   * @throws IOException if the
-   */
-  protected void entryPoint(Statistic operation) throws IOException {
+   * @param operation The operation being invoked
+   * @param path first path of operation
+   * @return a span for the audit
+   * @throws IOException failure of audit service
+   */
+  protected AuditSpan entryPoint(Statistic operation,
+      Path path) throws IOException {
+    return entryPoint(operation,
+        (path != null ? pathToKey(path): null),
+        null);
+  }
+
+  /**
+   * Entry point to an operation.
+   * Increments the statistic; verifies the FS is active.
+   * @param operation The operation being invoked
+   * @param path1 first path of operation
+   * @param path2 second path of operation
+   * @return a span for the audit
+   * @throws IOException failure of audit service
+   */
+  protected AuditSpan entryPoint(Statistic operation,
+      @Nullable String path1,
+      @Nullable String path2) throws IOException {
     checkNotClosed();
     incrementStatistic(operation);
+    return createSpan(operation.getSymbol(),
+        path1, path2);
+  }
+
+  /**
+   * Given an IOException raising callable/lambda expression,
+   * execute it and update the relevant statistic within a span
+   * of the same statistic.
+   * @param statistic statistic key
+   * @param path first path for span (nullable)
+   * @param path2 second path for span
+   * @param input input callable.
+   * @param <B> return type.
+   * @return the result of the operation.
+   * @throws IOException if raised in the callable
+   */
+  private <B> B trackDurationAndSpan(
+      Statistic statistic, String path, String path2,
+      CallableRaisingIOE<B> input) throws IOException {
+    checkNotClosed();
+    try (AuditSpan span = createSpan(statistic.getSymbol(),
+        path, path2)) {
+      return trackDuration(getDurationTrackerFactory(),
+          statistic.getSymbol(), input);
+    }
+  }
+
+  /**
+   * Overloaded version of {@code trackDurationAndSpan()}.
+   * Takes a single nullable path as the path param,
+   * @param statistic statistic key
+   * @param path path for span (nullable)
+   * @param input input callable.
+   * @param <B> return type.
+   * @return the result of the operation.
+   * @throws IOException if raised in the callable
+   */
+  private <B> B trackDurationAndSpan(
+      Statistic statistic,
+      @Nullable Path path,
+      CallableRaisingIOE<B> input) throws IOException {
+    return trackDurationAndSpan(statistic,
+        path != null ? pathToKey(path): null,
+        null, input);
   }
 
   /**
@@ -2025,12 +2377,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /**
    * Request object metadata; increments counters in the process.
    * Retry policy: retry untranslated.
+   * This method is used in some external applications and so
+   * must be viewed as a public entry point.
+   * Auditing: this call does NOT initiate a new AuditSpan; the expectation
+   * is that there is already an active span.
    * @param key key
    * @return the metadata
    * @throws IOException if the retry invocation raises one (it shouldn't).
    */
   @Retries.RetryRaw
   @VisibleForTesting
+  @InterfaceAudience.LimitedPrivate("external utilities")
   ObjectMetadata getObjectMetadata(String key) throws IOException {
     return getObjectMetadata(key, null, invoker, "getObjectMetadata");
   }
@@ -2052,12 +2409,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       ChangeTracker changeTracker,
       Invoker changeInvoker,
       String operation) throws IOException {
-    GetObjectMetadataRequest request =
-        new GetObjectMetadataRequest(bucket, key);
-    //SSE-C requires to be filled in if enabled for object metadata
-    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
     ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true,
         () -> {
+          GetObjectMetadataRequest request
+              = getRequestFactory().newGetObjectMetadataRequest(key);
           incrementStatistic(OBJECT_METADATA_REQUESTS);
           DurationTracker duration = getDurationTrackerFactory()
               .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol());
@@ -2160,7 +2515,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
               () -> {
                 if (useListV1) {
                   return S3ListResult.v1(
-                      s3.listNextBatchOfObjects(prevResult.getV1()));
+                      s3.listNextBatchOfObjects(
+                          getRequestFactory()
+                              .newListNextBatchOfObjectsRequest(
+                                  prevResult.getV1())));
                 } else {
                   request.getV2().setContinuationToken(prevResult.getV2()
                       .getNextContinuationToken());
@@ -2214,7 +2572,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             incrementStatistic(OBJECT_DELETE_OBJECTS);
             trackDurationOfInvocation(getDurationTrackerFactory(),
                 OBJECT_DELETE_REQUEST.getSymbol(),
-                () -> s3.deleteObject(bucket, key));
+                () -> s3.deleteObject(getRequestFactory()
+                    .newDeleteObjectRequest(key)));
             return null;
           });
     }
@@ -2327,46 +2686,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   public PutObjectRequest newPutObjectRequest(String key,
       ObjectMetadata metadata, File srcfile) {
-    Preconditions.checkNotNull(srcfile);
-    PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
-        srcfile);
-    setOptionalPutRequestParameters(putObjectRequest);
-    putObjectRequest.setCannedAcl(cannedACL);
-    putObjectRequest.setMetadata(metadata);
-    return putObjectRequest;
-  }
-
-  /**
-   * Create a {@link PutObjectRequest} request.
-   * The metadata is assumed to have been configured with the size of the
-   * operation.
-   * @param key key of object
-   * @param metadata metadata header
-   * @param inputStream source data.
-   * @return the request
-   */
-  PutObjectRequest newPutObjectRequest(String key,
-      ObjectMetadata metadata,
-      InputStream inputStream) {
-    Preconditions.checkNotNull(inputStream);
-    Preconditions.checkArgument(isNotEmpty(key), "Null/empty key");
-    PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key,
-        inputStream, metadata);
-    setOptionalPutRequestParameters(putObjectRequest);
-    putObjectRequest.setCannedAcl(cannedACL);
-    return putObjectRequest;
-  }
-
-  /**
-   * Create a new object metadata instance.
-   * Any standard metadata headers are added here, for example:
-   * encryption.
-   * @return a new metadata instance
-   */
-  public ObjectMetadata newObjectMetadata() {
-    final ObjectMetadata om = new ObjectMetadata();
-    setOptionalObjectMetadata(om);
-    return om;
+    return requestFactory.newPutObjectRequest(key, metadata, srcfile);
   }
 
   /**
@@ -2378,11 +2698,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @return a new metadata instance
    */
   public ObjectMetadata newObjectMetadata(long length) {
-    final ObjectMetadata om = newObjectMetadata();
-    if (length >= 0) {
-      om.setContentLength(length);
-    }
-    return om;
+    return requestFactory.newObjectMetadata(length);
   }
 
   /**
@@ -2397,6 +2713,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * must reference data (files, buffers) which stay valid until the upload
    * completes.
    * Retry policy: N/A: the transfer manager is performing the upload.
+   * Auditing: must be inside an audit span.
    * @param putObjectRequest the request
    * @return the upload initiated
    */
@@ -2416,6 +2733,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * file, from the content length of the header.
    *
    * Retry Policy: none.
+   * Auditing: must be inside an audit span.
    * <i>Important: this call will close any input stream in the request.</i>
    * @param putObjectRequest the request
    * @return the upload initiated
@@ -2432,13 +2750,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey());
     incrementPutStartStatistics(len);
     try {
-      PutObjectResult result = s3.putObject(putObjectRequest);
+      PutObjectResult result = trackDurationOfSupplier(
+          getDurationTrackerFactory(),
+          OBJECT_PUT_REQUESTS.getSymbol(), () ->
+              s3.putObject(putObjectRequest));
       incrementPutCompletedStatistics(true, len);
       // update metadata
       finishedWrite(putObjectRequest.getKey(), len,
           result.getETag(), result.getVersionId(), null);
       return result;
-    } catch (AmazonClientException e) {
+    } catch (SdkBaseException e) {
       incrementPutCompletedStatistics(false, len);
       throw e;
     }
@@ -2477,7 +2798,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     long len = request.getPartSize();
     incrementPutStartStatistics(len);
     try {
-      setOptionalUploadPartRequestParameters(request);
       UploadPartResult uploadPartResult = s3.uploadPart(request);
       incrementPutCompletedStatistics(true, len);
       return uploadPartResult;
@@ -2496,7 +2816,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   public void incrementPutStartStatistics(long bytes) {
     LOG.debug("PUT start {} bytes", bytes);
     incrementWriteOperations();
-    incrementStatistic(OBJECT_PUT_REQUESTS);
     incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
     if (bytes > 0) {
       incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
@@ -2512,7 +2831,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   public void incrementPutCompletedStatistics(boolean success, long bytes) {
     LOG.debug("PUT completed success={}; {} bytes", success, bytes);
-    incrementWriteOperations();
     if (bytes > 0) {
       incrementStatistic(OBJECT_PUT_BYTES, bytes);
       decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
@@ -2581,9 +2899,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     try {
       if (enableMultiObjectsDelete) {
         result = deleteObjects(
-            new DeleteObjectsRequest(bucket)
-                .withKeys(keysToDelete)
-                .withQuiet(quiet));
+            getRequestFactory().newBulkDeleteRequest(keysToDelete, quiet));
       } else {
         for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
           deleteObject(keyVersion.getKey());
@@ -2671,7 +2987,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws IOException other IO Exception.
    */
   @Retries.RetryMixed
-  DeleteObjectsResult removeKeys(
+  private DeleteObjectsResult removeKeys(
       final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
       final boolean deleteFakeDir,
       final List<Path> undeletedObjectsOnFailure,
@@ -2719,33 +3035,39 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * have surfaced.
    * @throws IOException due to inability to delete a directory or file.
    */
+  @Override
   @Retries.RetryTranslated
+  @AuditEntryPoint
   public boolean delete(Path f, boolean recursive) throws IOException {
-    try {
-      entryPoint(INVOCATION_DELETE);
-      DeleteOperation deleteOperation = new DeleteOperation(
-          createStoreContext(),
-          innerGetFileStatus(f, true, StatusProbeEnum.ALL),
-          recursive,
-          operationCallbacks,
-          pageSize);
-      boolean outcome = deleteOperation.execute();
+    checkNotClosed();
+    final Path path = qualify(f);
+    // span covers delete, getFileStatus, fake directory operations.
+    try (AuditSpan span = createSpan(INVOCATION_DELETE.getSymbol(),
+        path.toString(), null)) {
+      boolean outcome = trackDuration(getDurationTrackerFactory(),
+          INVOCATION_DELETE.getSymbol(),
+          new DeleteOperation(
+              createStoreContext(),
+              innerGetFileStatus(path, true, StatusProbeEnum.ALL),
+              recursive,
+              new OperationCallbacksImpl(),
+              pageSize));
       if (outcome) {
         try {
-          maybeCreateFakeParentDirectory(f);
+          maybeCreateFakeParentDirectory(path);
         } catch (AccessDeniedException e) {
           LOG.warn("Cannot create directory marker at {}: {}",
               f.getParent(), e.toString());
-          LOG.debug("Failed to create fake dir above {}", f, e);
+          LOG.debug("Failed to create fake dir above {}", path, e);
         }
       }
       return outcome;
     } catch (FileNotFoundException e) {
-      LOG.debug("Couldn't delete {} - does not exist: {}", f, e.toString());
+      LOG.debug("Couldn't delete {} - does not exist: {}", path, e.toString());
       instrumentation.errorIgnored();
       return false;
     } catch (AmazonClientException e) {
-      throw translateException("delete", f, e);
+      throw translateException("delete", path, e);
     }
   }
 
@@ -2792,11 +3114,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    *
    */
   @Override
+  @AuditEntryPoint
   public RemoteIterator<FileStatus> listStatusIterator(Path p)
           throws FileNotFoundException, IOException {
-    RemoteIterator<S3AFileStatus> listStatusItr = once("listStatus",
-            p.toString(), () -> innerListStatus(p));
-    return typeCastingRemoteIterator(listStatusItr);
+    Path path = qualify(p);
+    return typeCastingRemoteIterator(trackDurationAndSpan(
+        INVOCATION_LIST_STATUS, path, () ->
+            once("listStatus", path.toString(), () ->
+                innerListStatus(p))));
   }
 
   /**
@@ -2808,18 +3133,25 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws FileNotFoundException when the path does not exist;
    *         IOException see specific implementation
    */
+  @Override
+  @AuditEntryPoint
   public FileStatus[] listStatus(Path f) throws FileNotFoundException,
       IOException {
-    return once("listStatus",
-        f.toString(),
-        () -> iteratorToStatuses(innerListStatus(f), new HashSet<>()));
+    Path path = qualify(f);
+    return trackDurationAndSpan(INVOCATION_LIST_STATUS, path, () ->
+        once("listStatus", path.toString(),
+            () -> iteratorToStatuses(innerListStatus(path),
+                new HashSet<>())));
   }
 
   /**
    * List the statuses of the files/directories in the given path if the path is
-   * a directory.
+   * a directory. The returned iterator is within the current active span.
    *
-   * @param f given path
+   * Auditing: This method MUST be called within a span.
+   * The span is attached to the iterator. All further S3 calls
+   * made by the iterator will be within the span.
+   * @param f qualified path
    * @return the statuses of the files/directories in the given patch
    * @throws FileNotFoundException when the path does not exist;
    * @throws IOException due to an IO problem.
@@ -2830,11 +3162,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           IOException, AmazonClientException {
     Path path = qualify(f);
     LOG.debug("List status for path: {}", path);
-    entryPoint(INVOCATION_LIST_STATUS);
 
     Triple<RemoteIterator<S3AFileStatus>, DirListingMetadata, Boolean>
             statusesAssumingNonEmptyDir = listing
-            .getFileStatusesAssumingNonEmptyDir(path);
+            .getFileStatusesAssumingNonEmptyDir(path, getActiveAuditSpan());
 
     if (!statusesAssumingNonEmptyDir.getLeft().hasNext() &&
             statusesAssumingNonEmptyDir.getRight()) {
@@ -2896,34 +3227,28 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @VisibleForTesting
   public S3ListRequest createListObjectsRequest(String key,
       String delimiter) {
-    return createListObjectsRequest(key, delimiter, null);
+    return createListObjectsRequest(key, delimiter, maxKeys);
   }
 
+  /**
+   * Create the List objects request appropriate for the
+   * active list request option.
+   * @param key key for request
+   * @param delimiter any delimiter
+   * @param limit limit of keys
+   * @return the request
+   */
   private S3ListRequest createListObjectsRequest(String key,
-      String delimiter, Integer overrideMaxKeys) {
+      String delimiter, int limit) {
     if (!useListV1) {
       ListObjectsV2Request request =
-          new ListObjectsV2Request().withBucketName(bucket)
-              .withMaxKeys(maxKeys)
-              .withPrefix(key);
-      if (delimiter != null) {
-        request.setDelimiter(delimiter);
-      }
-      if (overrideMaxKeys != null) {
-        request.setMaxKeys(overrideMaxKeys);
-      }
+          getRequestFactory().newListObjectsV2Request(
+              key, delimiter, limit);
       return S3ListRequest.v2(request);
     } else {
-      ListObjectsRequest request = new ListObjectsRequest();
-      request.setBucketName(bucket);
-      request.setMaxKeys(maxKeys);
-      request.setPrefix(key);
-      if (delimiter != null) {
-        request.setDelimiter(delimiter);
-      }
-      if (overrideMaxKeys != null) {
-        request.setMaxKeys(overrideMaxKeys);
-      }
+      ListObjectsRequest request =
+          getRequestFactory().newListObjectsV1Request(
+              key, delimiter, limit);
       return S3ListRequest.v1(request);
     }
   }
@@ -2967,80 +3292,130 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * Make the given path and all non-existent parents into
    * directories. Has the semantics of Unix {@code 'mkdir -p'}.
    * Existence of the directory hierarchy is not an error.
-   * @param path path to create
-   * @param permission to apply to f
+   * @param p path to create
+   * @param permission to apply to path
    * @return true if a directory was created or already existed
    * @throws FileAlreadyExistsException there is a file at the path specified
+   * or is discovered on one of its ancestors.
    * @throws IOException other IO problems
    */
-  // TODO: If we have created an empty file at /foo/bar and we then call
-  // mkdirs for /foo/bar/baz/roo what happens to the empty file /foo/bar/?
-  public boolean mkdirs(Path path, FsPermission permission) throws IOException,
+  @Override
+  @AuditEntryPoint
+  public boolean mkdirs(Path p, FsPermission permission) throws IOException,
       FileAlreadyExistsException {
-    try {
-      entryPoint(INVOCATION_MKDIRS);
-      return innerMkdirs(path, permission);
-    } catch (AmazonClientException e) {
-      throw translateException("mkdirs", path, e);
+    Path path = qualify(p);
+    return trackDurationAndSpan(
+        INVOCATION_MKDIRS, path,
+        new MkdirOperation(
+            createStoreContext(),
+            path,
+            createMkdirOperationCallbacks()));
+  }
+
+  /**
+   * Override point: create the callbacks for Mkdir.
+   * This does not create a new span; caller must be in one.
+   * @return an implementation of the MkdirCallbacks,
+   */
+  @VisibleForTesting
+  public MkdirOperation.MkdirCallbacks createMkdirOperationCallbacks() {
+    return new MkdirOperationCallbacksImpl();
+  }
+
+  /**
+   * Callbacks from the {@link MkdirOperation}.
+   */
+  protected class MkdirOperationCallbacksImpl implements
+      MkdirOperation.MkdirCallbacks {
+
+    @Override
+    public S3AFileStatus probePathStatus(final Path path,
+        final Set<StatusProbeEnum> probes) throws IOException {
+      return S3AFileSystem.this.innerGetFileStatus(path, false, probes);
+    }
+
+    @Override
+    public void createFakeDirectory(final String key)
+        throws IOException {
+      S3AFileSystem.this.createEmptyObject(key);
     }
   }
 
   /**
-   *
-   * Make the given path and all non-existent parents into
-   * directories.
-   * See {@link #mkdirs(Path, FsPermission)}
-   * @param p path to create
-   * @param permission to apply to f
-   * @return true if a directory was created or already existed
-   * @throws FileAlreadyExistsException there is a file at the path specified
-   * @throws IOException other IO problems
-   * @throws AmazonClientException on failures inside the AWS SDK
+   * This is a very slow operation against object storage.
+   * Execute it as a single span with whatever optimizations
+   * have been implemented.
+   * {@inheritDoc}
    */
-  private boolean innerMkdirs(Path p, FsPermission permission)
-      throws IOException, FileAlreadyExistsException, AmazonClientException {
-    Path f = qualify(p);
-    LOG.debug("Making directory: {}", f);
-    if (p.isRoot()) {
-      // fast exit for root.
-      return true;
+  @Override
+  @Retries.RetryTranslated
+  @AuditEntryPoint
+  public ContentSummary getContentSummary(final Path f) throws IOException {
+    final Path path = qualify(f);
+    return trackDurationAndSpan(
+        INVOCATION_GET_CONTENT_SUMMARY, path,
+        new GetContentSummaryOperation(
+            createStoreContext(),
+            path,
+            createGetContentSummaryCallbacks()));
+  }
+
+  /**
+   * Override point: create the callbacks for getContentSummary.
+   * This does not create a new span; caller must be in one.
+   * @return an implementation of the GetContentSummaryCallbacksImpl
+   */
+  protected GetContentSummaryOperation.GetContentSummaryCallbacks
+      createGetContentSummaryCallbacks() {
+    return new GetContentSummaryCallbacksImpl();
+  }
+
+  /**
+   * Callbacks from the {@link GetContentSummaryOperation}.
+   */
+  protected class GetContentSummaryCallbacksImpl implements
+      GetContentSummaryOperation.GetContentSummaryCallbacks {
+
+    @Override
+    public S3AFileStatus probePathStatus(final Path path,
+        final Set<StatusProbeEnum> probes) throws IOException {
+      return S3AFileSystem.this.innerGetFileStatus(path, false, probes);
     }
-    FileStatus fileStatus;
 
-    try {
-      fileStatus = innerGetFileStatus(f, false,
-          StatusProbeEnum.ALL);
+    @Override
+    public RemoteIterator<S3AFileStatus> listStatusIterator(final Path path)
+        throws IOException {
+      return S3AFileSystem.this.innerListStatus(path);
+    }
+  }
 
-      if (fileStatus.isDirectory()) {
-        return true;
-      } else {
-        throw new FileAlreadyExistsException("Path is a file: " + f);
-      }
-    } catch (FileNotFoundException e) {
-      // Walk path to root, ensuring closest ancestor is a directory, not file
-      Path fPart = f.getParent();
-      while (fPart != null && !fPart.isRoot()) {
-        try {
-          fileStatus = getFileStatus(fPart);
-          if (fileStatus.isDirectory()) {
-            break;
-          }
-          if (fileStatus.isFile()) {
-            throw new FileAlreadyExistsException(String.format(
-                "Can't make directory for path '%s' since it is a file.",
-                fPart));
+  /**
+   * Soft check of access by forwarding to the audit manager
+   * and so on to the auditor.
+   * {@inheritDoc}
+   */
+  @Override
+  @AuditEntryPoint
+  public void access(final Path f, final FsAction mode)
+      throws AccessControlException, FileNotFoundException, IOException {
+    Path path = qualify(f);
+    LOG.debug("check access mode {} for {}", path, mode);
+    trackDurationAndSpan(
+        INVOCATION_ACCESS, path, () -> {
+          final S3AFileStatus stat = innerGetFileStatus(path, false,
+              StatusProbeEnum.ALL);
+          if (!getAuditManager().checkAccess(path, stat, mode)) {
+            incrementStatistic(AUDIT_ACCESS_CHECK_FAILURE);
+            throw new AccessControlException(String.format(
+                "Permission denied: user=%s, path=\"%s\":%s:%s:%s%s",
+                getOwner().getUserName(),
+                stat.getPath(),
+                stat.getOwner(), stat.getGroup(),
+                stat.isDirectory() ? "d" : "-", mode));
           }
-        } catch (FileNotFoundException fnfe) {
-          instrumentation.errorIgnored();
-        }
-        fPart = fPart.getParent();
-      }
-      String key = pathToKey(f);
-      // this will create the marker file, delete the parent entries
-      // and update S3Guard
-      createFakeDirectory(key);
-      return true;
-    }
+          // simply for the API binding.
+          return true;
+        });
   }
 
   /**
@@ -3050,10 +3425,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws FileNotFoundException when the path does not exist
    * @throws IOException on other problems.
    */
+  @Override
+  @AuditEntryPoint
   @Retries.RetryTranslated
   public FileStatus getFileStatus(final Path f) throws IOException {
-    entryPoint(INVOCATION_GET_FILE_STATUS);
-    return innerGetFileStatus(f, false, StatusProbeEnum.ALL);
+    Path path = qualify(f);
+    return trackDurationAndSpan(
+        INVOCATION_GET_FILE_STATUS, path, () ->
+            innerGetFileStatus(path, false, StatusProbeEnum.ALL));
   }
 
   /**
@@ -3398,12 +3777,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws AmazonClientException failure in the AWS SDK
    */
   @Override
+  @AuditEntryPoint
   public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src,
       Path dst) throws IOException {
-    entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE);
+    checkNotClosed();
     LOG.debug("Copying local file from {} to {}", src, dst);
-//    innerCopyFromLocalFile(delSrc, overwrite, src, dst);
-    super.copyFromLocalFile(delSrc, overwrite, src, dst);
+    trackDurationAndSpan(INVOCATION_COPY_FROM_LOCAL_FILE, dst, () -> {
+      //  innerCopyFromLocalFile(delSrc, overwrite, src, dst);
+      super.copyFromLocalFile(delSrc, overwrite, src, dst);
+      return null;
+    });
   }
 
   /**
@@ -3432,7 +3815,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite,
       Path src, Path dst)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
-    entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE);
     LOG.debug("Copying local file from {} to {}", src, dst);
 
     // Since we have a local file, we don't need to stream into a temporary file
@@ -3446,7 +3828,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     }
 
     try {
-      FileStatus status = getFileStatus(dst);
+      FileStatus status = innerGetFileStatus(dst, false, StatusProbeEnum.ALL);
       if (!status.isFile()) {
         throw new FileAlreadyExistsException(dst + " exists and is not a file");
       }
@@ -3551,6 +3933,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     } finally {
       stopAllServices();
     }
+    // Log IOStatistics at debug.
+    if (LOG.isDebugEnabled()) {
+      // robust extract and convert to string
+      LOG.debug("Statistics for {}: {}", uri,
+          IOStatisticsLogging.ioStatisticsToPrettyString(getIOStatistics()));
+    }
   }
 
   /**
@@ -3560,6 +3948,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * both the expected state of this FS and of failures while being stopped.
    */
   protected synchronized void stopAllServices() {
+    // shutting down the transfer manager also shuts
+    // down the S3 client it is bonded to.
     if (transfers != null) {
       try {
         transfers.shutdownNow(true);
@@ -3569,17 +3959,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       }
       transfers = null;
     }
+    // At this point the S3A client is shut down,
+    // now the executor pools are closed
     HadoopExecutors.shutdown(boundedThreadPool, LOG,
         THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
     boundedThreadPool = null;
     HadoopExecutors.shutdown(unboundedThreadPool, LOG,
         THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS);
     unboundedThreadPool = null;
+    // other services are shutdown.
     cleanupWithLogger(LOG,
         metadataStore,
         instrumentation,
         delegationTokens.orElse(null),
-        signerManager);
+        signerManager,
+        auditManager);
     closeAutocloseables(LOG, credentials);
     delegationTokens = Optional.empty();
     signerManager = null;
@@ -3637,13 +4031,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws IOException IO failure
    */
   @Override
+  @AuditEntryPoint
   public Token<AbstractS3ATokenIdentifier> getDelegationToken(String renewer)
       throws IOException {
-    entryPoint(Statistic.INVOCATION_GET_DELEGATION_TOKEN);
+    checkNotClosed();
     LOG.debug("Delegation token requested");
     if (delegationTokens.isPresent()) {
-      return delegationTokens.get().getBoundOrNewDT(encryptionSecrets,
-          (renewer != null ? new Text(renewer) : new Text()));
+      return trackDurationAndSpan(
+          INVOCATION_GET_DELEGATION_TOKEN, null, () ->
+              delegationTokens.get().getBoundOrNewDT(
+                  encryptionSecrets,
+                  (renewer != null ? new Text(renewer) : new Text())));
     } else {
       // Delegation token support is not set up
       LOG.debug("Token support is not enabled");
@@ -3661,6 +4059,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Override
   public DelegationTokenIssuer[] getAdditionalTokenIssuers()
       throws IOException {
+    checkNotClosed();
     if (delegationTokens.isPresent()) {
       return delegationTokens.get().getAdditionalTokenIssuers();
     } else {
@@ -3679,6 +4078,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @return a policy for use in roles
    */
   @Override
+  @InterfaceAudience.Private
   public List<RoleModel.Statement> listAWSPolicyRules(
       final Set<AccessLevel> access) {
     if (access.isEmpty()) {
@@ -3767,24 +4167,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           message,
           e);
     }
-    ObjectMetadata dstom = cloneObjectMetadata(srcom);
-    setOptionalObjectMetadata(dstom);
 
     return readInvoker.retry(
         action, srcKey,
         true,
         () -> {
           CopyObjectRequest copyObjectRequest =
-              new CopyObjectRequest(bucket, srcKey, bucket, dstKey);
+              getRequestFactory().newCopyObjectRequest(srcKey, dstKey, srcom);
           changeTracker.maybeApplyConstraint(copyObjectRequest);
-
-          setOptionalCopyObjectRequestParameters(srcom, copyObjectRequest);
-          copyObjectRequest.setCannedAccessControlList(cannedACL);
-          copyObjectRequest.setNewObjectMetadata(dstom);
-          Optional.ofNullable(srcom.getStorageClass())
-              .ifPresent(copyObjectRequest::setStorageClass);
           incrementStatistic(OBJECT_COPY_REQUESTS);
-          Copy copy = transfers.copy(copyObjectRequest);
+          Copy copy = transfers.copy(copyObjectRequest,
+              getAuditManager().createStateChangeListener());
           copy.addProgressListener(progressListener);
           CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy);
           InterruptedException interruptedException =
@@ -3810,62 +4203,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Propagate encryption parameters from source file if set else use the
-   * current filesystem encryption settings.
-   * @param srcom source object meta.
-   * @param copyObjectRequest copy object request body.
-   */
-  private void setOptionalCopyObjectRequestParameters(
-          ObjectMetadata srcom,
-          CopyObjectRequest copyObjectRequest) {
-    String sourceKMSId = srcom.getSSEAwsKmsKeyId();
-    if (isNotEmpty(sourceKMSId)) {
-      // source KMS ID is propagated
-      LOG.debug("Propagating SSE-KMS settings from source {}",
-          sourceKMSId);
-      copyObjectRequest.setSSEAwsKeyManagementParams(
-              new SSEAwsKeyManagementParams(sourceKMSId));
-    }
-    switch(getServerSideEncryptionAlgorithm()) {
-    /**
-     * Overriding with client encryption settings.
-     */
-    case SSE_C:
-      generateSSECustomerKey().ifPresent(customerKey -> {
-        copyObjectRequest.setSourceSSECustomerKey(customerKey);
-        copyObjectRequest.setDestinationSSECustomerKey(customerKey);
-      });
-      break;
-    case SSE_KMS:
-      generateSSEAwsKeyParams().ifPresent(
-              copyObjectRequest::setSSEAwsKeyManagementParams);
-      break;
-    default:
-    }
-  }
-
-  /**
-   * Set the optional parameters when initiating the request (encryption,
-   * headers, storage, etc).
-   * @param request request to patch.
-   */
-  protected void setOptionalMultipartUploadRequestParameters(
-      InitiateMultipartUploadRequest request) {
-    generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
-    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
-  }
-
-  /**
-   * Sets server side encryption parameters to the part upload
-   * request when encryption is enabled.
-   * @param request upload part request
-   */
-  protected void setOptionalUploadPartRequestParameters(
-      UploadPartRequest request) {
-    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
-  }
-
-  /**
    * Initiate a multipart upload from the preconfigured request.
    * Retry policy: none + untranslated.
    * @param request request to initiate
@@ -3877,42 +4214,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   InitiateMultipartUploadResult initiateMultipartUpload(
       InitiateMultipartUploadRequest request) throws IOException {
     LOG.debug("Initiate multipart upload to {}", request.getKey());
-    incrementStatistic(OBJECT_MULTIPART_UPLOAD_INITIATED);
-    return getAmazonS3Client().initiateMultipartUpload(request);
-  }
-
-  private void setOptionalPutRequestParameters(PutObjectRequest request) {
-    generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
-    generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
-  }
-
-  private void setOptionalObjectMetadata(ObjectMetadata metadata) {
-    final S3AEncryptionMethods algorithm
-        = getServerSideEncryptionAlgorithm();
-    if (S3AEncryptionMethods.SSE_S3.equals(algorithm)) {
-      metadata.setSSEAlgorithm(algorithm.getMethod());
-    }
-  }
-
-  /**
-   * Create the AWS SDK structure used to configure SSE,
-   * if the encryption secrets contain the information/settings for this.
-   * @return an optional set of KMS Key settings
-   */
-  private Optional<SSEAwsKeyManagementParams> generateSSEAwsKeyParams() {
-    return EncryptionSecretOperations.createSSEAwsKeyManagementParams(
-        encryptionSecrets);
-  }
-
-  /**
-   * Create the SSE-C structure for the AWS SDK, if the encryption secrets
-   * contain the information/settings for this.
-   * This will contain a secret extracted from the bucket/configuration.
-   * @return an optional customer key.
-   */
-  private Optional<SSECustomerKey> generateSSECustomerKey() {
-    return EncryptionSecretOperations.createSSECustomerKey(
-        encryptionSecrets);
+    return trackDurationOfSupplier(getDurationTrackerFactory(),
+        OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(),
+        () -> getAmazonS3Client().initiateMultipartUpload(request));
   }
 
   /**
@@ -3957,7 +4261,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     CompletableFuture<?> deletion;
     if (!keepDirectoryMarkers(p)) {
       deletion = submit(
-          unboundedThreadPool,
+          unboundedThreadPool, getActiveAuditSpan(),
           () -> {
             deleteUnnecessaryFakeDirectories(
                 p.getParent(),
@@ -4076,11 +4380,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.RetryTranslated
   private void createFakeDirectory(final String objectName)
       throws IOException {
-    if (!objectName.endsWith("/")) {
-      createEmptyObject(objectName + "/");
-    } else {
-      createEmptyObject(objectName);
-    }
+    createEmptyObject(objectName);
   }
 
   /**
@@ -4092,37 +4392,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.RetryTranslated
   private void createEmptyObject(final String objectName)
       throws IOException {
-    final InputStream im = new InputStream() {
-      @Override
-      public int read() throws IOException {
-        return -1;
-      }
-    };
-
-    PutObjectRequest putObjectRequest = newPutObjectRequest(objectName,
-        newObjectMetadata(0L),
-        im);
     invoker.retry("PUT 0-byte object ", objectName,
-         true,
-        () -> putObjectDirect(putObjectRequest));
+         true, () ->
+            putObjectDirect(getRequestFactory()
+                .newDirectoryMarkerRequest(objectName)));
     incrementPutProgressStatistics(objectName, 0);
     instrumentation.directoryCreated();
   }
 
   /**
-   * Creates a copy of the passed {@link ObjectMetadata}.
-   * Does so without using the {@link ObjectMetadata#clone()} method,
-   * to avoid copying unnecessary headers.
-   * @param source the {@link ObjectMetadata} to copy
-   * @return a copy of {@link ObjectMetadata} with only relevant attributes
-   */
-  private ObjectMetadata cloneObjectMetadata(ObjectMetadata source) {
-    ObjectMetadata ret = newObjectMetadata(source.getContentLength());
-    getHeaderProcessing().cloneObjectMetadata(source, ret);
-    return ret;
-  }
-
-  /**
    * Return the number of bytes that large input files should be optimally
    * be split into to minimize I/O time.
    * @deprecated use {@link #getDefaultBlockSize(Path)} instead
@@ -4166,6 +4444,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     if (blockFactory != null) {
       sb.append(", blockFactory=").append(blockFactory);
     }
+    sb.append(", auditManager=").append(auditManager);
     sb.append(", metastore=").append(metadataStore);
     sb.append(", authoritativeStore=").append(allowAuthoritativeMetadataStore);
     sb.append(", authoritativePath=").append(allowAuthoritativePaths);
@@ -4246,20 +4525,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * Increments the statistic {@link Statistic#INVOCATION_GLOB_STATUS}.
    * Override superclass so as to disable symlink resolution as symlinks
    * are not supported by S3A.
+   *
+   * Although an AuditEntryPoint, the globber itself will talk do
+   * the filesystem through the filesystem API, so its operations will
+   * all appear part of separate operations.
    * {@inheritDoc}
    */
   @Override
+  @AuditEntryPoint
   public FileStatus[] globStatus(
       final Path pathPattern,
       final PathFilter filter)
       throws IOException {
-    entryPoint(INVOCATION_GLOB_STATUS);
-    return Globber.createGlobber(this)
-        .withPathPattern(pathPattern)
-        .withPathFiltern(filter)
-        .withResolveSymlinks(false)
-        .build()
-        .glob();
+    return trackDurationAndSpan(
+        INVOCATION_GLOB_STATUS, pathPattern, () ->
+            Globber.createGlobber(this)
+                .withPathPattern(pathPattern)
+                .withPathFiltern(filter)
+                .withResolveSymlinks(false)
+                .build()
+                .glob());
   }
 
   /**
@@ -4267,9 +4552,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * {@inheritDoc}
    */
   @Override
+  @AuditEntryPoint
   public boolean exists(Path f) throws IOException {
-    entryPoint(INVOCATION_EXISTS);
-    return super.exists(f);
+    final Path path = qualify(f);
+    try {
+      trackDurationAndSpan(
+          INVOCATION_EXISTS, path, () ->
+              innerGetFileStatus(path, false, StatusProbeEnum.ALL));
+      return true;
+    } catch (FileNotFoundException e) {
+      return false;
+    }
   }
 
   /**
@@ -4279,12 +4572,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * {@inheritDoc}
    */
   @Override
+  @AuditEntryPoint
   @SuppressWarnings("deprecation")
   public boolean isDirectory(Path f) throws IOException {
-    entryPoint(INVOCATION_IS_DIRECTORY);
+    final Path path = qualify(f);
     try {
-      return innerGetFileStatus(f, false, StatusProbeEnum.DIRECTORIES)
-          .isDirectory();
+      return trackDurationAndSpan(
+          INVOCATION_IS_DIRECTORY, path, () ->
+              innerGetFileStatus(path, false, StatusProbeEnum.DIRECTORIES)
+                  .isDirectory());
     } catch (FileNotFoundException e) {
       // not found or it is a file.
       return false;
@@ -4298,12 +4594,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * {@inheritDoc}
    */
   @Override
+  @AuditEntryPoint
   @SuppressWarnings("deprecation")
   public boolean isFile(Path f) throws IOException {
-    entryPoint(INVOCATION_IS_FILE);
+    final Path path = qualify(f);
     try {
-      return innerGetFileStatus(f, false, StatusProbeEnum.HEAD_ONLY)
-          .isFile();
+      return trackDurationAndSpan(INVOCATION_IS_FILE, path, () ->
+          innerGetFileStatus(path, false, StatusProbeEnum.HEAD_ONLY)
+              .isFile());
     } catch (FileNotFoundException e) {
       // not found or it is a dir.
       return false;
@@ -4331,18 +4629,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   @Override
   @Retries.RetryTranslated
+  @AuditEntryPoint
   public EtagChecksum getFileChecksum(Path f, final long length)
       throws IOException {
     Preconditions.checkArgument(length >= 0);
-    entryPoint(INVOCATION_GET_FILE_CHECKSUM);
-
+    final Path path = qualify(f);
     if (getConf().getBoolean(ETAG_CHECKSUM_ENABLED,
         ETAG_CHECKSUM_ENABLED_DEFAULT)) {
-      Path path = qualify(f);
-      LOG.debug("getFileChecksum({})", path);
-      ObjectMetadata headers = getObjectMetadata(path);
-      String eTag = headers.getETag();
-      return eTag != null ? new EtagChecksum(eTag) : null;
+      return trackDurationAndSpan(INVOCATION_GET_FILE_CHECKSUM, path, () -> {
+        LOG.debug("getFileChecksum({})", path);
+        ObjectMetadata headers = getObjectMetadata(path, null,
+            invoker,
+            "getFileChecksum are");
+        String eTag = headers.getETag();
+        return eTag != null ? new EtagChecksum(eTag) : null;
+      });
     } else {
       // disabled
       return null;
@@ -4351,36 +4652,84 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Get header processing support.
-   * @return the header processing of this instance.
+   * @return a new header processing instance.
    */
   private HeaderProcessing getHeaderProcessing() {
-    return headerProcessing;
+    return new HeaderProcessing(createStoreContext(),
+        createHeaderProcessingCallbacks());
   }
 
   @Override
+  @AuditEntryPoint
   public byte[] getXAttr(final Path path, final String name)
       throws IOException {
-    return getHeaderProcessing().getXAttr(path, name);
+    checkNotClosed();
+    try (AuditSpan span = createSpan(
+        INVOCATION_XATTR_GET_NAMED.getSymbol(),
+        path.toString(), null)) {
+      return getHeaderProcessing().getXAttr(path, name);
+    }
   }
 
   @Override
+  @AuditEntryPoint
   public Map<String, byte[]> getXAttrs(final Path path) throws IOException {
-    return getHeaderProcessing().getXAttrs(path);
+    checkNotClosed();
+    try (AuditSpan span = createSpan(
+        INVOCATION_XATTR_GET_MAP.getSymbol(),
+        path.toString(), null)) {
+      return getHeaderProcessing().getXAttrs(path);
+    }
   }
 
   @Override
+  @AuditEntryPoint
   public Map<String, byte[]> getXAttrs(final Path path,
       final List<String> names)
       throws IOException {
-    return getHeaderProcessing().getXAttrs(path, names);
+    checkNotClosed();
+    try (AuditSpan span = createSpan(
+        INVOCATION_XATTR_GET_NAMED_MAP.getSymbol(),
+        path.toString(), null)) {
+      return getHeaderProcessing().getXAttrs(path, names);
+    }
   }
 
   @Override
+  @AuditEntryPoint
   public List<String> listXAttrs(final Path path) throws IOException {
-    return getHeaderProcessing().listXAttrs(path);
+    checkNotClosed();
+    try (AuditSpan span = createSpan(
+        INVOCATION_OP_XATTR_LIST.getSymbol(),
+        path.toString(), null)) {
+      return getHeaderProcessing().listXAttrs(path);
+    }
+  }
+
+  /**
+   * Create the callbacks.
+   * @return An implementation of the header processing
+   * callbacks.
+   */
+  protected HeaderProcessing.HeaderProcessingCallbacks
+      createHeaderProcessingCallbacks() {
+    return new HeaderProcessingCallbacksImpl();
   }
 
   /**
+   * Operations needed for Header Processing.
+   */
+  protected final class HeaderProcessingCallbacksImpl implements
+      HeaderProcessing.HeaderProcessingCallbacks {
+
+    @Override
+    public ObjectMetadata getObjectMetadata(final String key)
+        throws IOException {
+      return once("getObjectMetadata", key, () ->
+          S3AFileSystem.this.getObjectMetadata(key));
+    }
+  }
+  /**
    * {@inheritDoc}.
    *
    * This implementation is optimized for S3, which can do a bulk listing
@@ -4407,38 +4756,53 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   @Override
   @Retries.RetryTranslated
+  @AuditEntryPoint
   public RemoteIterator<LocatedFileStatus> listFiles(Path f,
       boolean recursive) throws FileNotFoundException, IOException {
-    return toLocatedFileStatusIterator(innerListFiles(f, recursive,
-        new Listing.AcceptFilesOnly(qualify(f)), null, true, false));
+    final Path path = qualify(f);
+    return toLocatedFileStatusIterator(
+        trackDurationAndSpan(INVOCATION_LIST_FILES, path, () ->
+            innerListFiles(path, recursive,
+                new Listing.AcceptFilesOnly(path), null, true, false)));
   }
 
   /**
    * Recursive List of files and empty directories.
    * @param f path to list from
+   * @param recursive recursive?
    * @return an iterator.
    * @throws IOException failure
    */
+  @InterfaceAudience.Private
   @Retries.RetryTranslated
+  @AuditEntryPoint
   public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectories(
       Path f, boolean recursive) throws IOException {
-    return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N,
-        null, true, false);
+    final Path path = qualify(f);
+    return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () ->
+        innerListFiles(path, recursive,
+            Listing.ACCEPT_ALL_BUT_S3N,
+            null, true, false));
   }
 
   /**
    * Recursive List of files and empty directories, force metadatastore
    * to act like it is non-authoritative.
    * @param f path to list from
-   * @param recursive
+   * @param recursive recursive listing?
    * @return an iterator.
    * @throws IOException failure
    */
+  @InterfaceAudience.Private
   @Retries.RetryTranslated
+  @AuditEntryPoint
   public RemoteIterator<S3ALocatedFileStatus> listFilesAndEmptyDirectoriesForceNonAuth(
       Path f, boolean recursive) throws IOException {
-    return innerListFiles(f, recursive, Listing.ACCEPT_ALL_BUT_S3N,
-        null, true, true);
+    final Path path = qualify(f);
+    return trackDurationAndSpan(INVOCATION_LIST_FILES, path, () ->
+        innerListFiles(path, recursive,
+            Listing.ACCEPT_ALL_BUT_S3N,
+            null, true, true));
   }
 
   /**
@@ -4487,7 +4851,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       final S3AFileStatus status,
       final boolean collectTombstones,
       final boolean forceNonAuthoritativeMS) throws IOException {
-    entryPoint(INVOCATION_LIST_FILES);
     Path path = qualify(f);
     LOG.debug("listFiles({}, {})", path, recursive);
     try {
@@ -4505,7 +4868,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
                       recursive,
                       acceptor,
                       collectTombstones,
-                      forceNonAuthoritativeMS);
+                      forceNonAuthoritativeMS,
+                      getActiveAuditSpan());
       // If there are no list entries present, we
       // fallback to file existence check as the path
       // can be a file or empty directory.
@@ -4513,7 +4877,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         // If file status was already passed, reuse it.
         final S3AFileStatus fileStatus = status != null
                 ? status
-                : (S3AFileStatus) getFileStatus(path);
+                : innerGetFileStatus(path, false, StatusProbeEnum.ALL);
         if (fileStatus.isFile()) {
           return listing.createSingleStatusIterator(
                   toLocatedFileStatus(fileStatus));
@@ -4552,11 +4916,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   @Override
   @Retries.OnceTranslated("s3guard not retrying")
+  @AuditEntryPoint
   public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f,
       final PathFilter filter)
       throws FileNotFoundException, IOException {
-    entryPoint(INVOCATION_LIST_LOCATED_STATUS);
     Path path = qualify(f);
+    AuditSpan span = entryPoint(INVOCATION_LIST_LOCATED_STATUS, path);
     LOG.debug("listLocatedStatus({}, {}", path, filter);
     RemoteIterator<? extends LocatedFileStatus> iterator =
         once("listLocatedStatus", path.toString(),
@@ -4565,12 +4930,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             // trigger a list call directly.
             final RemoteIterator<S3ALocatedFileStatus>
                     locatedFileStatusIteratorForDir =
-                    listing.getLocatedFileStatusIteratorForDir(path, filter);
+                    listing.getLocatedFileStatusIteratorForDir(path, filter,
+                        span);
 
             // If no listing is present then path might be a file.
             if (!locatedFileStatusIteratorForDir.hasNext()) {
               final S3AFileStatus fileStatus =
-                      (S3AFileStatus) getFileStatus(path);
+                  innerGetFileStatus(path, false, StatusProbeEnum.ALL);
               if (fileStatus.isFile()) {
                 // simple case: File
                 LOG.debug("Path is a file");
@@ -4587,43 +4953,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Generate list located status for a directory.
-   * Also performing tombstone reconciliation for guarded directories.
-   * @param dir directory to check.
-   * @param filter a path filter.
-   * @return an iterator that traverses statuses of the given dir.
-   * @throws IOException in case of failure.
-   */
-  private RemoteIterator<S3ALocatedFileStatus> getLocatedFileStatusIteratorForDir(
-          Path dir, PathFilter filter) throws IOException {
-    final String key = maybeAddTrailingSlash(pathToKey(dir));
-    final Listing.FileStatusAcceptor acceptor =
-        new Listing.AcceptAllButSelfAndS3nDirs(dir);
-    boolean allowAuthoritative = allowAuthoritative(dir);
-    DirListingMetadata meta =
-        S3Guard.listChildrenWithTtl(metadataStore, dir,
-            ttlTimeProvider, allowAuthoritative);
-    Set<Path> tombstones = meta != null
-            ? meta.listTombstones()
-            : null;
-    final RemoteIterator<S3AFileStatus> cachedFileStatusIterator =
-        listing.createProvidedFileStatusIterator(
-            dirMetaToStatuses(meta), filter, acceptor);
-    return (allowAuthoritative && meta != null
-        && meta.isAuthoritative())
-        ? listing.createLocatedFileStatusIterator(
-        cachedFileStatusIterator)
-        : listing.createTombstoneReconcilingIterator(
-            listing.createLocatedFileStatusIterator(
-            listing.createFileStatusListingIterator(dir,
-                createListObjectsRequest(key, "/"),
-                filter,
-                acceptor,
-                cachedFileStatusIterator)),
-            tombstones);
-  }
-
-  /**
    * Build a {@link S3ALocatedFileStatus} from a {@link FileStatus} instance.
    * @param status file status
    * @return a located status with block locations set up from this FS.
@@ -4647,10 +4976,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @return Iterator over multipart uploads.
    * @throws IOException on failure
    */
+  @InterfaceAudience.Private
+  @Retries.RetryTranslated
+  @AuditEntryPoint
   public MultipartUtils.UploadIterator listUploads(@Nullable String prefix)
       throws IOException {
-    return MultipartUtils.listMultipartUploads(s3, invoker, bucket, maxKeys,
-        prefix);
+    // span is picked up retained in the listing.
+    return trackDurationAndSpan(MULTIPART_UPLOAD_LIST, prefix, null, () ->
+        MultipartUtils.listMultipartUploads(
+            createStoreContext(),
+            s3, prefix, maxKeys
+        ));
   }
 
   /**
@@ -4666,17 +5002,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.RetryTranslated
   public List<MultipartUpload> listMultipartUploads(String prefix)
       throws IOException {
-    ListMultipartUploadsRequest request = new ListMultipartUploadsRequest(
-        bucket);
-    if (!prefix.isEmpty()) {
-      if (!prefix.endsWith("/")) {
-        prefix = prefix + "/";
-      }
-      request.setPrefix(prefix);
+    // add a trailing / if needed.
+    if (prefix != null && !prefix.isEmpty() && !prefix.endsWith("/")) {
+      prefix = prefix + "/";
     }
-
-    return invoker.retry("listMultipartUploads", prefix, true,
-        () -> s3.listMultipartUploads(request).getMultipartUploads());
+    String p = prefix;
+    return invoker.retry("listMultipartUploads", p, true, () -> {
+      ListMultipartUploadsRequest request = getRequestFactory()
+          .newListMultipartUploadsRequest(p);
+      return s3.listMultipartUploads(request).getMultipartUploads();
+    });
   }
 
   /**
@@ -4689,7 +5024,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   void abortMultipartUpload(String destKey, String uploadId) {
     LOG.info("Aborting multipart upload {} to {}", uploadId, destKey);
     getAmazonS3Client().abortMultipartUpload(
-        new AbortMultipartUploadRequest(getBucket(),
+        getRequestFactory().newAbortMultipartUploadRequest(
             destKey,
             uploadId));
   }
@@ -4712,7 +5047,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           df.format(upload.getInitiated()));
     }
     getAmazonS3Client().abortMultipartUpload(
-        new AbortMultipartUploadRequest(getBucket(),
+        getRequestFactory().newAbortMultipartUploadRequest(
             destKey,
             uploadId));
   }
@@ -4740,7 +5075,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
     case SelectConstants.S3_SELECT_CAPABILITY:
       // select is only supported if enabled
-      return selectBinding.isEnabled();
+      return SelectBinding.isSelectEnabled(getConf());
 
     case CommonPathCapabilities.FS_CHECKSUMS:
       // capability depends on FS configuration
@@ -4817,8 +5152,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * This is a proof of concept of a select API.
-   * Once a proper factory mechanism for opening files is added to the
-   * FileSystem APIs, this will be deleted <i>without any warning</i>.
    * @param source path to source data
    * @param expression select expression
    * @param options request configuration from the builder.
@@ -4827,12 +5160,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @throws IOException IO failure
    */
   @Retries.RetryTranslated
+  @AuditEntryPoint
   private FSDataInputStream select(final Path source,
       final String expression,
       final Configuration options,
       final Optional<S3AFileStatus> providedStatus)
       throws IOException {
-    entryPoint(OBJECT_SELECT_REQUESTS);
+    final AuditSpan auditSpan = entryPoint(OBJECT_SELECT_REQUESTS, source);
     requireSelectSupport(source);
     final Path path = makeQualified(source);
     final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path,
@@ -4842,7 +5176,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     long ra = options.getLong(READAHEAD_RANGE, readAhead);
     S3ObjectAttributes objectAttributes = createObjectAttributes(fileStatus);
     S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy,
-        changeDetectionPolicy, ra);
+        changeDetectionPolicy, ra, auditSpan);
 
     if (changeDetectionPolicy.getSource() != ChangeDetectionPolicy.Source.None
         && fileStatus.getETag() != null) {
@@ -4866,13 +5200,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       Invoker readInvoker = readContext.getReadInvoker();
       getObjectMetadata(path, changeTracker, readInvoker, "select");
     }
+    // instantiate S3 Select support using the current span
+    // as the active span for operations.
+    SelectBinding selectBinding = new SelectBinding(
+        createWriteOperationHelper(auditSpan));
 
     // build and execute the request
     return selectBinding.select(
         readContext,
         expression,
         options,
-        generateSSECustomerKey(),
         objectAttributes);
   }
 
@@ -4883,7 +5220,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   private void requireSelectSupport(final Path source) throws
       UnsupportedOperationException {
-    if (!selectBinding.isEnabled()) {
+    if (!SelectBinding.isSelectEnabled(getConf())) {
+
       throw new UnsupportedOperationException(
           SelectConstants.SELECT_UNSUPPORTED);
     }
@@ -4922,7 +5260,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Initiate the open or select operation.
-   * This is invoked from both the FileSystem and FileContext APIs
+   * This is invoked from both the FileSystem and FileContext APIs.
+   * It's declared as an audit entry point but the span creation is pushed
+   * down into the open/select methods it ultimately calls.
    * @param rawPath path to the file
    * @param parameters open file parameters from the builder.
    * @return a future which will evaluate to the opened/selected file.
@@ -4933,6 +5273,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   @Override
   @Retries.RetryTranslated
+  @AuditEntryPoint
   public CompletableFuture<FSDataInputStream> openFileWithOptions(
       final Path rawPath,
       final OpenFileParameters parameters) throws IOException {
@@ -4998,15 +5339,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   @Override
+  @AuditEntryPoint
   public S3AMultipartUploaderBuilder createMultipartUploader(
       final Path basePath)
       throws IOException {
-    StoreContext ctx = createStoreContext();
-    return new S3AMultipartUploaderBuilder(this,
-        getWriteOperationHelper(),
-        ctx,
-        basePath,
-        statisticsContext.createMultipartUploaderStatistics());
+    final Path path = makeQualified(basePath);
+    try (AuditSpan span = entryPoint(MULTIPART_UPLOAD_INSTANTIATED, path)) {
+      StoreContext ctx = createStoreContext();
+      return new S3AMultipartUploaderBuilder(this,
+          createWriteOperationHelper(span),
+          ctx,
+          path,
+          statisticsContext.createMultipartUploaderStatistics());
+    }
   }
 
   /**
@@ -5035,16 +5380,24 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         .setUseListV1(useListV1)
         .setContextAccessors(new ContextAccessorsImpl())
         .setTimeProvider(getTtlTimeProvider())
+        .setAuditor(getAuditor())
         .build();
   }
 
   /**
    * Create a marker tools operations binding for this store.
+   * Auditing:
+   * @param target target path
    * @return callbacks for operations.
+   * @throws IOException if raised during span creation
    */
+  @AuditEntryPoint
   @InterfaceAudience.Private
-  public MarkerToolOperations createMarkerToolOperations() {
-    return new MarkerToolOperationsImpl(operationCallbacks);
+  public MarkerToolOperations createMarkerToolOperations(final String target)
+      throws IOException {
+    createSpan("marker-tool-scan", target,
+        null);
+    return new MarkerToolOperationsImpl(new OperationCallbacksImpl());
   }
 
   /**
@@ -5088,10 +5441,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     }
 
     @Override
-    public ObjectMetadata getObjectMetadata(final String key)
-        throws IOException {
-      return once("getObjectMetadata", key, () ->
-          S3AFileSystem.this.getObjectMetadata(key));
+    public AuditSpan getActiveAuditSpan() {
+      return S3AFileSystem.this.getActiveAuditSpan();
+    }
+
+    @Override
+    public RequestFactory getRequestFactory() {
+      return S3AFileSystem.this.getRequestFactory();
     }
   }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index c725fdf..d56d4ac 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -20,14 +20,11 @@ package org.apache.hadoop.fs.s3a;
 
 import javax.annotation.Nullable;
 
-import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.S3Object;
 import com.amazonaws.services.s3.model.S3ObjectInputStream;
-import com.amazonaws.services.s3.model.SSECustomerKey;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CanSetReadahead;
@@ -45,6 +42,7 @@ import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;
 import java.io.EOFException;
 import java.io.IOException;
 import java.net.SocketTimeoutException;
@@ -101,7 +99,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   private S3Object object;
   private S3ObjectInputStream wrappedStream;
   private final S3AReadOpContext context;
-  private final AmazonS3 client;
+  private final InputStreamCallbacks client;
   private final String bucket;
   private final String key;
   private final String pathStr;
@@ -110,8 +108,6 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   private static final Logger LOG =
       LoggerFactory.getLogger(S3AInputStream.class);
   private final S3AInputStreamStatistics streamStatistics;
-  private S3AEncryptionMethods serverSideEncryptionAlgorithm;
-  private String serverSideEncryptionKey;
   private S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
 
@@ -150,7 +146,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    */
   public S3AInputStream(S3AReadOpContext ctx,
       S3ObjectAttributes s3Attributes,
-      AmazonS3 client) {
+      InputStreamCallbacks client) {
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
         "No Bucket");
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
@@ -166,9 +162,6 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     this.streamStatistics = ctx.getS3AStatisticsContext()
         .newInputStreamStatistics();
     this.ioStatistics = streamStatistics.getIOStatistics();
-    this.serverSideEncryptionAlgorithm =
-        s3Attributes.getServerSideEncryptionAlgorithm();
-    this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
     this.changeTracker = new ChangeTracker(uri,
         ctx.getChangeDetectionPolicy(),
         streamStatistics.getChangeTrackerStatistics(),
@@ -211,16 +204,13 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         inputPolicy);
 
     long opencount = streamStatistics.streamOpened();
-    GetObjectRequest request = new GetObjectRequest(bucket, key)
+    GetObjectRequest request = client.newGetRequest(key)
         .withRange(targetPos, contentRangeFinish - 1);
-    if (S3AEncryptionMethods.SSE_C.equals(serverSideEncryptionAlgorithm) &&
-        StringUtils.isNotBlank(serverSideEncryptionKey)){
-      request.setSSECustomerKey(new SSECustomerKey(serverSideEncryptionKey));
-    }
     String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN;
     String text = String.format("%s %s at %d",
         operation, uri, targetPos);
     changeTracker.maybeApplyConstraint(request);
+
     DurationTracker tracker = streamStatistics.initiateGetRequest();
     try {
       object = Invoker.once(text, uri,
@@ -567,6 +557,8 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         // close or abort the stream
         closeStream("close() operation", this.contentRangeFinish, false);
         LOG.debug("Statistics of stream {}\n{}", key, streamStatistics);
+        // end the client+audit span.
+        client.close();
         // this is actually a no-op
         super.close();
       } finally {
@@ -908,4 +900,27 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   public IOStatistics getIOStatistics() {
     return ioStatistics;
   }
+
+  /**
+   * Callbacks for input stream IO.
+   */
+  public interface InputStreamCallbacks extends Closeable {
+
+    /**
+     * Create a GET request.
+     * @param key object key
+     * @return the request
+     */
+    GetObjectRequest newGetRequest(String key);
+
+    /**
+     * Execute the request.
+     * @param request the request
+     * @return the response
+     */
+    @Retries.OnceRaw
+    S3Object getObject(GetObjectRequest request);
+
+  }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 169a74a..a185bac 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -28,15 +28,16 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.s3a.s3guard.MetastoreInstrumentation;
-import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource;
+import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
 import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
 import org.apache.hadoop.fs.s3a.statistics.CountersAndGauges;
-import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
 import org.apache.hadoop.fs.s3a.statistics.DelegationTokenStatistics;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
-import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum;
+import org.apache.hadoop.fs.s3a.statistics.impl.AbstractS3AStatisticsSource;
+import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.impl.ForwardingIOStatisticsStore;
 import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
@@ -469,6 +470,14 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
   }
 
   /**
+   * Create an IOStatistics store which updates FS metrics
+   * as well as IOStatistics.
+   */
+  public IOStatisticsStore createMetricsUpdatingStore() {
+    return new MetricsUpdatingIOStatisticsStore();
+  }
+
+  /**
    * String representation. Includes the IOStatistics
    * when logging is at DEBUG.
    * @return a string form.
@@ -547,10 +556,24 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
    * @param count increment value
    */
   public void incrementCounter(Statistic op, long count) {
-    String name = op.getSymbol();
+    incrementNamedCounter(op.getSymbol(), count);
+  }
+
+  /**
+   * Increments a mutable counter and the matching
+   * instance IOStatistics counter.
+   * No-op if the counter is not defined, or the count == 0.
+   * @param name counter name
+   * @param count increment value
+   * @return the updated value or, if the counter is unknown: 0
+   */
+  private long incrementNamedCounter(final String name,
+      final long count) {
     if (count != 0) {
       incrementMutableCounter(name, count);
-      instanceIOStatistics.incrementCounter(name, count);
+      return instanceIOStatistics.incrementCounter(name, count);
+    } else {
+      return 0;
     }
   }
 
@@ -1868,4 +1891,43 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
       return map;
     }
   }
+
+  /**
+   * An IOStatisticsStore which updates metrics on calls to
+   * {@link #incrementCounter(String, long)}.
+   * This helps keeps FS metrics and IOStats in sync.
+   * Duration tracking methods are forwarded to
+   * the S3A Instrumentation duration tracker, which will
+   * update the instance IOStatistics.
+   */
+  private final class MetricsUpdatingIOStatisticsStore
+      extends ForwardingIOStatisticsStore {
+
+    private MetricsUpdatingIOStatisticsStore() {
+      super(S3AInstrumentation.this.getIOStatistics());
+    }
+
+    /**
+     * Incrementing the counter also implements the metric alongside
+     * the IOStatistics value.
+     * @param key counter key
+     * @param value increment value.
+     * @return the value in the wrapped IOStatistics.
+     */
+    @Override
+    public long incrementCounter(final String key, final long value) {
+      incrementMutableCounter(key, value);
+      return super.incrementCounter(key, value);
+    }
+
+    @Override
+    public DurationTracker trackDuration(final String key, final long count) {
+      return S3AInstrumentation.this.trackDuration(key, count);
+    }
+
+    @Override
+    public DurationTracker trackDuration(final String key) {
+      return S3AInstrumentation.this.trackDuration(key);
+    }
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java
index 1029548..725a7fe 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java
@@ -66,6 +66,7 @@ public class S3ALocatedFileStatus extends LocatedFileStatus {
   /**
    * Generate an S3AFileStatus instance, including etag and
    * version ID, if present.
+   * @return the S3A status.
    */
   public S3AFileStatus toS3AFileStatus() {
     return new S3AFileStatus(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
index 3729341..43398fc 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
 import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
 
 import javax.annotation.Nullable;
 
@@ -55,6 +56,8 @@ public class S3AReadOpContext extends S3AOpContext {
    */
   private final long readahead;
 
+  private final AuditSpan auditSpan;
+
   /**
    * Instantiate.
    * @param path path of read
@@ -65,8 +68,9 @@ public class S3AReadOpContext extends S3AOpContext {
    * @param instrumentation statistics context
    * @param dstFileStatus target file status
    * @param inputPolicy the input policy
-   * @param readahead readahead for GET operations/skip, etc.
    * @param changeDetectionPolicy change detection policy.
+   * @param readahead readahead for GET operations/skip, etc.
+   * @param auditSpan active audit
    */
   public S3AReadOpContext(
       final Path path,
@@ -78,11 +82,13 @@ public class S3AReadOpContext extends S3AOpContext {
       FileStatus dstFileStatus,
       S3AInputPolicy inputPolicy,
       ChangeDetectionPolicy changeDetectionPolicy,
-      final long readahead) {
+      final long readahead,
+      final AuditSpan auditSpan) {
 
     super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation,
         dstFileStatus);
     this.path = checkNotNull(path);
+    this.auditSpan = auditSpan;
     Preconditions.checkArgument(readahead >= 0,
         "invalid readahead %d", readahead);
     this.inputPolicy = checkNotNull(inputPolicy);
@@ -133,6 +139,14 @@ public class S3AReadOpContext extends S3AOpContext {
     return readahead;
   }
 
+  /**
+   * Get the audit which was active when the file was opened.
+   * @return active span
+   */
+  public AuditSpan getAuditSpan() {
+    return auditSpan;
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 314f13f..b6af8e7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -34,7 +34,6 @@ import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.MultiObjectDeleteException;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
-
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 
@@ -42,7 +41,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
@@ -91,6 +89,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
 import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
 
 /**
  * Utility methods for S3A code.
@@ -201,7 +200,8 @@ public final class S3AUtils {
       }
       if (exception instanceof CredentialInitializationException) {
         // the exception raised by AWSCredentialProvider list if the
-        // credentials were not accepted.
+        // credentials were not accepted,
+        // or auditing blocked the operation.
         return (AccessDeniedException)new AccessDeniedException(path, null,
             exception.toString()).initCause(exception);
       }
@@ -1423,23 +1423,19 @@ public final class S3AUtils {
    * an array. Given tombstones are filtered out. If the iterator
    * does return any item, an empty array is returned.
    * @param iterator a non-null iterator
-   * @param tombstones
+   * @param tombstones possibly empty set of tombstones
    * @return a possibly-empty array of file status entries
-   * @throws IOException
+   * @throws IOException failure
    */
   public static S3AFileStatus[] iteratorToStatuses(
       RemoteIterator<S3AFileStatus> iterator, Set<Path> tombstones)
       throws IOException {
-    List<FileStatus> statuses = new ArrayList<>();
-
-    while (iterator.hasNext()) {
-      S3AFileStatus status = iterator.next();
-      if (!tombstones.contains(status.getPath())) {
-        statuses.add(status);
-      }
-    }
-
-    return statuses.toArray(new S3AFileStatus[0]);
+    // this will close the span afterwards
+    RemoteIterator<S3AFileStatus> source = filteringRemoteIterator(iterator,
+        st -> !tombstones.contains(st.getPath()));
+    S3AFileStatus[] statuses = RemoteIterators
+        .toArray(source, new S3AFileStatus[0]);
+    return statuses;
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index dbb39fb..5ef99ed 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -115,6 +115,11 @@ public interface S3ClientFactory {
      */
     private String userAgentSuffix = "";
 
+    /**
+     * List of request handlers to include in the chain
+     * of request execution in the SDK.
+     * @return the handler list
+     */
     public List<RequestHandler2> getRequestHandlers() {
       return requestHandlers;
     }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index c613c06..7890e2d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.audit.AuditStatisticNames;
 import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum;
 import org.apache.hadoop.fs.statistics.StoreStatisticNames;
 import org.apache.hadoop.fs.statistics.StreamStatisticNames;
@@ -30,6 +31,7 @@ import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_COUNTER
 import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_DURATION;
 import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_GAUGE;
 import static org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum.TYPE_QUANTILE;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES;
 
 /**
  * Statistic which are collected in S3A.
@@ -93,14 +95,18 @@ public enum Statistic {
       StoreStatisticNames.OP_ABORT,
       "Calls of abort()",
       TYPE_DURATION),
+  INVOCATION_ACCESS(
+      StoreStatisticNames.OP_ACCESS,
+      "Calls of access()",
+      TYPE_DURATION),
   INVOCATION_COPY_FROM_LOCAL_FILE(
       StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE,
       "Calls of copyFromLocalFile()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_CREATE(
       StoreStatisticNames.OP_CREATE,
       "Calls of create()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_CREATE_NON_RECURSIVE(
       StoreStatisticNames.OP_CREATE_NON_RECURSIVE,
       "Calls of createNonRecursive()",
@@ -108,35 +114,39 @@ public enum Statistic {
   INVOCATION_DELETE(
       StoreStatisticNames.OP_DELETE,
       "Calls of delete()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_EXISTS(
       StoreStatisticNames.OP_EXISTS,
       "Calls of exists()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
+  INVOCATION_GET_CONTENT_SUMMARY(
+      StoreStatisticNames.OP_GET_CONTENT_SUMMARY,
+      "Calls of getContentSummary()",
+      TYPE_DURATION),
   INVOCATION_GET_DELEGATION_TOKEN(
       StoreStatisticNames.OP_GET_DELEGATION_TOKEN,
       "Calls of getDelegationToken()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_GET_FILE_CHECKSUM(
       StoreStatisticNames.OP_GET_FILE_CHECKSUM,
       "Calls of getFileChecksum()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_GET_FILE_STATUS(
       StoreStatisticNames.OP_GET_FILE_STATUS,
       "Calls of getFileStatus()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_GLOB_STATUS(
       StoreStatisticNames.OP_GLOB_STATUS,
       "Calls of globStatus()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_IS_DIRECTORY(
       StoreStatisticNames.OP_IS_DIRECTORY,
       "Calls of isDirectory()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_IS_FILE(
       StoreStatisticNames.OP_IS_FILE,
       "Calls of isFile()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_HFLUSH(
       StoreStatisticNames.OP_HFLUSH,
       "Calls of hflush()",
@@ -148,7 +158,7 @@ public enum Statistic {
   INVOCATION_LIST_FILES(
       StoreStatisticNames.OP_LIST_FILES,
       "Calls of listFiles()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_LIST_LOCATED_STATUS(
       StoreStatisticNames.OP_LIST_LOCATED_STATUS,
       "Calls of listLocatedStatus()",
@@ -156,11 +166,11 @@ public enum Statistic {
   INVOCATION_LIST_STATUS(
       StoreStatisticNames.OP_LIST_STATUS,
       "Calls of listStatus()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_MKDIRS(
       StoreStatisticNames.OP_MKDIRS,
       "Calls of mkdirs()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   INVOCATION_OPEN(
       StoreStatisticNames.OP_OPEN,
       "Calls of open()",
@@ -168,7 +178,7 @@ public enum Statistic {
   INVOCATION_RENAME(
       StoreStatisticNames.OP_RENAME,
       "Calls of rename()",
-      TYPE_COUNTER),
+      TYPE_DURATION),
 
   /* The XAttr API metrics are all durations */
   INVOCATION_XATTR_GET_MAP(
@@ -215,15 +225,15 @@ public enum Statistic {
   OBJECT_MULTIPART_UPLOAD_INITIATED(
       StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_INITIATED,
       "Object multipart upload initiated",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   OBJECT_MULTIPART_UPLOAD_ABORTED(
       StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED,
       "Object multipart upload aborted",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   OBJECT_PUT_REQUESTS(
       StoreStatisticNames.OBJECT_PUT_REQUEST,
       "Object put/multipart upload count",
-      TYPE_COUNTER),
+      TYPE_DURATION),
   OBJECT_PUT_REQUESTS_COMPLETED(
       StoreStatisticNames.OBJECT_PUT_REQUEST_COMPLETED,
       "Object put/multipart upload completed count",
@@ -421,7 +431,7 @@ public enum Statistic {
       "Count of bytes uploaded duing commit operations",
       TYPE_COUNTER),
   COMMITTER_COMMITS_FAILED(
-      "committer_commits"+ StoreStatisticNames.SUFFIX_FAILURES,
+      "committer_commits"+ SUFFIX_FAILURES,
       "Count of commits failed",
       TYPE_COUNTER),
   COMMITTER_COMMITS_ABORTED(
@@ -487,6 +497,9 @@ public enum Statistic {
 
 
   /* General Store operations */
+  STORE_EXISTS_PROBE(StoreStatisticNames.STORE_EXISTS_PROBE,
+      "Store Existence Probe",
+      TYPE_DURATION),
   STORE_IO_REQUEST(StoreStatisticNames.STORE_IO_REQUEST,
       "requests made of the remote store",
       TYPE_COUNTER),
@@ -538,9 +551,32 @@ public enum Statistic {
       StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED,
       "Multipart Upload Completed",
       TYPE_COUNTER),
+  MULTIPART_UPLOAD_LIST(
+      StoreStatisticNames.MULTIPART_UPLOAD_LIST,
+      "Multipart Upload List",
+      TYPE_DURATION),
   MULTIPART_UPLOAD_STARTED(
       StoreStatisticNames.MULTIPART_UPLOAD_STARTED,
       "Multipart Upload Started",
+      TYPE_COUNTER),
+
+  /* Audit statistics. */
+  /* If more are added: update AuditTestSupport to include the new values. */
+  AUDIT_ACCESS_CHECK_FAILURE(
+      AuditStatisticNames.AUDIT_ACCESS_CHECK_FAILURE,
+      "Audit access check was rejected",
+      TYPE_COUNTER),
+  AUDIT_SPAN_CREATION(
+      AuditStatisticNames.AUDIT_SPAN_CREATION,
+      "Audit Span Created",
+      TYPE_COUNTER),
+  AUDIT_FAILURE(
+      AuditStatisticNames.AUDIT_FAILURE,
+      "Audit failure/rejection",
+      TYPE_COUNTER),
+  AUDIT_REQUEST_EXECUTION(
+      AuditStatisticNames.AUDIT_REQUEST_EXECUTION,
+      "AWS request made",
       TYPE_COUNTER);
 
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 8b71fc3..2eb4841 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -23,7 +23,6 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -51,19 +50,20 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.s3a.api.RequestFactory;
+import org.apache.hadoop.fs.s3a.impl.StoreContext;
 import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3a.select.SelectBinding;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.fs.store.audit.AuditSpanSource;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
-import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.longOption;
-import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
-import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT;
+import static org.apache.hadoop.fs.store.audit.AuditingFunctions.withinAuditSpan;
 
 /**
  * Helper for low-level operations against an S3 Bucket for writing data,
@@ -87,6 +87,17 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_
  * </ul>
  *
  * This API is for internal use only.
+ * Span scoping: This helper is instantiated with span; it will be used
+ * before operations which query/update S3
+ *
+ * History
+ * <pre>
+ * - A nested class in S3AFileSystem
+ * - Single shared instance created and reused.
+ * - [HADOOP-13786] A separate class, single instance in S3AFS
+ * - [HDFS-13934] Split into interface and implementation
+ * - [HADOOP-15711] Adds audit tracking; one instance per use.
+ * </pre>
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -117,21 +128,49 @@ public class WriteOperationHelper implements WriteOperations {
   private final S3AStatisticsContext statisticsContext;
 
   /**
+   * Store Context; extracted from owner.
+   */
+  private final StoreContext storeContext;
+
+  /**
+   * Source of Audit spans.
+   */
+  private final AuditSpanSource auditSpanSource;
+
+  /**
+   * Audit Span.
+   */
+  private AuditSpan auditSpan;
+
+  /**
+   * Factory for AWS requests.
+   */
+  private final RequestFactory requestFactory;
+
+  /**
    * Constructor.
    * @param owner owner FS creating the helper
    * @param conf Configuration object
    * @param statisticsContext statistics context
+   * @param auditSpanSource source of spans
+   * @param auditSpan span to activate
    *
    */
   protected WriteOperationHelper(S3AFileSystem owner,
       Configuration conf,
-      S3AStatisticsContext statisticsContext) {
+      S3AStatisticsContext statisticsContext,
+      final AuditSpanSource auditSpanSource,
+      final AuditSpan auditSpan) {
     this.owner = owner;
     this.invoker = new Invoker(new S3ARetryPolicy(conf),
         this::operationRetried);
     this.conf = conf;
     this.statisticsContext = statisticsContext;
-    bucket = owner.getBucket();
+    this.storeContext = owner.createStoreContext();
+    this.bucket = owner.getBucket();
+    this.auditSpanSource = auditSpanSource;
+    this.auditSpan = checkNotNull(auditSpan);
+    this.requestFactory = owner.getRequestFactory();
   }
 
   /**
@@ -150,6 +189,7 @@ public class WriteOperationHelper implements WriteOperations {
 
   /**
    * Execute a function with retry processing.
+   * Also activates the current span.
    * @param <T> type of return value
    * @param action action to execute (used in error messages)
    * @param path path of work (used in error messages)
@@ -164,11 +204,34 @@ public class WriteOperationHelper implements WriteOperations {
       boolean idempotent,
       CallableRaisingIOE<T> operation)
       throws IOException {
-
+    activateAuditSpan();
     return invoker.retry(action, path, idempotent, operation);
   }
 
   /**
+   * Get the audit span this object was created with.
+   * @return the audit span
+   */
+  public AuditSpan getAuditSpan() {
+    return auditSpan;
+  }
+
+  /**
+   * Activate the audit span.
+   * @return the span
+   */
+  private AuditSpan activateAuditSpan() {
+    return auditSpan.activate();
+  }
+
+  /**
+   * Deactivate the audit span.
+   */
+  private void deactivateAuditSpan() {
+    auditSpan.deactivate();
+  }
+
+  /**
    * Create a {@link PutObjectRequest} request against the specific key.
    * @param destKey destination key
    * @param inputStream source data.
@@ -176,15 +239,18 @@ public class WriteOperationHelper implements WriteOperations {
    * @param headers optional map of custom headers.
    * @return the request
    */
+  @Retries.OnceRaw
   public PutObjectRequest createPutObjectRequest(String destKey,
       InputStream inputStream,
       long length,
       final Map<String, String> headers) {
+    activateAuditSpan();
     ObjectMetadata objectMetadata = newObjectMetadata(length);
     if (headers != null) {
       objectMetadata.setUserMetadata(headers);
     }
-    return owner.newPutObjectRequest(destKey,
+    return getRequestFactory().newPutObjectRequest(
+        destKey,
         objectMetadata,
         inputStream);
   }
@@ -195,13 +261,16 @@ public class WriteOperationHelper implements WriteOperations {
    * @param sourceFile source file
    * @return the request
    */
+  @Retries.OnceRaw
   public PutObjectRequest createPutObjectRequest(String dest,
       File sourceFile) {
     Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE,
         "File length is too big for a single PUT upload");
-    return owner.newPutObjectRequest(dest,
-        newObjectMetadata((int) sourceFile.length()),
-        sourceFile);
+    activateAuditSpan();
+    return getRequestFactory().
+        newPutObjectRequest(dest,
+            newObjectMetadata((int) sourceFile.length()),
+            sourceFile);
   }
 
   /**
@@ -227,7 +296,7 @@ public class WriteOperationHelper implements WriteOperations {
    * @return a new metadata instance
    */
   public ObjectMetadata newObjectMetadata(long length) {
-    return owner.newObjectMetadata(length);
+    return getRequestFactory().newObjectMetadata(length);
   }
 
   /**
@@ -240,15 +309,16 @@ public class WriteOperationHelper implements WriteOperations {
   @Retries.RetryTranslated
   public String initiateMultiPartUpload(String destKey) throws IOException {
     LOG.debug("Initiating Multipart upload to {}", destKey);
-    final InitiateMultipartUploadRequest initiateMPURequest =
-        new InitiateMultipartUploadRequest(bucket,
-            destKey,
-            newObjectMetadata(-1));
-    initiateMPURequest.setCannedACL(owner.getCannedACL());
-    owner.setOptionalMultipartUploadRequestParameters(initiateMPURequest);
-
-    return retry("initiate MultiPartUpload", destKey, true,
-        () -> owner.initiateMultipartUpload(initiateMPURequest).getUploadId());
+    try (AuditSpan span = activateAuditSpan()) {
+      return retry("initiate MultiPartUpload", destKey, true,
+          () -> {
+            final InitiateMultipartUploadRequest initiateMPURequest =
+                getRequestFactory().newMultipartUploadRequest(
+                    destKey);
+            return owner.initiateMultipartUpload(initiateMPURequest)
+                .getUploadId();
+          });
+    }
   }
 
   /**
@@ -278,23 +348,22 @@ public class WriteOperationHelper implements WriteOperations {
       throw new PathIOException(destKey,
           "No upload parts in multipart upload");
     }
-    CompleteMultipartUploadResult uploadResult =
-        invoker.retry("Completing multipart upload", destKey,
-            true,
-            retrying,
-            () -> {
-              // a copy of the list is required, so that the AWS SDK doesn't
-              // attempt to sort an unmodifiable list.
-              return owner.getAmazonS3Client().completeMultipartUpload(
-                  new CompleteMultipartUploadRequest(bucket,
-                      destKey,
-                      uploadId,
-                      new ArrayList<>(partETags)));
-            }
-    );
-    owner.finishedWrite(destKey, length, uploadResult.getETag(),
-        uploadResult.getVersionId(), operationState);
-    return uploadResult;
+    try (AuditSpan span = activateAuditSpan()) {
+      CompleteMultipartUploadResult uploadResult;
+      uploadResult = invoker.retry("Completing multipart upload", destKey,
+          true,
+          retrying,
+          () -> {
+            final CompleteMultipartUploadRequest request =
+                getRequestFactory().newCompleteMultipartUploadRequest(
+                    destKey, uploadId, partETags);
+            return owner.getAmazonS3Client().completeMultipartUpload(
+                  request);
+          });
+      owner.finishedWrite(destKey, length, uploadResult.getETag(),
+          uploadResult.getVersionId(), operationState);
+      return uploadResult;
+    }
   }
 
   /**
@@ -351,16 +420,17 @@ public class WriteOperationHelper implements WriteOperations {
           destKey,
           true,
           retrying,
-          () -> owner.abortMultipartUpload(
-              destKey,
-              uploadId));
+          withinAuditSpan(getAuditSpan(), () ->
+              owner.abortMultipartUpload(
+                  destKey, uploadId)));
     } else {
       // single pass attempt.
       once("Aborting multipart upload ID " + uploadId,
           destKey,
-          () -> owner.abortMultipartUpload(
-              destKey,
-              uploadId));
+          withinAuditSpan(getAuditSpan(), () ->
+              owner.abortMultipartUpload(
+                  destKey,
+                  uploadId)));
     }
   }
 
@@ -373,7 +443,8 @@ public class WriteOperationHelper implements WriteOperations {
   public void abortMultipartUpload(MultipartUpload upload)
       throws IOException {
     invoker.retry("Aborting multipart commit", upload.getKey(), true,
-        () -> owner.abortMultipartUpload(upload));
+        withinAuditSpan(getAuditSpan(),
+            () -> owner.abortMultipartUpload(upload)));
   }
 
 
@@ -389,7 +460,7 @@ public class WriteOperationHelper implements WriteOperations {
       throws IOException {
     LOG.debug("Aborting multipart uploads under {}", prefix);
     int count = 0;
-    List<MultipartUpload> multipartUploads = owner.listMultipartUploads(prefix);
+    List<MultipartUpload> multipartUploads = listMultipartUploads(prefix);
     LOG.debug("Number of outstanding uploads: {}", multipartUploads.size());
     for (MultipartUpload upload: multipartUploads) {
       try {
@@ -402,6 +473,14 @@ public class WriteOperationHelper implements WriteOperations {
     return count;
   }
 
+  @Override
+  @Retries.RetryTranslated
+  public List<MultipartUpload> listMultipartUploads(final String prefix)
+      throws IOException {
+    activateAuditSpan();
+    return owner.listMultipartUploads(prefix);
+  }
+
   /**
    * Abort a multipart commit operation.
    * @param destKey destination key of ongoing operation
@@ -409,6 +488,7 @@ public class WriteOperationHelper implements WriteOperations {
    * @throws IOException on problems.
    * @throws FileNotFoundException if the abort ID is unknown
    */
+  @Override
   @Retries.RetryTranslated
   public void abortMultipartCommit(String destKey, String uploadId)
       throws IOException {
@@ -423,6 +503,7 @@ public class WriteOperationHelper implements WriteOperations {
    * in {@code offset} and a length of block in {@code size} equal to
    * or less than the remaining bytes.
    * The part number must be less than 10000.
+   * Retry policy is once-translated; to much effort
    * @param destKey destination key of ongoing operation
    * @param uploadId ID of ongoing upload
    * @param partNumber current part number of the upload
@@ -431,9 +512,11 @@ public class WriteOperationHelper implements WriteOperations {
    * @param sourceFile optional source file.
    * @param offset offset in file to start reading.
    * @return the request.
-   * @throws IllegalArgumentException if the parameters are invalid -including
+   * @throws IllegalArgumentException if the parameters are invalid.
    * @throws PathIOException if the part number is out of range.
    */
+  @Override
+  @Retries.OnceTranslated
   public UploadPartRequest newUploadPartRequest(
       String destKey,
       String uploadId,
@@ -441,52 +524,17 @@ public class WriteOperationHelper implements WriteOperations {
       int size,
       InputStream uploadStream,
       File sourceFile,
-      Long offset) throws PathIOException {
-    checkNotNull(uploadId);
-    // exactly one source must be set; xor verifies this
-    checkArgument((uploadStream != null) ^ (sourceFile != null),
-        "Data source");
-    checkArgument(size >= 0, "Invalid partition size %s", size);
-    checkArgument(partNumber > 0,
-        "partNumber must be between 1 and %s inclusive, but is %s",
-            DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber);
-
-    LOG.debug("Creating part upload request for {} #{} size {}",
-        uploadId, partNumber, size);
-    long partCountLimit = longOption(conf,
-        UPLOAD_PART_COUNT_LIMIT,
-        DEFAULT_UPLOAD_PART_COUNT_LIMIT,
-        1);
-    if (partCountLimit != DEFAULT_UPLOAD_PART_COUNT_LIMIT) {
-      LOG.warn("Configuration property {} shouldn't be overridden by client",
-              UPLOAD_PART_COUNT_LIMIT);
-    }
-    final String pathErrorMsg = "Number of parts in multipart upload exceeded."
-        + " Current part count = %s, Part count limit = %s ";
-    if (partNumber > partCountLimit) {
-      throw new PathIOException(destKey,
-          String.format(pathErrorMsg, partNumber, partCountLimit));
-    }
-    UploadPartRequest request = new UploadPartRequest()
-        .withBucketName(bucket)
-        .withKey(destKey)
-        .withUploadId(uploadId)
-        .withPartNumber(partNumber)
-        .withPartSize(size);
-    if (uploadStream != null) {
-      // there's an upload stream. Bind to it.
-      request.setInputStream(uploadStream);
-    } else {
-      checkArgument(sourceFile.exists(),
-          "Source file does not exist: %s", sourceFile);
-      checkArgument(offset >= 0, "Invalid offset %s", offset);
-      long length = sourceFile.length();
-      checkArgument(offset == 0 || offset < length,
-          "Offset %s beyond length of file %s", offset, length);
-      request.setFile(sourceFile);
-      request.setFileOffset(offset);
-    }
-    return request;
+      Long offset) throws IOException {
+    return once("upload part request", destKey,
+        withinAuditSpan(getAuditSpan(), () ->
+            getRequestFactory().newUploadPartRequest(
+                destKey,
+                uploadId,
+                partNumber,
+                size,
+                uploadStream,
+                sourceFile,
+                offset)));
   }
 
   /**
@@ -514,7 +562,8 @@ public class WriteOperationHelper implements WriteOperations {
       throws IOException {
     return retry("Writing Object",
         putObjectRequest.getKey(), true,
-        () -> owner.putObjectDirect(putObjectRequest));
+        withinAuditSpan(getAuditSpan(), () ->
+            owner.putObjectDirect(putObjectRequest)));
   }
 
   /**
@@ -529,7 +578,8 @@ public class WriteOperationHelper implements WriteOperations {
     // no retry; rely on xfer manager logic
     return retry("Writing Object",
         putObjectRequest.getKey(), true,
-        () -> owner.executePut(putObjectRequest, null));
+        withinAuditSpan(getAuditSpan(), () ->
+            owner.executePut(putObjectRequest, null)));
   }
 
   /**
@@ -543,13 +593,12 @@ public class WriteOperationHelper implements WriteOperations {
   public void revertCommit(String destKey,
       @Nullable BulkOperationState operationState) throws IOException {
     once("revert commit", destKey,
-        () -> {
+        withinAuditSpan(getAuditSpan(), () -> {
           Path destPath = owner.keyToQualifiedPath(destKey);
           owner.deleteObjectAtPath(destPath,
               destKey, true, operationState);
           owner.maybeCreateFakeParentDirectory(destPath);
-        }
-    );
+        }));
   }
 
   /**
@@ -620,10 +669,11 @@ public class WriteOperationHelper implements WriteOperations {
   public UploadPartResult uploadPart(UploadPartRequest request)
       throws IOException {
     return retry("upload part #" + request.getPartNumber()
-        + " upload ID "+ request.getUploadId(),
+            + " upload ID " + request.getUploadId(),
         request.getKey(),
         true,
-        () -> owner.uploadPart(request));
+        withinAuditSpan(getAuditSpan(),
+            () -> owner.uploadPart(request)));
   }
 
   /**
@@ -642,10 +692,10 @@ public class WriteOperationHelper implements WriteOperations {
    * @return the request
    */
   public SelectObjectContentRequest newSelectRequest(Path path) {
-    SelectObjectContentRequest request = new SelectObjectContentRequest();
-    request.setBucketName(bucket);
-    request.setKey(owner.pathToKey(path));
-    return request;
+    try (AuditSpan span = getAuditSpan()) {
+      return getRequestFactory().newSelectRequest(
+          storeContext.pathToKey(path));
+    }
   }
 
   /**
@@ -664,6 +714,8 @@ public class WriteOperationHelper implements WriteOperations {
       final SelectObjectContentRequest request,
       final String action)
       throws IOException {
+    // no setting of span here as the select binding is (statically) created
+    // without any span.
     String bucketName = request.getBucketName();
     Preconditions.checkArgument(bucket.equals(bucketName),
         "wrong bucket: %s", bucketName);
@@ -676,7 +728,7 @@ public class WriteOperationHelper implements WriteOperations {
         action,
         source.toString(),
         true,
-        () -> {
+        withinAuditSpan(getAuditSpan(), () -> {
           try (DurationInfo ignored =
                    new DurationInfo(LOG, "S3 Select operation")) {
             try {
@@ -691,11 +743,35 @@ public class WriteOperationHelper implements WriteOperations {
               throw e;
             }
           }
-        });
+        }));
+  }
+
+  @Override
+  public AuditSpan createSpan(final String operation,
+      @Nullable final String path1,
+      @Nullable final String path2) throws IOException {
+    return auditSpanSource.createSpan(operation, path1, path2);
   }
 
   @Override
   public void incrementWriteOperations() {
     owner.incrementWriteOperations();
   }
+
+  /**
+   * Deactivate the audit span.
+    */
+  @Override
+  public void close() throws IOException {
+    deactivateAuditSpan();
+  }
+
+  /**
+   * Get the request factory which uses this store's audit span.
+   * @return the request factory.
+   */
+  public RequestFactory getRequestFactory() {
+    return requestFactory;
+  }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
index 0a8150c..9400ef2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import javax.annotation.Nullable;
+import java.io.Closeable;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -43,6 +44,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
+import org.apache.hadoop.fs.store.audit.AuditSpanSource;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 /**
@@ -54,7 +56,7 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
  * use `WriteOperationHelper` directly.
  * @since Hadoop 3.3.0
  */
-public interface WriteOperations {
+public interface WriteOperations extends AuditSpanSource, Closeable {
 
   /**
    * Execute a function with retry processing.
@@ -185,6 +187,16 @@ public interface WriteOperations {
       throws IOException;
 
   /**
+   * Abort multipart uploads under a path: limited to the first
+   * few hundred.
+   * @param prefix prefix for uploads to abort
+   * @return a count of aborts
+   * @throws IOException trouble; FileNotFoundExceptions are swallowed.
+   */
+  List<MultipartUpload> listMultipartUploads(String prefix)
+      throws IOException;
+
+  /**
    * Abort a multipart commit operation.
    * @param destKey destination key of ongoing operation
    * @param uploadId multipart operation Id
@@ -210,7 +222,7 @@ public interface WriteOperations {
    * @param sourceFile optional source file.
    * @param offset offset in file to start reading.
    * @return the request.
-   * @throws IllegalArgumentException if the parameters are invalid -including
+   * @throws IllegalArgumentException if the parameters are invalid
    * @throws PathIOException if the part number is out of range.
    */
   UploadPartRequest newUploadPartRequest(
@@ -220,7 +232,7 @@ public interface WriteOperations {
       int size,
       InputStream uploadStream,
       File sourceFile,
-      Long offset) throws PathIOException;
+      Long offset) throws IOException;
 
   /**
    * PUT an object directly (i.e. not via the transfer manager).
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java
new file mode 100644
index 0000000..9bffcc9
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java
@@ -0,0 +1,296 @@
+/*
+ * 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.s3a.api;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.InputStream;
+import java.util.List;
+import java.util.Optional;
+
+import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CannedAccessControlList;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CopyObjectRequest;
+import com.amazonaws.services.s3.model.DeleteObjectRequest;
+import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
+import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
+import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
+import com.amazonaws.services.s3.model.SSECustomerKey;
+import com.amazonaws.services.s3.model.SelectObjectContentRequest;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
+import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
+
+/**
+ * Factory for S3 objects.
+ *
+ * This is where the owner FS's {@code prepareRequest()}
+ * callback is invoked to mark up a request for this span.
+ *
+ * All AWS request objects MUST BE created through this, in
+ * the active audit span.
+ * The standard implementation provides a callback for the S3AFS or
+ * tests to be invoked to prepare each request.
+ * Such callbacks SHOULD NOT raise exceptions other
+ * than argument validation exceptions.
+ * as there are no guarantees how they are processed.
+ * That is: no guarantees of retry or translation.
+ */
+public interface RequestFactory {
+
+  /**
+   * Set the encryption secrets for all subsequent requests.
+   * @param secrets encryption secrets.
+   */
+  void setEncryptionSecrets(EncryptionSecrets secrets);
+
+  /**
+   * Get the canned ACL of this FS.
+   * @return an ACL, if any
+   */
+  CannedAccessControlList getCannedACL();
+
+  /**
+   * Create the AWS SDK structure used to configure SSE,
+   * if the encryption secrets contain the information/settings for this.
+   * @return an optional set of KMS Key settings
+   */
+  Optional<SSEAwsKeyManagementParams> generateSSEAwsKeyParams();
+
+  /**
+   * Create the SSE-C structure for the AWS SDK, if the encryption secrets
+   * contain the information/settings for this.
+   * This will contain a secret extracted from the bucket/configuration.
+   * @return an optional customer key.
+   */
+  Optional<SSECustomerKey> generateSSECustomerKey();
+
+  /**
+   * Get the encryption algorithm of this endpoint.
+   * @return the encryption algorithm.
+   */
+  S3AEncryptionMethods getServerSideEncryptionAlgorithm();
+
+  /**
+   * Create a new object metadata instance.
+   * Any standard metadata headers are added here, for example:
+   * encryption.
+   *
+   * @param length length of data to set in header; Ignored if negative
+   * @return a new metadata instance
+   */
+  ObjectMetadata newObjectMetadata(long length);
+
+  /**
+   * Create a copy request.
+   * This includes the work of copying the relevant parts
+   * of the metadata from the source
+   * @param srcKey source
+   * @param dstKey destination
+   * @param srcom source object metadata.
+   * @return the request
+   */
+  CopyObjectRequest newCopyObjectRequest(String srcKey,
+      String dstKey,
+      ObjectMetadata srcom);
+
+  /**
+   * Create a putObject request.
+   * Adds the ACL and metadata
+   * @param key key of object
+   * @param metadata metadata header
+   * @param srcfile source file
+   * @return the request
+   */
+  PutObjectRequest newPutObjectRequest(String key,
+      ObjectMetadata metadata, File srcfile);
+
+  /**
+   * Create a {@link PutObjectRequest} request.
+   * The metadata is assumed to have been configured with the size of the
+   * operation.
+   * @param key key of object
+   * @param metadata metadata header
+   * @param inputStream source data.
+   * @return the request
+   */
+  PutObjectRequest newPutObjectRequest(String key,
+      ObjectMetadata metadata,
+      InputStream inputStream);
+
+  /**
+   * Create a {@link PutObjectRequest} request for creating
+   * an empty directory.
+   *
+   * @param directory destination directory.
+   * @return request for a zero byte upload.
+   */
+  PutObjectRequest newDirectoryMarkerRequest(String directory);
+
+  /**
+   * List all multipart uploads under a prefix.
+   * @param prefix prefix to list under
+   * @return the request.
+   */
+  ListMultipartUploadsRequest newListMultipartUploadsRequest(
+      @Nullable String prefix);
+
+  /**
+   * Abort a multipart upload.
+   * @param destKey destination object key
+   * @param uploadId ID of initiated upload
+   * @return the request.
+   */
+  AbortMultipartUploadRequest newAbortMultipartUploadRequest(
+      String destKey,
+      String uploadId);
+
+  /**
+   * Start a multipart upload.
+   * @param destKey destination object key
+   * @return the request.
+   */
+  InitiateMultipartUploadRequest newMultipartUploadRequest(
+      String destKey);
+
+  /**
+   * Complete a multipart upload.
+   * @param destKey destination object key
+   * @param uploadId ID of initiated upload
+   * @param partETags ordered list of etags
+   * @return the request.
+   */
+  CompleteMultipartUploadRequest newCompleteMultipartUploadRequest(
+      String destKey,
+      String uploadId,
+      List<PartETag> partETags);
+
+  /**
+   * Create a HEAD request.
+   * @param key key, may have trailing /
+   * @return the request.
+   */
+  GetObjectMetadataRequest newGetObjectMetadataRequest(String key);
+
+  /**
+   * Create a GET request.
+   * @param key object key
+   * @return the request.
+   */
+  GetObjectRequest newGetObjectRequest(String key);
+
+  /**
+   * Create and initialize a part request of a multipart upload.
+   * Exactly one of: {@code uploadStream} or {@code sourceFile}
+   * must be specified.
+   * A subset of the file may be posted, by providing the starting point
+   * in {@code offset} and a length of block in {@code size} equal to
+   * or less than the remaining bytes.
+   * @param destKey destination key of ongoing operation
+   * @param uploadId ID of ongoing upload
+   * @param partNumber current part number of the upload
+   * @param size amount of data
+   * @param uploadStream source of data to upload
+   * @param sourceFile optional source file.
+   * @param offset offset in file to start reading.
+   * @return the request.
+   * @throws PathIOException if the part number is out of range.
+   */
+  UploadPartRequest newUploadPartRequest(
+      String destKey,
+      String uploadId,
+      int partNumber,
+      int size,
+      InputStream uploadStream,
+      File sourceFile,
+      long offset) throws PathIOException;
+
+  /**
+   * Create a S3 Select request for the destination object.
+   * This does not build the query.
+   * @param key object key
+   * @return the request
+   */
+  SelectObjectContentRequest newSelectRequest(String key);
+
+  /**
+   * Create the (legacy) V1 list request.
+   * @param key key to list under
+   * @param delimiter delimiter for keys
+   * @param maxKeys maximum number in a list page.
+   * @return the request
+   */
+  ListObjectsRequest newListObjectsV1Request(String key,
+      String delimiter,
+      int maxKeys);
+
+  /**
+   * Create the next V1 page list request, following
+   * on from the previous response.
+   * @param prev previous response
+   * @return the request
+   */
+
+  ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest(
+      ObjectListing prev);
+
+  /**
+   * Create a V2 list request.
+   * This will be recycled for any subsequent requests.
+   * @param key key to list under
+   * @param delimiter delimiter for keys
+   * @param maxKeys maximum number in a list page.
+   * @return the request
+   */
+  ListObjectsV2Request newListObjectsV2Request(String key,
+      String delimiter,
+      int maxKeys);
+
+  /**
+   * Create a request to delete a single object.
+   * @param key object to delete
+   * @return the request
+   */
+  DeleteObjectRequest newDeleteObjectRequest(String key);
+
+  /**
+   * Bulk delete request.
+   * @param keysToDelete list of keys to delete.
+   * @param quiet should a bulk query be quiet, or should its result list
+   * all deleted keys?
+   * @return the request
+   */
+  DeleteObjectsRequest newBulkDeleteRequest(
+      List<DeleteObjectsRequest.KeyVersion> keysToDelete,
+      boolean quiet);
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java
similarity index 57%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java
index 54a8836..625c8a3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/package-info.java
@@ -16,27 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
-
 /**
- * Log exactly once, even across threads.
+ * Where the interfaces for classes implemented in {@code o.a.h.fs.s3a.impl}
+ * should go. This is to allow for extension points to use these interfaces
+ * without having any java module access to the .impl package.
+ *
+ * This is public for S3A extension points, however there are no
+ * guarantees of stability -changes may break things, possibly
+ * unintentionally.
  */
-public class LogExactlyOnce {
-
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
 
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
-  }
+@InterfaceAudience.LimitedPrivate("extensions")
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.api;
 
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
-  }
-}
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java
new file mode 100644
index 0000000..8134d5c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java
@@ -0,0 +1,160 @@
+/*
+ * 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.s3a.audit;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.Request;
+import com.amazonaws.Response;
+import com.amazonaws.SdkBaseException;
+import com.amazonaws.handlers.HandlerAfterAttemptContext;
+import com.amazonaws.handlers.HandlerBeforeAttemptContext;
+import com.amazonaws.http.HttpResponse;
+
+import org.apache.hadoop.fs.s3a.Retries;
+
+/**
+ * Callbacks for audit spans. This is implemented
+ * in the span manager as well as individual audit spans.
+ * If any of the code in a callback raises an InterruptedException,
+ * it must be caught and {@code Thread.interrupt()} called to
+ * redeclare the thread as interrupted. The AWS SDK will
+ * detect this and raise an exception.
+ *
+ * Look at the documentation for
+ * {@code com.amazonaws.handlers.IRequestHandler2} for details
+ * on the callbacks.
+ */
+public interface AWSAuditEventCallbacks {
+
+  /**
+   * Return a span ID which must be unique for all spans within
+   * everywhere. That effectively means part of the
+   * span SHOULD be derived from a UUID.
+   * Callers MUST NOT make any assumptions about the actual
+   * contents or structure of this string other than the
+   * uniqueness.
+   * @return a non-empty string
+   */
+  String getSpanId();
+
+  /**
+   * Get the name of the operation.
+   * @return the operation name.
+   */
+  String getOperationName();
+
+  /**
+   * Callback when a request is created in the S3A code.
+   * This is called in {@code RequestFactoryImpl} after
+   * each request is created.
+   * It is not invoked on any AWS requests created in the SDK.
+   * Avoid raising exceptions or talking to any remote service;
+   * this callback is for annotation rather than validation.
+   * @param request request request.
+   * @param <T> type of request
+   * @return the request, possibly modified.
+   */
+  default <T extends AmazonWebServiceRequest> T requestCreated(T request) {
+    return request;
+  }
+
+  /**
+   * Preflight preparation of AWS request.
+   * @param request request
+   * @param <T> type of request
+   * @return an updated request.
+   * @throws AuditFailureException for generic audit failures
+   * @throws SdkBaseException for other reasons.
+   */
+  @Retries.OnceRaw
+  default <T extends AmazonWebServiceRequest> T beforeExecution(T request)
+      throws AuditFailureException, SdkBaseException {
+    return request;
+  }
+
+  /**
+   * Callback after S3 responded to a request.
+   * @param request request
+   * @param response response.
+   * @throws AuditFailureException for generic audit failures
+   * @throws SdkBaseException for other reasons.
+   */
+  default void afterResponse(Request<?> request,
+      Response<?> response)
+      throws AuditFailureException, SdkBaseException {
+  }
+
+  /**
+   * Callback after a request resulted in an error.
+   * @param request request
+   * @param response response.
+   * @param exception exception raised.
+   * @throws AuditFailureException for generic audit failures
+   * @throws SdkBaseException for other reasons.
+   */
+  default void afterError(Request<?> request,
+      Response<?> response,
+      Exception exception)
+      throws AuditFailureException, SdkBaseException {
+  }
+
+  /**
+   * Request before marshalling.
+   * @param request request
+   * @return possibly modified request.
+   */
+  default AmazonWebServiceRequest beforeMarshalling(
+      AmazonWebServiceRequest request) {
+    return request;
+  }
+
+  /**
+   * Request before marshalling.
+   * @param request request
+   */
+  default void beforeRequest(Request<?> request) {
+  }
+
+  /**
+   * Before any attempt is made.
+   * @param context full context, including the request.
+   */
+  default void beforeAttempt(HandlerBeforeAttemptContext context) {
+  }
+
+  /**
+   * After any attempt is made.
+   * @param context full context, including the request.
+   */
+  default void afterAttempt(
+      HandlerAfterAttemptContext context) {
+  }
+
+  /**
+   * Before unmarshalling the response.
+   * @param request request made.
+   * @param httpResponse response received
+   * @return updated response.
+   */
+  default HttpResponse beforeUnmarshalling(
+      final Request<?> request,
+      final HttpResponse httpResponse) {
+    return httpResponse;
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java
new file mode 100644
index 0000000..a5a6dbc
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java
@@ -0,0 +1,296 @@
+/*
+ * 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.s3a.audit;
+
+import java.util.List;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
+import com.amazonaws.services.s3.model.CopyPartRequest;
+import com.amazonaws.services.s3.model.DeleteObjectRequest;
+import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.GetBucketLocationRequest;
+import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
+import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
+import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsRequest;
+import com.amazonaws.services.s3.model.ListObjectsV2Request;
+import com.amazonaws.services.s3.model.ObjectListing;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.SelectObjectContentRequest;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_ABORTED;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_LIST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_PART_PUT;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_STARTED;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_BULK_DELETE_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_DELETE_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_PUT_REQUEST;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_SELECT_REQUESTS;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_EXISTS_PROBE;
+
+/**
+ * Extract information from a request.
+ * Intended for reporting and error logs.
+ */
+public class AWSRequestAnalyzer {
+
+  /**
+   * Given an AWS request, try to analyze it to operation,
+   * read/write and path.
+   * @param request request.
+   * @return information about the request.
+   * @param <T> type of request.
+   */
+  public <T extends AmazonWebServiceRequest> RequestInfo analyze(T request) {
+
+    // this is where Scala's case statement would massively
+    // simplify life.
+    // Please Keep in Alphabetical Order.
+    if (request instanceof AbortMultipartUploadRequest) {
+      return writing(MULTIPART_UPLOAD_ABORTED,
+          ((AbortMultipartUploadRequest) request).getKey(),
+          0);
+    } else if (request instanceof CompleteMultipartUploadRequest) {
+      CompleteMultipartUploadRequest r
+          = (CompleteMultipartUploadRequest) request;
+      return writing(MULTIPART_UPLOAD_COMPLETED,
+          r.getKey(),
+          r.getPartETags().size());
+    } else if (request instanceof DeleteObjectRequest) {
+      // DeleteObject: single object
+      return writing(OBJECT_DELETE_REQUEST,
+          ((DeleteObjectRequest) request).getKey(),
+          1);
+    } else if (request instanceof DeleteObjectsRequest) {
+      // DeleteObjects: bulk delete
+      // use first key as the path
+      DeleteObjectsRequest r = (DeleteObjectsRequest) request;
+      List<DeleteObjectsRequest.KeyVersion> keys
+          = r.getKeys();
+      return writing(OBJECT_BULK_DELETE_REQUEST,
+          keys.isEmpty() ? null : keys.get(0).getKey(),
+          keys.size());
+    } else if (request instanceof GetBucketLocationRequest) {
+      GetBucketLocationRequest r = (GetBucketLocationRequest) request;
+      return reading(STORE_EXISTS_PROBE,
+          r.getBucketName(),
+          0);
+    } else if (request instanceof GetObjectMetadataRequest) {
+      return reading(ACTION_HTTP_HEAD_REQUEST,
+          ((GetObjectMetadataRequest) request).getKey(), 0);
+    } else if (request instanceof GetObjectRequest) {
+      GetObjectRequest r = (GetObjectRequest) request;
+      long[] range = r.getRange();
+      long size = range == null
+          ? -1
+          : range[1] - range[0];
+      return reading(ACTION_HTTP_GET_REQUEST,
+          r.getKey(),
+          size);
+    } else if (request instanceof InitiateMultipartUploadRequest) {
+      return writing(MULTIPART_UPLOAD_STARTED,
+          ((InitiateMultipartUploadRequest) request).getKey(),
+          0);
+    } else if (request instanceof ListMultipartUploadsRequest) {
+      ListMultipartUploadsRequest r
+          = (ListMultipartUploadsRequest) request;
+      return reading(MULTIPART_UPLOAD_LIST,
+          r.getPrefix(),
+          r.getMaxUploads());
+    } else if (request instanceof ListObjectsRequest) {
+      ListObjectsRequest r = (ListObjectsRequest) request;
+      return reading(OBJECT_LIST_REQUEST,
+          r.getPrefix(),
+          r.getMaxKeys());
+    } else if (request instanceof ListNextBatchOfObjectsRequest) {
+      ListNextBatchOfObjectsRequest r = (ListNextBatchOfObjectsRequest) request;
+      ObjectListing l = r.getPreviousObjectListing();
+      String prefix = "";
+      int size = 0;
+      if (l != null) {
+        prefix = l.getPrefix();
+        size = l.getMaxKeys();
+      }
+      return reading(OBJECT_LIST_REQUEST,
+          prefix,
+          size);
+    } else if (request instanceof ListObjectsV2Request) {
+      ListObjectsV2Request r = (ListObjectsV2Request) request;
+      return reading(OBJECT_LIST_REQUEST,
+          r.getPrefix(),
+          r.getMaxKeys());
+    } else if (request instanceof PutObjectRequest) {
+      PutObjectRequest r = (PutObjectRequest) request;
+      return writing(OBJECT_PUT_REQUEST,
+          r.getKey(),
+          0);
+    } else if (request instanceof SelectObjectContentRequest) {
+      SelectObjectContentRequest r =
+          (SelectObjectContentRequest) request;
+      return reading(OBJECT_SELECT_REQUESTS,
+          r.getKey(),
+          1);
+    } else if (request instanceof UploadPartRequest) {
+      UploadPartRequest r = (UploadPartRequest) request;
+      return writing(MULTIPART_UPLOAD_PART_PUT,
+          r.getKey(),
+          r.getPartSize());
+    }
+    // no explicit support, return classname
+    return writing(request.getClass().getName(), null, 0);
+  }
+
+  /**
+   * A request.
+   * @param verb verb
+   * @param mutating does this update the store
+   * @param key object/prefix, etc.
+   * @param size nullable size
+   * @return request info
+   */
+  private RequestInfo request(final String verb,
+      final boolean mutating,
+      final String key,
+      final Number size) {
+    return new RequestInfo(verb, mutating, key, size);
+  }
+
+  /**
+   * A read request.
+   * @param verb verb
+   * @param key object/prefix, etc.
+   * @param size nullable size
+   * @return request info
+   */
+  private RequestInfo reading(final String verb,
+      final String key, final Number size) {
+    return request(verb, false, key, size);
+  }
+
+  /**
+   * A write request of some form.
+   * @param verb verb
+   * @param key object/prefix, etc.
+   * @param size nullable size
+   * @return request info
+   */
+  private RequestInfo writing(final String verb,
+      final String key, final Number size) {
+    return request(verb, true, key, size);
+  }
+
+  /**
+   * Predicate which returns true if the request is of a kind which
+   * could be outside a span because of how the AWS SDK generates them.
+   * @param request request
+   * @return true if the transfer manager creates them.
+   */
+  public static final boolean
+      isRequestNotAlwaysInSpan(final Object request) {
+    return request instanceof CopyPartRequest
+        || request instanceof CompleteMultipartUploadRequest
+        || request instanceof GetBucketLocationRequest;
+  }
+
+  /**
+   * Info about a request.
+   */
+  public static final class RequestInfo {
+
+    /**
+     * Verb.
+     */
+    private String verb;
+
+    /**
+     * Is this a mutating call?
+     */
+    private boolean mutating;
+
+    /**
+     * Key if there is one; maybe first key in a list.
+     */
+    private String key;
+
+    /**
+     * Size, where the meaning of size depends on the request.
+     */
+    private long size;
+
+    /**
+     * Construct.
+     * @param verb operation/classname, etc.
+     * @param mutating does this update S3 State.
+     * @param key key/path/bucket operated on.
+     * @param size size of request (bytes, elements, limit...). Nullable.
+     */
+    private RequestInfo(final String verb,
+        final boolean mutating,
+        final String key,
+        final Number size) {
+      this.verb = verb;
+      this.mutating = mutating;
+      this.key = key;
+      this.size = toSafeLong(size);
+    }
+
+    public String getVerb() {
+      return verb;
+    }
+
+    public boolean isMutating() {
+      return mutating;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public long getSize() {
+      return size;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "{");
+      sb.append(verb);
+      if (key != null) {
+        sb.append(" '").append(key).append('\'');
+      }
+      sb.append(" size=").append(size);
+      sb.append(", mutating=").append(mutating);
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+  private static long toSafeLong(final Number size) {
+    return size != null ? size.longValue() : 0;
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java
similarity index 55%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java
index 54a8836..5cd2b96 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditFailureException.java
@@ -16,27 +16,26 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
+package org.apache.hadoop.fs.s3a.audit;
 
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
+import org.apache.hadoop.fs.s3a.CredentialInitializationException;
 
 /**
- * Log exactly once, even across threads.
+ * This is in the AWS exception tree so that exceptions raised in the
+ * AWS SDK are correctly reported up.
+ * It is a subclass of {@link CredentialInitializationException}
+ * so that
+ * {@code S3AUtils.translateException()} recognizes these exceptions
+ * and converts them to AccessDeniedException.
  */
-public class LogExactlyOnce {
-
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
+public class AuditFailureException extends CredentialInitializationException {
 
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
+  public AuditFailureException(final String message, final Throwable t) {
+    super(message, t);
   }
 
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
+  public AuditFailureException(final String message) {
+    super(message);
   }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java
new file mode 100644
index 0000000..7b9e669
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java
@@ -0,0 +1,137 @@
+/*
+ * 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.s3a.audit;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+import com.amazonaws.HandlerContextAware;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.audit.impl.ActiveAuditManagerS3A;
+import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor;
+import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_HANDLER_CONTEXT;
+
+/**
+ * Support for integrating auditing within the S3A code.
+ */
+public final class AuditIntegration {
+
+  /**
+   * Logging.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AuditIntegration.class);
+
+  private AuditIntegration() {
+  }
+
+  /**
+   * Create and start an audit manager.
+   * @param conf configuration
+   * @param iostatistics IOStatistics source.
+   * @return audit manager.
+   */
+  public static AuditManagerS3A createAndStartAuditManager(
+      Configuration conf,
+      IOStatisticsStore iostatistics) {
+    ActiveAuditManagerS3A auditManager = new ActiveAuditManagerS3A(
+        requireNonNull(iostatistics));
+    auditManager.init(conf);
+    auditManager.start();
+    LOG.debug("Started Audit Manager {}", auditManager);
+    return auditManager;
+  }
+
+  /**
+   * Return a stub audit manager.
+   * @return an audit manager.
+   */
+  public static AuditManagerS3A stubAuditManager() {
+    return new NoopAuditManagerS3A();
+  }
+
+  /**
+   * Create and initialize an audit service.
+   * The service start operation is not called: that is left to
+   * the caller.
+   * @param conf configuration to read the key from and to use to init
+   * the service.
+   * @param key key containing the classname
+   * @param options options to initialize with.
+   * @return instantiated class.
+   * @throws IOException failure to initialise.
+   */
+  public static OperationAuditor createAndInitAuditor(
+      Configuration conf,
+      String key,
+      OperationAuditorOptions options) throws IOException {
+    final Class<? extends OperationAuditor> auditClassname
+        = conf.getClass(
+        key,
+        LoggingAuditor.class,
+        OperationAuditor.class);
+    try {
+      LOG.debug("Auditor class is {}", auditClassname);
+      final Constructor<? extends OperationAuditor> constructor
+          = auditClassname.getConstructor();
+      final OperationAuditor instance = constructor.newInstance();
+      instance.init(options);
+      return instance;
+    } catch (NoSuchMethodException | InstantiationException
+        | RuntimeException
+        | IllegalAccessException | InvocationTargetException e) {
+      throw new IOException("Failed to instantiate class "
+          + auditClassname
+          + " defined in " + key
+          + ": " + e,
+          e);
+    }
+  }
+
+  /**
+   * Get the span from a handler context.
+   * @param request request
+   * @param <T> type of request.
+   * @return the span callbacks or null
+   */
+  public static <T extends HandlerContextAware> AWSAuditEventCallbacks
+      retrieveAttachedSpan(final T request) {
+    return request.getHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT);
+  }
+
+  /**
+   * Attach a span to a handler context.
+   * @param request request
+   * @param span span to attach
+   * @param <T> type of request.
+   */
+  public static <T extends HandlerContextAware> void attachSpanToRequest(
+      final T request, final AWSAuditEventCallbacks span) {
+    request.addHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT, span);
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java
new file mode 100644
index 0000000..c1302d5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java
@@ -0,0 +1,92 @@
+/*
+ * 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.s3a.audit;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.amazonaws.handlers.RequestHandler2;
+import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
+import org.apache.hadoop.fs.store.audit.AuditSpanSource;
+import org.apache.hadoop.service.Service;
+
+/**
+ * Interface for Audit Managers auditing operations through the
+ * AWS libraries.
+ * The Audit Manager is the binding between S3AFS and the instantiated
+ * plugin point -it adds:
+ * <ol>
+ *   <li>per-thread tracking of audit spans </li>
+ *   <li>The wiring up to the AWS SDK</li>
+ *   <li>State change tracking for copy operations (does not address issue)</li>
+ * </ol>
+ */
+@InterfaceAudience.Private
+public interface AuditManagerS3A extends Service,
+    AuditSpanSource<AuditSpanS3A>,
+    AWSAuditEventCallbacks,
+    ActiveThreadSpanSource<AuditSpanS3A> {
+
+  /**
+   * Get the auditor; valid once initialized.
+   * @return the auditor.
+   */
+  OperationAuditor getAuditor();
+
+  /**
+   * Create the request handler(s) for this audit service.
+   * The list returned is mutable; new handlers may be added.
+   * @return list of handlers for the SDK.
+   * @throws IOException failure.
+   */
+  List<RequestHandler2> createRequestHandlers() throws IOException;
+
+  /**
+   * Return a transfer state change callback which
+   * fixes the active span context to be that in which
+   * the state change listener was created.
+   * This can be used to audit the creation of the multipart
+   * upload initiation request which the transfer manager
+   * makes when a file to be copied is split up.
+   * This must be invoked/used within the active span.
+   * @return a state change listener.
+   */
+  TransferStateChangeListener createStateChangeListener();
+
+  /**
+   * Check for permission to access a path.
+   * The path is fully qualified and the status is the
+   * status of the path.
+   * This is called from the {@code FileSystem.access()} command
+   * and is a soft permission check used by Hive.
+   * @param path path to check
+   * @param status status of the path.
+   * @param mode access mode.
+   * @return true if access is allowed.
+   * @throws IOException failure
+   */
+  boolean checkAccess(Path path, S3AFileStatus status, FsAction mode)
+      throws IOException;
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java
similarity index 63%
copy from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
copy to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java
index 54a8836..3e4e709 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/LogExactlyOnce.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditSpanS3A.java
@@ -16,27 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a.impl;
+package org.apache.hadoop.fs.s3a.audit;
 
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.Logger;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
 
 /**
- * Log exactly once, even across threads.
+ * An Audit Span with S3A integration, specifically
+ * callbacks from the AWS client SDK.
  */
-public class LogExactlyOnce {
-
-  private final AtomicBoolean logged = new AtomicBoolean(false);
-  private final Logger log;
-
-  public LogExactlyOnce(final Logger log) {
-    this.log = log;
-  }
-
-  public void warn(String format, Object...args) {
-    if (!logged.getAndSet(true)) {
-      log.warn(format, args);
-    }
-  }
+public interface AuditSpanS3A extends AuditSpan,
+    AWSAuditEventCallbacks {
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java
new file mode 100644
index 0000000..672bcdf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java
@@ -0,0 +1,73 @@
+/*
+ * 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.s3a.audit;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.store.audit.AuditSpanSource;
+import org.apache.hadoop.service.Service;
+
+/**
+ * Interfaces for audit services to implement.
+ */
+public interface OperationAuditor extends Service,
+    IOStatisticsSource, AuditSpanSource<AuditSpanS3A> {
+
+  /**
+   * Initialize.
+   * The base class will call {@link Service#init(Configuration)}.
+   * @param options options to initialize with.
+   */
+  void init(OperationAuditorOptions options);
+
+  /**
+   * Get the unbonded span to use after deactivating an active
+   * span.
+   * @return a span.
+   */
+  AuditSpanS3A getUnbondedSpan();
+
+  /**
+   * Check for permission to access a path.
+   * The path is fully qualified and the status is the
+   * status of the path.
+   * This is called from the {@code FileSystem.access()} command
+   * and is a soft permission check used by Hive.
+   * @param path path to check
+   * @param status status of the path.
+   * @param mode access mode.
+   * @return true if access is allowed.
+   * @throws IOException failure
+   */
+  default boolean checkAccess(Path path, S3AFileStatus status, FsAction mode)
+      throws IOException {
+    return true;
+  }
+
+  /**
+   * Get the Auditor ID.
+   * @return ID
+   */
+  String getAuditorId();
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java
new file mode 100644
index 0000000..40d1b23
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditorOptions.java
@@ -0,0 +1,74 @@
+/*
+ * 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.s3a.audit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+
+/**
+ * Options for the {@link OperationAuditor}.
+ * Done as a builder and passed in so
+ * that if it is extended, external auditors will still link.
+ */
+public final class OperationAuditorOptions {
+
+  private Configuration configuration;
+  private IOStatisticsStore ioStatisticsStore;
+
+
+  private OperationAuditorOptions() {
+  }
+
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public OperationAuditorOptions withConfiguration(final Configuration value) {
+    configuration = value;
+    return this;
+  }
+
+  public IOStatisticsStore getIoStatisticsStore() {
+    return ioStatisticsStore;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public OperationAuditorOptions withIoStatisticsStore(
+      final IOStatisticsStore value) {
+    ioStatisticsStore = value;
+    return this;
+  }
+
+  /**
+   * Create one.
+   * @return a new option instance
+   */
+  public static OperationAuditorOptions builder() {
+    return new OperationAuditorOptions();
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java
new file mode 100644
index 0000000..2b08893
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java
@@ -0,0 +1,104 @@
+/*
+ * 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.s3a.audit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.LimitedPrivate("S3A auditing extensions")
+@InterfaceStability.Unstable
+public final class S3AAuditConstants {
+
+  private S3AAuditConstants() {
+  }
+
+  /**
+   * What to look for in logs for ops outside any audit.
+   * {@value}.
+   */
+  public static final String UNAUDITED_OPERATION = "unaudited operation";
+
+  /**
+   * Name of class used for audit logs: {@value}.
+   */
+  public static final String AUDIT_SERVICE_CLASSNAME =
+      "fs.s3a.audit.service.classname";
+
+  /**
+   * Classname of the logging auditor: {@value}.
+   */
+  public static final String LOGGING_AUDIT_SERVICE =
+      "org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor";
+
+  /**
+   * Classname of the No-op auditor: {@value}.
+   */
+  public static final String NOOP_AUDIT_SERVICE =
+      "org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor";
+
+  /**
+   * List of extra AWS SDK request handlers: {@value}.
+   * These are added to the SDK request chain <i>after</i>
+   * any audit service.
+   */
+  public static final String AUDIT_REQUEST_HANDLERS =
+      "fs.s3a.audit.request.handlers";
+
+  /**
+   * Should operations outside spans be rejected?
+   * This is for testing coverage of the span code; if used
+   * in production there's a risk of unexpected failures.
+   * {@value}.
+   */
+  public static final String REJECT_OUT_OF_SPAN_OPERATIONS
+      = "fs.s3a.audit.reject.out.of.span.operations";
+
+  /**
+   * Should the logging auditor add the HTTP Referrer header?
+   * {@value}.
+   */
+  public static final String REFERRER_HEADER_ENABLED
+      = "fs.s3a.audit.referrer.enabled";
+
+  /**
+   * Should the logging auditor add the HTTP Referrer header?
+   * Default value: {@value}.
+   */
+  public static final boolean REFERRER_HEADER_ENABLED_DEFAULT
+      = true;
+
+  /**
+   * List of audit fields to strip from referrer headers.
+   * {@value}.
+   */
+  public static final String REFERRER_HEADER_FILTER
+      = "fs.s3a.audit.referrer.filter";
+
+  /**
+   * Span name used during initialization.
+   */
+  public static final String INITIALIZE_SPAN = "initialize";
+
+  /**
+   * Operation name for any operation outside of an explicit
+   * span.
+   */
+  public static final String OUTSIDE_SPAN =
+      "outside-span";
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java
new file mode 100644
index 0000000..ac29234
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java
@@ -0,0 +1,309 @@
+/*
+ * 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.s3a.audit;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Class to help parse AWS S3 Logs.
+ * see https://docs.aws.amazon.com/AmazonS3/latest/userguide/LogFormat.html
+ *
+ * Getting the regexp right is surprisingly hard; this class does it
+ * explicitly and names each group in the process.
+ * All group names are included in {@link #AWS_LOG_REGEXP_GROUPS} in the order
+ * within the log entries.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class S3LogParser {
+
+  private S3LogParser() {
+  }
+
+  /**
+   * Simple entry: anything up to a space.
+   * {@value}.
+   */
+  private static final String SIMPLE = "[^ ]*";
+
+  /**
+   * Date/Time. Everything within square braces.
+   * {@value}.
+   */
+  private static final String DATETIME = "\\[(.*?)\\]";
+
+  /**
+   * A natural number or "-".
+   * {@value}.
+   */
+  private static final String NUMBER = "(-|[0-9]*)";
+
+  /**
+   * A Quoted field or "-".
+   * {@value}.
+   */
+  private static final String QUOTED = "(-|\"[^\"]*\")";
+
+
+  /**
+   * An entry in the regexp.
+   * @param name name of the group
+   * @param pattern pattern to use in the regexp
+   * @return the pattern for the regexp
+   */
+  private static String e(String name, String pattern) {
+    return String.format("(?<%s>%s) ", name, pattern);
+  }
+
+  /**
+   * An entry in the regexp.
+   * @param name name of the group
+   * @param pattern pattern to use in the regexp
+   * @return the pattern for the regexp
+   */
+  private static String eNoTrailing(String name, String pattern) {
+    return String.format("(?<%s>%s)", name, pattern);
+  }
+
+  /**
+   * Simple entry using the {@link #SIMPLE} pattern.
+   * @param name name of the element (for code clarity only)
+   * @return the pattern for the regexp
+   */
+  private static String e(String name) {
+    return e(name, SIMPLE);
+  }
+
+  /**
+   * Quoted entry using the {@link #QUOTED} pattern.
+   * @param name name of the element (for code clarity only)
+   * @return the pattern for the regexp
+   */
+  private static String q(String name) {
+    return e(name, QUOTED);
+  }
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String OWNER_GROUP = "owner";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String BUCKET_GROUP = "bucket";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String TIMESTAMP_GROUP = "timestamp";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String REMOTEIP_GROUP = "remoteip";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String REQUESTER_GROUP = "requester";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String REQUESTID_GROUP = "requestid";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String VERB_GROUP = "verb";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String KEY_GROUP = "key";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String REQUESTURI_GROUP = "requesturi";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String HTTP_GROUP = "http";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String AWSERRORCODE_GROUP = "awserrorcode";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String BYTESSENT_GROUP = "bytessent";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String OBJECTSIZE_GROUP = "objectsize";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String TOTALTIME_GROUP = "totaltime";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String TURNAROUNDTIME_GROUP = "turnaroundtime";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String REFERRER_GROUP = "referrer";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String USERAGENT_GROUP = "useragent";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String VERSION_GROUP = "version";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String HOSTID_GROUP = "hostid";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String SIGV_GROUP = "sigv";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String CYPHER_GROUP = "cypher";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String AUTH_GROUP = "auth";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String ENDPOINT_GROUP = "endpoint";
+
+  /**
+   * Log group {@value}.
+   */
+  public static final String TLS_GROUP = "tls";
+
+  /**
+   * This is where anything at the tail of a log
+   * entry ends up; it is null unless/until the AWS
+   * logs are enhanced in future.
+   * Value {@value}.
+   */
+  public static final String TAIL_GROUP = "tail";
+
+  /**
+   * Construct the log entry pattern.
+   */
+  public static final String LOG_ENTRY_REGEXP = ""
+      + e(OWNER_GROUP)
+      + e(BUCKET_GROUP)
+      + e(TIMESTAMP_GROUP, DATETIME)
+      + e(REMOTEIP_GROUP)
+      + e(REQUESTER_GROUP)
+      + e(REQUESTID_GROUP)
+      + e(VERB_GROUP)
+      + e(KEY_GROUP)
+      + q(REQUESTURI_GROUP)
+      + e(HTTP_GROUP, NUMBER)
+      + e(AWSERRORCODE_GROUP)
+      + e(BYTESSENT_GROUP)
+      + e(OBJECTSIZE_GROUP)
+      + e(TOTALTIME_GROUP)
+      + e(TURNAROUNDTIME_GROUP)
+      + q(REFERRER_GROUP)
+      + q(USERAGENT_GROUP)
+      + e(VERSION_GROUP)
+      + e(HOSTID_GROUP)
+      + e(SIGV_GROUP)
+      + e(CYPHER_GROUP)
+      + e(AUTH_GROUP)
+      + e(ENDPOINT_GROUP)
+      + eNoTrailing(TLS_GROUP, SIMPLE)
+      + eNoTrailing(TAIL_GROUP, ".*") // anything which follows
+      + "$"; // end of line
+
+  /**
+   * Groups in order.
+   */
+  private static final String[] GROUPS = {
+      OWNER_GROUP,
+      BUCKET_GROUP,
+      TIMESTAMP_GROUP,
+      REMOTEIP_GROUP,
+      REQUESTER_GROUP,
+      REQUESTID_GROUP,
+      VERB_GROUP,
+      KEY_GROUP,
+      REQUESTURI_GROUP,
+      HTTP_GROUP,
+      AWSERRORCODE_GROUP,
+      BYTESSENT_GROUP,
+      OBJECTSIZE_GROUP,
+      TOTALTIME_GROUP,
+      TURNAROUNDTIME_GROUP,
+      REFERRER_GROUP,
+      USERAGENT_GROUP,
+      VERSION_GROUP,
+      HOSTID_GROUP,
+      SIGV_GROUP,
+      CYPHER_GROUP,
+      AUTH_GROUP,
+      ENDPOINT_GROUP,
+      TLS_GROUP,
+      TAIL_GROUP
+  };
+
+  /**
+   * Ordered list of regular expression group names.
+   */
+  public static final List<String> AWS_LOG_REGEXP_GROUPS =
+      Collections.unmodifiableList(Arrays.asList(GROUPS));
+
+  /**
+   * And the actual compiled pattern.
+   */
+  public static final Pattern LOG_ENTRY_PATTERN = Pattern.compile(
+      LOG_ENTRY_REGEXP);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java
new file mode 100644
index 0000000..0bafdf2
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogVerbs.java
@@ -0,0 +1,51 @@
+/*
+ * 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.s3a.audit;
+
+/**
+ * Various verbs in the S3 logs.
+ * https://stackoverflow.com/questions/42707878/amazon-s3-logs-operation-definition
+ */
+public final class S3LogVerbs {
+
+  private S3LogVerbs() {
+  }
+
+  public static final String DELETE = "REST.DELETE.OBJECT";
+  public static final String COPY = "REST.COPY.OBJECT";
+  public static final String DELETE_BULK = "REST.POST.MULTI_OBJECT_DELETE";
+  public static final String DELETE_BULK_ENTRY = "BATCH.DELETE.OBJECT";
+  public static final String GET = "REST.GET.OBJECT";
+  public static final String HEAD = "REST.HEAD.OBJECT";
+  public static final String GET_ACL = "REST.GET.ACL";
+  public static final String GET_LOGGING_STATUS = "REST.GET.LOGGING_STATUS";
+  public static final String LIST = "REST.GET.BUCKET";
+  public static final String MULTIPART_UPLOAD_START = "REST.POST.UPLOADS";
+  public static final String MULTIPART_UPLOAD_PART = "REST.PUT.PART";
+  public static final String MULTIPART_UPLOAD_COMPLETE = "REST.POST.UPLOAD";
+  public static final String MULTIPART_UPLOADS_LIST = "REST.GET.UPLOADS";
+  public static final String MULTIPART_UPLOAD_ABORT = "REST.DELETE.UPLOAD";
+  public static final String PUT = "REST.PUT.OBJECT";
+  public static final String REST_GET_POLICY_STATUS = "REST.GET.POLICY_STATUS";
+  public static final String REST_GET_PUBLIC_ACCESS_BLOCK =
+      "REST.GET.PUBLIC_ACCESS_BLOCK";
+  public static final String REST_GET_TAGGING = "REST.GET.TAGGING";
+  public static final String S3_EXPIRE_OBJECT = "S3.EXPIRE.OBJECT";
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java
new file mode 100644
index 0000000..510fbab
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractAuditSpanImpl.java
@@ -0,0 +1,99 @@
+/*
+ * 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.s3a.audit.impl;
+
+import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.util.Time;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Base class for the audit spans implementations..
+ */
+public abstract class AbstractAuditSpanImpl implements AuditSpanS3A {
+
+  /**
+   * Span ID.
+   */
+  private final String spanId;
+
+  /**
+   * Timestamp in UTC of span creation.
+   */
+  private final long timestamp;
+
+  private final String operationName;
+
+  /**
+   * Constructor.
+   * @param spanId span ID.
+   * @param operationName operation name
+   */
+  protected AbstractAuditSpanImpl(
+      final String spanId,
+      final String operationName) {
+    this(spanId, Time.now(), operationName);
+  }
+
+  /**
+   * Constructor.
+   * @param spanId span ID.
+   * @param timestamp timestamp in millis
+   * @param operationName operation name
+   */
+  protected AbstractAuditSpanImpl(
+      final String spanId,
+      final long timestamp,
+      final String operationName) {
+    this.spanId = requireNonNull(spanId);
+    this.timestamp = timestamp;
+    this.operationName = operationName;
+  }
+
+  @Override
+  public final String getSpanId() {
+    return spanId;
+  }
+
+  @Override
+  public String getOperationName() {
+    return operationName;
+  }
+
+  @Override
+  public final long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public AuditSpanS3A activate() {
+    return this;
+  }
+
+  /**
+   * Invoke {@link AuditSpan#deactivate()}.
+   * This is final: subclasses MUST override the
+   * {@code deactivate()} method.
+   */
+  @Override
+  public final void close() {
+    deactivate();
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java
new file mode 100644
index 0000000..97ee92a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java
@@ -0,0 +1,121 @@
+/*
+ * 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.s3a.audit.impl;
+
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
+import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.service.AbstractService;
+
+/**
+ * This is a long-lived service which is created in S3A FS initialize
+ * (make it fast!) which provides context for tracking operations made to S3.
+ * An IOStatisticsStore is passed in -in production this is expected to
+ * be the S3AFileSystem instrumentation, which will have the
+ * {@code AUDIT_SPAN_START} statistic configured for counting durations.
+ */
+public abstract class AbstractOperationAuditor extends AbstractService
+    implements OperationAuditor {
+
+  /**
+   * Base of IDs is a UUID.
+   */
+  public static final String BASE = UUID.randomUUID().toString();
+
+  /**
+   * Counter to create unique auditor IDs.
+   */
+  private static final AtomicLong SPAN_ID_COUNTER = new AtomicLong(1);
+
+  /**
+   * Destination for recording statistics, especially duration/count of
+   * operations.
+   * Set in {@link #init(OperationAuditorOptions)}.
+   */
+  private IOStatisticsStore iostatistics;
+
+  /**
+   * Options: set in {@link #init(OperationAuditorOptions)}.
+   */
+  private OperationAuditorOptions options;
+
+  /**
+   * Auditor ID as a UUID.
+   */
+  private final UUID auditorUUID = UUID.randomUUID();
+
+  /**
+   * ID of the auditor, which becomes that of the filesystem
+   * in request contexts.
+   */
+  private final String auditorID = auditorUUID.toString();
+
+  /**
+   * Construct.
+   * @param name name
+   *
+   */
+  protected AbstractOperationAuditor(final String name) {
+    super(name);
+  }
+
+  /**
+   * Sets the IOStats and then calls init().
+   * @param opts options to initialize with.
+   */
+  @Override
+  public void init(final OperationAuditorOptions opts) {
+    this.options = opts;
+    this.iostatistics = opts.getIoStatisticsStore();
+    init(opts.getConfiguration());
+  }
+
+  @Override
+  public String getAuditorId() {
+    return auditorID;
+  }
+
+  /**
+   * Get the IOStatistics Store.
+   * @return the IOStatistics store updated with statistics.
+   */
+  public IOStatisticsStore getIOStatistics() {
+    return iostatistics;
+  }
+
+  /**
+   * Get the options this auditor was initialized with.
+   * @return options.
+   */
+  protected OperationAuditorOptions getOptions() {
+    return options;
+  }
+
+  /**
+   * Create a span ID.
+   * @return a unique span ID.
+   */
+  protected final String createSpanID() {
+    return String.format("%s-%08d",
+        auditorID, SPAN_ID_COUNTER.incrementAndGet());
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java
new file mode 100644
index 0000000..e7e775d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java
@@ -0,0 +1,773 @@
+/*
+ * 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.s3a.audit.impl;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.HandlerContextAware;
+import com.amazonaws.Request;
+import com.amazonaws.Response;
+import com.amazonaws.SdkBaseException;
+import com.amazonaws.handlers.HandlerAfterAttemptContext;
+import com.amazonaws.handlers.HandlerBeforeAttemptContext;
+import com.amazonaws.handlers.RequestHandler2;
+import com.amazonaws.http.HttpResponse;
+import com.amazonaws.services.s3.transfer.Transfer;
+import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks;
+import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer;
+import org.apache.hadoop.fs.s3a.audit.AuditFailureException;
+import org.apache.hadoop.fs.s3a.audit.AuditIntegration;
+import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A;
+import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
+import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
+import org.apache.hadoop.fs.s3a.audit.S3AAuditConstants;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.functional.FutureIO;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE;
+import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION;
+import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.attachSpanToRequest;
+import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.retrieveAttachedSpan;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS;
+
+/**
+ * Thread management for the active audit.
+ * This should be created by whatever wants to have active
+ * audit span tracking.
+ *
+ * It creates and starts the actual
+ * {@link OperationAuditor} for auditing.
+ * It then stores the thread-local span and returns a wrapping
+ * span.
+ *
+ * When the wrapper is closed/deactivated it
+ * will deactivate the wrapped span and then
+ * switch the active span to the unbounded span.
+ *
+ * The inner class {@link AWSAuditEventCallbacks} is returned
+ * as a request handler in {@link #createRequestHandlers()};
+ * this forwards all requests to the outer {@code ActiveAuditManagerS3A},
+ * which then locates the active span and forwards the request.
+ * If any such invocation raises an {@link AuditFailureException}
+ * then the IOStatistics counter for {@code AUDIT_FAILURE}
+ * is incremented.
+ *
+ */
+@InterfaceAudience.Private
+public final class ActiveAuditManagerS3A
+    extends CompositeService
+    implements AuditManagerS3A {
+
+  /**
+   * Logging.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ActiveAuditManagerS3A.class);
+
+  /**
+   * One of logger for warnings about span retrieval.
+   */
+  public static final LogExactlyOnce WARN_OF_SPAN_TYPE =
+      new LogExactlyOnce(LOG);
+
+  public static final String AUDIT_MANAGER_OPERATION = "AuditManagerS3A";
+
+  public static final String NOT_A_WRAPPED_SPAN
+      = "Span attached to request is not a wrapped span";
+
+  /**
+   * Audit service.
+   */
+  private OperationAuditor auditor;
+
+  /**
+   * Some basic analysis for the logs.
+   */
+  private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer();
+
+  /**
+   * This is the span returned to after a wrapper is closed or
+   * the span is reset to the unbonded span..
+   */
+  private WrappingAuditSpan unbondedSpan;
+
+  /**
+   * Thread local span. This defaults to being
+   * the unbonded span.
+   */
+  private final ThreadLocal<WrappingAuditSpan> activeSpan =
+      ThreadLocal.withInitial(() -> getUnbondedSpan());
+
+  /**
+   * Destination for recording statistics, especially duration/count of
+   * operations.
+   */
+  private final IOStatisticsStore ioStatisticsStore;
+
+  /**
+   * Instantiate.
+   * @param iostatistics statistics target
+   */
+  public ActiveAuditManagerS3A(final IOStatisticsStore iostatistics) {
+    super("ActiveAuditManagerS3A");
+    this.ioStatisticsStore = iostatistics;
+  }
+
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    // create and register the service so it follows the same lifecycle
+    OperationAuditorOptions options =
+        OperationAuditorOptions.builder()
+            .withConfiguration(conf)
+            .withIoStatisticsStore(ioStatisticsStore);
+    auditor = AuditIntegration.createAndInitAuditor(
+        getConfig(),
+        S3AAuditConstants.AUDIT_SERVICE_CLASSNAME,
+        options);
+    addService(auditor);
+    LOG.debug("Audit manager initialized with audit service {}", auditor);
+  }
+
+  /**
+   * After starting the auditor, it is queried for its
+   * unbonded span, which is then wrapped and stored for
+   * use.
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    setUnbondedSpan(new WrappingAuditSpan(
+        auditor.getUnbondedSpan(), false));
+    LOG.debug("Started audit service {}", auditor);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(super.toString());
+    sb.append(", auditor=").append(auditor);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public OperationAuditor getAuditor() {
+    return auditor;
+  }
+
+  /**
+   * Get the unbounded span. Until this manager
+   * is fully initialized it will return the no-op
+   * span.
+   * @return the unbounded span.
+   */
+  private WrappingAuditSpan getUnbondedSpan() {
+    return unbondedSpan;
+  }
+
+  /**
+   * Set the unbonded span.
+   * @param unbondedSpan the new unbonded span
+   */
+  private void setUnbondedSpan(final WrappingAuditSpan unbondedSpan) {
+    this.unbondedSpan = unbondedSpan;
+  }
+
+  /**
+   * Return the active wrapped span.
+   * @return a span.
+   */
+  @Override
+  public AuditSpanS3A getActiveAuditSpan() {
+    return activeSpan();
+  }
+
+  /**
+   * Get the active span.
+   * This is the wrapped span, not the inner one, and it is
+   * of that type.
+   * @return the active WrappingAuditSpan
+   */
+  private WrappingAuditSpan activeSpan() {
+    return activeSpan.get();
+  }
+
+  /**
+   * Set a specific span as the active span.
+   * This will wrap it.
+   * @param span span to use.
+   * @return the wrapped span.
+   */
+  private AuditSpanS3A setActiveThreadSpan(AuditSpanS3A span) {
+    return switchToActiveSpan(
+        new WrappingAuditSpan(span, span.isValidSpan()));
+  }
+
+  /**
+   * Switch to a given span. If it is null, use the
+   * unbounded span.
+   * @param span to switch to; may be null
+   * @return the span switched to
+   */
+  private WrappingAuditSpan switchToActiveSpan(WrappingAuditSpan span) {
+    if (span != null && span.isValidSpan()) {
+      activeSpan.set(span);
+    } else {
+      activeSpan.set(unbondedSpan);
+    }
+    return activeSpan();
+  }
+
+  /**
+   * The Span ID in the audit manager is the ID of the auditor,
+   * which can be used in the filesystem toString() method
+   * to assist in correlating client logs with S3 logs.
+   * It is returned here as part of the implementation of
+   * {@link AWSAuditEventCallbacks}.
+   * @return the unique ID of the FS.
+   */
+  @Override
+  public String getSpanId() {
+    return auditor != null
+        ? auditor.getAuditorId()
+        : "(auditor not yet created)";
+  }
+
+  @Override
+  public String getOperationName() {
+    return AUDIT_MANAGER_OPERATION;
+  }
+
+  /**
+   * Start an operation; as well as invoking the audit
+   * service to do this, sets the operation as the
+   * active operation for this thread.
+   * @param operation operation name.
+   * @param path1 first path of operation
+   * @param path2 second path of operation
+   * @return a wrapped audit span
+   * @throws IOException failure
+   */
+  @Override
+  public AuditSpanS3A createSpan(final String operation,
+      @Nullable final String path1,
+      @Nullable final String path2) throws IOException {
+    // must be started
+    Preconditions.checkState(isInState(STATE.STARTED),
+        "Audit Manager %s is in wrong state: %s",
+        this, getServiceState());
+    ioStatisticsStore.incrementCounter(
+        Statistic.AUDIT_SPAN_CREATION.getSymbol());
+    return setActiveThreadSpan(auditor.createSpan(
+        operation, path1, path2));
+  }
+
+  /**
+   * Return a request handler for the AWS SDK which
+   * relays to this class.
+   * @return a request handler.
+   */
+  @Override
+  public List<RequestHandler2> createRequestHandlers()
+      throws IOException {
+
+    // wire up the AWS SDK To call back into this class when
+    // preparing to make S3 calls.
+    List<RequestHandler2> requestHandlers = new ArrayList<>();
+    requestHandlers.add(new SdkRequestHandler());
+    // now look for any more handlers
+    final Class<?>[] handlers = getConfig().getClasses(AUDIT_REQUEST_HANDLERS);
+    if (handlers != null) {
+      for (Class<?> handler : handlers) {
+        try {
+          Constructor<?> ctor = handler.getConstructor();
+          requestHandlers.add((RequestHandler2)ctor.newInstance());
+        } catch (ExceptionInInitializerError e) {
+          throw FutureIO.unwrapInnerException(e);
+        } catch (Exception e) {
+          throw new IOException(e);
+        }
+      }
+    }
+    return requestHandlers;
+  }
+
+  @Override
+  public TransferStateChangeListener createStateChangeListener() {
+    final WrappingAuditSpan span = activeSpan();
+    return new TransferStateChangeListener() {
+      @Override
+      public void transferStateChanged(final Transfer transfer,
+          final Transfer.TransferState state) {
+        switchToActiveSpan(span);
+      }
+    };
+  }
+
+  @Override
+  public boolean checkAccess(final Path path,
+      final S3AFileStatus status,
+      final FsAction mode)
+      throws IOException {
+    return auditor.checkAccess(path, status, mode);
+  }
+
+  /**
+   * Attach a reference to the active thread span, then
+   * invoke the same callback on that active thread.
+   */
+  @Override
+  public <T extends AmazonWebServiceRequest> T requestCreated(
+      final T request) {
+    AuditSpanS3A span = getActiveAuditSpan();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Created Request {} in span {}",
+          analyzer.analyze(request), span);
+    }
+    attachSpanToRequest(request, span);
+    try {
+      return span.requestCreated(request);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  /**
+   * Forward to the active span.
+   * All invocations increment the statistics counter for
+   * {@link Statistic#AUDIT_REQUEST_EXECUTION};
+   * failures will also increment
+   * {@link Statistic#AUDIT_FAILURE};
+   * {@inheritDoc}
+   */
+  @Override
+  public <T extends AmazonWebServiceRequest> T beforeExecution(
+      final T request) {
+    ioStatisticsStore.incrementCounter(AUDIT_REQUEST_EXECUTION.getSymbol());
+
+    // identify the span and invoke the callback
+    try {
+      return extractAndActivateSpanFromRequest(request)
+          .beforeExecution(request);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  /**
+   * Forward to active span.
+   * @param request request
+   * @param response response.
+   */
+  @Override
+  public void afterResponse(final Request<?> request,
+      final Response<?> response)
+      throws AuditFailureException, SdkBaseException {
+    try {
+      extractAndActivateSpanFromRequest(request)
+          .afterResponse(request, response);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  /**
+   * Get the active span from the handler context,
+   * falling back to the active thread span if there
+   * is nothing in the context.
+   * Provided the span is a wrapped span, the
+   * @param request request
+   * @param <T> type of request.
+   * @return the callbacks
+   */
+  private <T extends HandlerContextAware> AWSAuditEventCallbacks
+      extractAndActivateSpanFromRequest(final T request) {
+    AWSAuditEventCallbacks span;
+    span = retrieveAttachedSpan(request);
+    if (span == null) {
+      // no span is attached. Not unusual for the copy operations,
+      // or for calls to GetBucketLocation made by the AWS client
+      LOG.debug("No audit span attached to request {}",
+          request);
+      // fall back to the active thread span.
+      // this will be the unbonded span if the thread is unbonded.
+      span = getActiveAuditSpan();
+    } else {
+      if (span instanceof WrappingAuditSpan) {
+        switchToActiveSpan((WrappingAuditSpan) span);
+      } else {
+        // warn/log and continue without switching.
+        WARN_OF_SPAN_TYPE.warn(NOT_A_WRAPPED_SPAN + ": {}", span);
+        LOG.debug(NOT_A_WRAPPED_SPAN + ": {}", span);
+      }
+    }
+    return span;
+  }
+
+  /**
+   * Forward to active span.
+   * @param request request
+   * @param response response.
+   * @param exception exception raised.
+   */
+  @Override
+  public void afterError(final Request<?> request,
+      final Response<?> response,
+      final Exception exception)
+      throws AuditFailureException, SdkBaseException {
+    try {
+      extractAndActivateSpanFromRequest(request)
+          .afterError(request, response, exception);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  @Override
+  public AmazonWebServiceRequest beforeMarshalling(
+      final AmazonWebServiceRequest request) {
+    try {
+      return extractAndActivateSpanFromRequest(request)
+          .beforeMarshalling(request);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  @Override
+  public void beforeRequest(final Request<?> request) {
+    try {
+      extractAndActivateSpanFromRequest(request)
+          .beforeRequest(request);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  @Override
+  public void beforeAttempt(final HandlerBeforeAttemptContext context) {
+    try {
+      extractAndActivateSpanFromRequest(context.getRequest())
+          .beforeAttempt(context);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  @Override
+  public void afterAttempt(final HandlerAfterAttemptContext context) {
+    try {
+      extractAndActivateSpanFromRequest(context.getRequest())
+          .afterAttempt(context);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+  }
+
+  @Override
+  public HttpResponse beforeUnmarshalling(final Request<?> request,
+      final HttpResponse httpResponse) {
+    try {
+      extractAndActivateSpanFromRequest(request.getOriginalRequest())
+          .beforeUnmarshalling(request, httpResponse);
+    } catch (AuditFailureException e) {
+      ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
+      throw e;
+    }
+    return httpResponse;
+  }
+
+  /**
+   * Callbacks from the AWS SDK; all forward to the ActiveAuditManagerS3A.
+   * We need a separate class because the SDK requires the handler list
+   * to be list of {@code RequestHandler2} instances.
+   */
+  private class SdkRequestHandler extends RequestHandler2 {
+
+    @Override
+    public AmazonWebServiceRequest beforeExecution(
+        final AmazonWebServiceRequest request) {
+      return ActiveAuditManagerS3A.this.beforeExecution(request);
+    }
+
+    @Override
+    public void afterResponse(final Request<?> request,
+        final Response<?> response) {
+      ActiveAuditManagerS3A.this.afterResponse(request, response);
+    }
+
+    @Override
+    public void afterError(final Request<?> request,
+        final Response<?> response,
+        final Exception e) {
+      ActiveAuditManagerS3A.this.afterError(request, response, e);
+    }
+
+    @Override
+    public AmazonWebServiceRequest beforeMarshalling(
+        final AmazonWebServiceRequest request) {
+      return ActiveAuditManagerS3A.this.beforeMarshalling(request);
+    }
+
+    @Override
+    public void beforeRequest(final Request<?> request) {
+      ActiveAuditManagerS3A.this.beforeRequest(request);
+    }
+
+    @Override
+    public void beforeAttempt(
+        final HandlerBeforeAttemptContext context) {
+      ActiveAuditManagerS3A.this.beforeAttempt(context);
+    }
+
+    @Override
+    public HttpResponse beforeUnmarshalling(
+        final Request<?> request,
+        final HttpResponse httpResponse) {
+      return ActiveAuditManagerS3A.this.beforeUnmarshalling(request,
+          httpResponse);
+    }
+
+    @Override
+    public void afterAttempt(
+        final HandlerAfterAttemptContext context) {
+      ActiveAuditManagerS3A.this.afterAttempt(context);
+    }
+  }
+
+  /**
+   * Wraps the plugged in spans with management of the active thread
+   * span, including switching to the unbounded span when a valid
+   * span is deactivated.
+   * Package-private for testing.
+   */
+  private final class WrappingAuditSpan extends AbstractAuditSpanImpl {
+
+    /**
+     * Inner span.
+     */
+    private final AuditSpanS3A span;
+
+    /**
+     * Is this span considered valid?
+     */
+    private final boolean isValid;
+
+    /**
+     * Create, wrapped.
+     * The spanID, name, timestamp etc copied from the span being wrapped.
+     * Why not the isValid state? We want to set our unbonded span without
+     * relying on the auditor doing the right thing.
+     * @param span inner span.
+     * @param isValid is the span valid
+     */
+    private WrappingAuditSpan(
+        final AuditSpanS3A span, final boolean isValid) {
+      super(span.getSpanId(), span.getTimestamp(), span.getOperationName());
+      this.span = requireNonNull(span);
+      this.isValid = isValid;
+    }
+
+    /**
+     * Is the span active?
+     * @return true if this span is the active one for the current thread.
+     */
+    private boolean isActive() {
+      return this == getActiveAuditSpan();
+    }
+
+    /**
+     * Makes this the thread's active span and activate.
+     * If the span was already active: no-op.
+     */
+    @Override
+    public AuditSpanS3A activate() {
+      if (!isActive()) {
+        switchToActiveSpan(this);
+        span.activate();
+      }
+      return this;
+    }
+
+    /**
+     * Switch to the unbounded span and then deactivate this span.
+     * No-op for invalid spans,
+     * so as to prevent the unbounded span from being closed
+     * and everything getting very confused.
+     */
+    @Override
+    public void deactivate() {
+      // no-op for invalid spans,
+      // so as to prevent the unbounded span from being closed
+      // and everything getting very confused.
+      if (!isValid || !isActive()) {
+        return;
+      }
+      // deactivate the span
+      span.deactivate();
+      // and go to the unbounded one.
+      switchToActiveSpan(getUnbondedSpan());
+    }
+
+    /**
+     * Forward to the wrapped span.
+     * {@inheritDoc}
+     */
+    @Override
+    public <T extends AmazonWebServiceRequest> T requestCreated(
+        final T request) {
+      return span.requestCreated(request);
+    }
+
+    /**
+     * This span is valid if the span isn't closed and the inner
+     * span is valid.
+     * @return true if the span is considered valid.
+     */
+    @Override
+    public boolean isValidSpan() {
+      return isValid && span.isValidSpan();
+    }
+
+    /**
+     * Forward to the inner span.
+     * {@inheritDoc}
+     */
+    @Override
+    public void set(final String key, final String value) {
+      span.set(key, value);
+    }
+
+    /**
+     * Forward to the inner span.
+     * @param request request
+     * @param <T> type of request
+     * @return an updated request.
+     */
+    @Override
+    public <T extends AmazonWebServiceRequest> T beforeExecution(
+        final T request) {
+      return span.beforeExecution(request);
+    }
+
+    /**
+     * Forward to the inner span.
+     * @param request request
+     * @param response response.
+     */
+    @Override
+    public void afterResponse(final Request<?> request,
+        final Response<?> response) {
+      span.afterResponse(request, response);
+    }
+
+    /**
+     * Forward to the inner span.
+     * @param request request
+     * @param response response.
+     * @param exception exception raised.
+     */
+    @Override
+    public void afterError(final Request<?> request,
+        final Response<?> response,
+        final Exception exception) {
+      span.afterError(request, response, exception);
+    }
+
+    /**
+     * Forward to the inner span.
+     * @param request request
+     * @return request to marshall
+     */
+    @Override
+    public AmazonWebServiceRequest beforeMarshalling(
+        final AmazonWebServiceRequest request) {
+      return span.beforeMarshalling(request);
+    }
+
+    /**
+     * Forward to the inner span.
+     * @param request request
+     */
+    @Override
+    public void beforeRequest(final Request<?> request) {
+      span.beforeRequest(request);
+    }
+
+    /**
+     * Forward to the inner span.
+     * @param context full context, including the request.
+     */
+    @Override
+    public void beforeAttempt(
+        final HandlerBeforeAttemptContext context) {
+      span.beforeAttempt(context);
+    }
+
+    /**
+     * Forward to the inner span.
+     *
+     * @param context full context, including the request.
+     */
+    @Override
+    public void afterAttempt(
+        final HandlerAfterAttemptContext context) {
+      span.afterAttempt(context);
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "WrappingAuditSpan{");
+      sb.append("span=").append(span);
+      sb.append(", valid=").append(isValidSpan());
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
new file mode 100644
index 0000000..1722df1
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
@@ -0,0 +1,435 @@
+/*
+ * 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.s3a.audit.impl;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.audit.AuditConstants;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
+import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer;
+import org.apache.hadoop.fs.s3a.audit.AuditFailureException;
+import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
+import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_FILESYSTEM_ID;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PRINCIPAL;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext;
+import static org.apache.hadoop.fs.audit.CommonAuditContext.currentThreadID;
+import static org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer.isRequestNotAlwaysInSpan;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.OUTSIDE_SPAN;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED_DEFAULT;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
+import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION;
+import static org.apache.hadoop.fs.s3a.commit.CommitUtils.extractJobID;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER;
+
+/**
+ * The LoggingAuditor logs operations at DEBUG (in SDK Request) and
+ * in span lifecycle and S3 request class construction at TRACE.
+ * The context information is added as the HTTP referrer.
+ */
+@InterfaceAudience.Private
+public class LoggingAuditor
+    extends AbstractOperationAuditor {
+
+  /**
+   * This is where the context gets logged to.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(LoggingAuditor.class);
+
+
+  /**
+   * Some basic analysis for the logs.
+   */
+  private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer();
+
+  /**
+   * Default span to use when there is no other.
+   */
+  private AuditSpanS3A warningSpan;
+
+  /**
+   * Should out of scope ops be rejected?
+   */
+  private boolean rejectOutOfSpan;
+
+  /**
+   * Map of attributes which will be added to all operations.
+   */
+  private final Map<String, String> attributes = new HashMap<>();
+
+  /**
+   * Should the referrer header be added?
+   */
+  private boolean headerEnabled;
+
+  /**
+   * This is the header sent by the last S3 operation through
+   * this auditor.
+   * <p>
+   * It is for testing -allows for Integration tests to
+   * verify that a header was sent and query what was in it.
+   * Initially an empty string.
+   */
+  private volatile String lastHeader = "";
+
+  /**
+   * Attributes to filter.
+   */
+  private Collection<String> filters;
+
+  /**
+   * Create the auditor.
+   * The UGI current user is used to provide the principal;
+   * this will be cached and provided in the referrer header.
+   */
+  public LoggingAuditor() {
+
+    super("LoggingAuditor ");
+    attributes.put(PARAM_FILESYSTEM_ID, getAuditorId());
+
+
+    // add the principal
+    try {
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      addAttribute(PARAM_PRINCIPAL, ugi.getUserName());
+    } catch (IOException ex) {
+      LOG.warn("Auditor unable to determine principal", ex);
+    }
+  }
+
+  /**
+   * Service init, look for jobID and attach as an attribute in log entries.
+   * This is where the warning span is created, so the relevant attributes
+   * (and filtering options) are applied.
+   * @param conf configuration
+   * @throws Exception failure
+   */
+  @Override
+  protected void serviceInit(final Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    rejectOutOfSpan = conf.getBoolean(
+        REJECT_OUT_OF_SPAN_OPERATIONS, false);
+    // attach the job ID if there is one in the configuration used
+    // to create this file.
+    String jobID = extractJobID(conf);
+    if (jobID != null) {
+      addAttribute(AuditConstants.PARAM_JOB_ID, jobID);
+    }
+    headerEnabled = getConfig().getBoolean(REFERRER_HEADER_ENABLED,
+        REFERRER_HEADER_ENABLED_DEFAULT);
+    filters = conf.getTrimmedStringCollection(REFERRER_HEADER_FILTER);
+    final CommonAuditContext currentContext = currentAuditContext();
+    warningSpan = new WarningSpan(OUTSIDE_SPAN,
+        currentContext, createSpanID(), null, null);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "LoggingAuditor{");
+    sb.append("ID='").append(getAuditorId()).append('\'');
+    sb.append(", headerEnabled=").append(headerEnabled);
+    sb.append(", rejectOutOfSpan=").append(rejectOutOfSpan);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public AuditSpanS3A createSpan(final String operation,
+      @Nullable final String path1,
+      @Nullable final String path2) {
+    LoggingAuditSpan span = new LoggingAuditSpan(
+        createSpanID(),
+        operation,
+        prepareActiveContext(),
+        path1,
+        path2);
+    span.start();
+    return span;
+  }
+
+  /**
+   * Get/Prepare the active context for a span.
+   * @return the common audit context.
+   */
+  private CommonAuditContext prepareActiveContext() {
+    return currentAuditContext();
+  }
+
+  /**
+   * Add an attribute.
+   * @param key key
+   * @param value value
+   */
+  public final void addAttribute(String key, String value) {
+    attributes.put(key, value);
+  }
+
+  @Override
+  public AuditSpanS3A getUnbondedSpan() {
+    return warningSpan;
+  }
+
+  /**
+   * Get the last header used.
+   * @return the last referrer header generated.
+   */
+  public String getLastHeader() {
+    return lastHeader;
+  }
+
+  /**
+   * Set that last header.
+   * @param lastHeader the value for the lastHeader field.
+   */
+  private void setLastHeader(final String lastHeader) {
+    this.lastHeader = lastHeader;
+  }
+
+  /**
+   * Span which logs at debug and sets the HTTP referrer on
+   * invocations.
+   * Note: checkstyle complains that this should be final because
+   * it is private. This is not true, as it is subclassed in
+   * the same file.
+   */
+  private class LoggingAuditSpan extends AbstractAuditSpanImpl {
+
+    private final HttpReferrerAuditHeader referrer;
+
+    private final String description;
+
+    private LoggingAuditSpan(
+        final String spanId,
+        final String operationName,
+        final CommonAuditContext context,
+        final String path1,
+        final String path2) {
+      super(spanId, operationName);
+
+      this.referrer = HttpReferrerAuditHeader.builder()
+          .withContextId(getAuditorId())
+          .withSpanId(spanId)
+          .withOperationName(operationName)
+          .withPath1(path1)
+          .withPath2(path2)
+          .withAttributes(attributes)
+          // thread at the time of creation.
+          .withAttribute(PARAM_THREAD0,
+              currentThreadID())
+          .withAttribute(PARAM_TIMESTAMP, Long.toString(getTimestamp()))
+          .withEvaluated(context.getEvaluatedEntries())
+          .withFilter(filters)
+          .build();
+
+      this.description = referrer.buildHttpReferrer();
+    }
+
+    public void start() {
+      LOG.trace("{} Start {}", getSpanId(), getDescription());
+    }
+
+    /**
+     * Get the span description built in the constructor.
+     * @return description text.
+     */
+    protected String getDescription() {
+      return description;
+    }
+
+    /**
+     * Activate: log at TRACE.
+     * @return this span.
+     */
+    @Override
+    public AuditSpanS3A activate() {
+      LOG.trace("[{}] {} Activate {}",
+          currentThreadID(), getSpanId(), getDescription());
+      return this;
+    }
+
+    /**
+     * Log at TRACE.
+     */
+    @Override
+    public void deactivate() {
+      LOG.trace("[{}] {} Deactivate {}",
+          currentThreadID(), getSpanId(), getDescription());
+    }
+
+
+    /**
+     * Pass to the HTTP referrer.
+     * {@inheritDoc}
+     */
+    @Override
+    public void set(final String key, final String value) {
+      referrer.set(key, value);
+    }
+
+    /**
+     * Before execution, the logging auditor always builds
+     * the referrer header, saves to the outer class
+     * (where {@link #getLastHeader()} can retrieve it,
+     * and logs at debug.
+     * If configured to add the header to the S3 logs, it will
+     * be set as the HTTP referrer.
+     * @param request request
+     * @param <T> type of request.
+     * @return the request with any extra headers.
+     */
+    @Override
+    public <T extends AmazonWebServiceRequest> T beforeExecution(
+        final T request) {
+      // build the referrer header
+      final String header = referrer.buildHttpReferrer();
+      // update the outer class's field.
+      setLastHeader(header);
+      if (headerEnabled) {
+        // add the referrer header
+        request.putCustomRequestHeader(HEADER_REFERRER,
+            header);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("[{}] {} Executing {} with {}; {}",
+            currentThreadID(),
+            getSpanId(),
+            getOperationName(),
+            analyzer.analyze(request),
+            header);
+      }
+      return request;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "LoggingAuditSpan{");
+      sb.append(", id='").append(getSpanId()).append('\'');
+      sb.append("description='").append(description).append('\'');
+      sb.append('}');
+      return sb.toString();
+    }
+
+    /**
+     * Get the referrer; visible for tests.
+     * @return the referrer.
+     */
+    HttpReferrerAuditHeader getReferrer() {
+      return referrer;
+    }
+  }
+
+  /**
+   * Span which logs at WARN; used to highlight spans
+   * without a containing span.
+   */
+  private final class WarningSpan extends LoggingAuditSpan {
+
+    private WarningSpan(
+        final String name,
+        final CommonAuditContext context,
+        final String spanId,
+        final String path1, final String path2) {
+      super(spanId, name, context, path1, path2);
+    }
+
+    @Override
+    public void start() {
+      LOG.warn("[{}] {} Start {}",
+          currentThreadID(), getSpanId(), getDescription());
+    }
+
+    @Override
+    public AuditSpanS3A activate() {
+      LOG.warn("[{}] {} Activate {}",
+          currentThreadID(), getSpanId(), getDescription());
+      return this;
+    }
+
+    @Override
+    public boolean isValidSpan() {
+      return false;
+    }
+
+    @Override
+    public <T extends AmazonWebServiceRequest> T requestCreated(
+        final T request) {
+      String error = "Creating a request outside an audit span "
+          + analyzer.analyze(request);
+      LOG.info(error);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(error, new AuditFailureException("unaudited"));
+      }
+      return request;
+    }
+
+    /**
+     * Handle requests made without a real context by logging and
+     * increment the failure count.
+     * Some requests (e.g. copy part) are not expected in spans due
+     * to how they are executed; these do not trigger failures.
+     * @param request request
+     * @param <T> type of request
+     * @return an updated request.
+     * @throws AuditFailureException if failure is enabled.
+     */
+    @Override
+    public <T extends AmazonWebServiceRequest> T beforeExecution(
+        final T request) {
+
+
+      String error = "executing a request outside an audit span "
+          + analyzer.analyze(request);
+      LOG.warn("{} {}",
+          getSpanId(), error);
+      final String unaudited = getSpanId() + " "
+          + UNAUDITED_OPERATION + " " + error;
+      if (isRequestNotAlwaysInSpan(request)) {
+        // can get by auditing during a copy, so don't overreact
+        LOG.debug(unaudited);
+      } else {
+        final RuntimeException ex = new AuditFailureException(unaudited);
+        LOG.debug(unaudited, ex);
+        if (rejectOutOfSpan) {
+          throw ex;
+        }
+      }
+      // now hand off to the superclass for its normal preparation
+      return super.beforeExecution(request);
+    }
+  }
... 7924 lines suppressed ...

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org