mehakmeet commented on a change in pull request #2807:
URL: https://github.com/apache/hadoop/pull/2807#discussion_r636762939



##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.handlers.RequestHandler2;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+
+import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Unit tests related to span lifecycle.
+ */
+public class TestAuditSpanLifecycle extends AbstractAuditingTest {
+
+  private Configuration conf;
+
+  private AuditSpan resetSpan;
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    resetSpan = getManager().getActiveAuditSpan();
+  }
+
+  protected Configuration createConfig() {
+    return noopAuditConfig();
+  }
+
+  /**
+   * Core lifecycle (remember: the service has already been started).
+   */
+  @Test
+  public void testStop() throws Throwable {
+    getManager().stop();
+  }
+
+  @Test
+  public void testCreateRequestHandlers() throws Throwable {
+    List<RequestHandler2> handlers
+        = getManager().createRequestHandlers();
+    assertThat(handlers).isNotEmpty();
+  }
+
+  @Test
+  public void testInitialSpanIsInvalid() throws Throwable {
+    assertThat(resetSpan)
+        .matches(f -> !f.isValidSpan(), "is invalid");
+  }
+
+  @Test
+  public void testCreateCloseSpan() throws Throwable {
+    AuditSpan span = getManager().createSpan("op", null, null);
+    assertThat(span)
+        .matches(AuditSpan::isValidSpan, "is valid");
+    assertActiveSpan(span);
+    // activation when already active is no-op
+    span.activate();
+    assertActiveSpan(span);
+    // close the span
+    span.close();
+    // the original span is restored.
+    assertActiveSpan(resetSpan);
+  }
+
+  @Test
+  public void testSpanActivation() throws Throwable {
+    // real activation switches spans in the current thead.

Review comment:
       typo: "thread"

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.handlers.RequestHandler2;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+
+import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Unit tests related to span lifecycle.
+ */
+public class TestAuditSpanLifecycle extends AbstractAuditingTest {
+
+  private Configuration conf;

Review comment:
       Never used.

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.Map;
+
+import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import org.junit.After;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.api.RequestFactory;
+import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl;
+import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_GET_FILE_STATUS;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION;
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.createIOStatisticsStoreForAuditing;
+import static org.apache.hadoop.service.ServiceOperations.stopQuietly;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Abstract class for auditor unit tests.
+ */
+public abstract class AbstractAuditingTest extends AbstractHadoopTestBase {
+
+  protected static final String OPERATION
+      = INVOCATION_GET_FILE_STATUS.getSymbol();
+
+  /**
+   * Logging.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractAuditingTest.class);
+
+  public static final String PATH_1 = "/path1";
+
+  public static final String PATH_2 = "/path2";
+
+  /**
+   * Statistics store with the auditor counters wired up.
+   */
+  private final IOStatisticsStore ioStatistics =
+      createIOStatisticsStoreForAuditing();
+
+  private RequestFactory requestFactory;
+
+  private AuditManagerS3A manager;
+
+  @Before
+  public void setup() throws Exception {
+    requestFactory = RequestFactoryImpl.builder()
+        .withBucket("bucket")
+        .build();
+    manager = AuditIntegration.createAndStartAuditManager(
+        createConfig(),
+        ioStatistics);
+  }
+
+  /**
+   * Create config.
+   * @return config to use when creating a manager
+   */
+  protected abstract Configuration createConfig();
+
+  @After
+  public void teardown() {
+    stopQuietly(manager);
+  }
+
+  protected IOStatisticsStore getIOStatistics() {
+    return ioStatistics;
+  }
+
+  protected RequestFactory getRequestFactory() {
+    return requestFactory;
+  }
+
+  protected AuditManagerS3A getManager() {
+    return manager;
+  }
+
+  /**
+   * Assert that a specific span is active.
+   * This matches on the wrapped spans.
+   * @param span span to assert over.
+   */
+  protected void assertActiveSpan(final AuditSpan span) {
+    assertThat(activeSpan())
+        .isSameAs(span);
+  }
+
+  /**
+   * Assert a span is unbound/invalid.
+   * @param span span to assert over.
+   */
+  protected void assertUnbondedSpan(final AuditSpan span) {
+    assertThat(span.isValidSpan())
+        .describedAs("Validity of %s", span)
+        .isFalse();
+  }
+
+  protected AuditSpanS3A activeSpan() {
+    return manager.getActiveAuditSpan();
+  }
+
+  /**
+   * Create a head request and pass it through the manager's beforeExecution()
+   * callback.
+   * @return a processed request.
+   */
+  protected GetObjectMetadataRequest head() {
+    return manager.beforeExecution(
+        requestFactory.newGetObjectMetadataRequest("/"));
+  }
+
+  /**
+   * Assert a head request fails as there is no
+   * active span.
+   */
+  protected void assertHeadUnaudited() throws Exception {
+    intercept(AuditFailureException.class,
+        UNAUDITED_OPERATION, this::head);
+  }
+
+  /**
+   * Assert that the audit failure is of a given value.
+   * Returns the value to assist in chaining,
+   * @param expected expected value
+   * @return the expected value.
+   */
+  protected long verifyAuditFailureCount(
+      final long expected) {
+    return verifyCounter(Statistic.AUDIT_FAILURE, expected);
+  }
+
+  /**
+   * Assert that the audit execution count
+   * is of a given value.
+   * Returns the value to assist in chaining,
+   * @param expected expected value
+   * @return the expected value.
+   */
+  protected long verifyAuditExecutionCount(
+      final long expected) {
+    return verifyCounter(Statistic.AUDIT_REQUEST_EXECUTION, expected);
+  }
+
+  /**
+   * Assert that a statistic counter is of a given value.
+   * Returns the value to assist in chaining,
+   * @param statistic statistic to check
+   * @param expected expected value
+   * @return the expected value.
+   */
+  protected long verifyCounter(final Statistic statistic,

Review comment:
       can make this private.

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.FileNotFoundException;
+import java.io.IOException;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.security.AccessControlException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_ACCESS_CHECK_FAILURE;
+import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION;
+import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_ACCESS;
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME;
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions;
+import static 
org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_ALL_PROBES;
+import static 
org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_FILE_PROBE;
+import static 
org.apache.hadoop.fs.s3a.performance.OperationCost.ROOT_FILE_STATUS_PROBE;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
+
+/**
+ * Test S3A FS Access permit/deny is passed through all the way to the
+ * auditor.
+ * Uses {@link AccessCheckingAuditor} to enable/disable access.
+ * There are not currently any contract tests for this; behaviour
+ * based on base FileSystem implementation.
+ */
+public class ITestAuditAccessChecks extends AbstractS3ACostTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestAuditAccessChecks.class);
+
+  private AccessCheckingAuditor auditor;
+
+  public ITestAuditAccessChecks() {
+    super(true);
+  }
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    resetAuditOptions(conf);
+    conf.set(AUDIT_SERVICE_CLASSNAME, AccessCheckingAuditor.CLASS);
+    return conf;
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    auditor = (AccessCheckingAuditor) getFileSystem().getAuditor();
+  }
+
+  @Test
+  public void testFileAccessAllowed() throws Throwable {
+    describe("Enable checkaccess and verify it works with expected"
+        + " statitics");
+    auditor.setAccessAllowed(true);
+    Path path = methodPath();
+    S3AFileSystem fs = getFileSystem();
+    touch(fs, path);
+    verifyMetrics(
+        () -> access(fs, path),
+        with(INVOCATION_ACCESS, 1),
+        whenRaw(FILE_STATUS_FILE_PROBE));
+  }
+
+  private String access(final S3AFileSystem fs, final Path path)

Review comment:
       Maybe move this to the end of the test, after all ```@Test```.

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.nio.file.AccessDeniedException;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.WriteOperationHelper;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+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.AuditTestSupport.enableLoggingAuditor;
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Test S3A FS Access permit/deny is passed through all the way to the
+ * auditor.
+ * Uses {@link AccessCheckingAuditor} to enable/disable access.
+ * There are not currently any contract tests for this; behaviour
+ * based on base FileSystem implementation.
+ */
+public class ITestAuditManager extends AbstractS3ACostTest {
+
+  public ITestAuditManager() {
+    super(true);
+  }
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    resetAuditOptions(conf);
+    enableLoggingAuditor(conf);
+    conf.set(AUDIT_REQUEST_HANDLERS,
+        SimpleAWSRequestHandler.CLASS);
+    return conf;
+  }
+
+  /**
+   * Get the FS IOStatistics.
+   * @return the FS live IOSTats.
+   */
+  private IOStatistics iostats() {
+    return getFileSystem().getIOStatistics();
+  }
+
+  /**
+   * Verify that operations outside a span are rejected
+   * by ensuring that the thread is outside a span, create
+   * a write operation helper, then
+   * reject it.
+   */
+  @Test
+  public void testInvokeOutOfSpanRejected() throws Throwable {
+    describe("Operations against S3 will be rejected outside of a span");
+    final S3AFileSystem fs = getFileSystem();
+    final long failures0 = lookupCounterStatistic(iostats(),
+        AUDIT_FAILURE.getSymbol());
+    final long exec0 = lookupCounterStatistic(iostats(),
+        AUDIT_REQUEST_EXECUTION.getSymbol());
+    // API call
+    // create and close a span, so the FS is not in a span.
+    fs.createSpan("span", null, null).close();
+
+    // this will be out of span
+    final WriteOperationHelper writer
+        = fs.getWriteOperationHelper();
+
+    // which can be verified
+    Assertions.assertThat(writer.getAuditSpan())
+        .matches(s -> !s.isValidSpan(), "Span is not valid");
+
+    // an S3 API call will fail and be mapped to access denial.
+    final AccessDeniedException ex = intercept(
+        AccessDeniedException.class, UNAUDITED_OPERATION, () ->
+            writer.listMultipartUploads("/"));
+
+    // verify the type of the inner cause, throwing the outer ex
+    // if it is null or a different class
+    if (!(ex.getCause() instanceof AuditFailureException)) {
+      throw ex;
+    }
+
+    assertThatStatisticCounter(iostats(), AUDIT_REQUEST_EXECUTION.getSymbol())
+        .isGreaterThan(exec0);
+    assertThatStatisticCounter(iostats(), AUDIT_FAILURE.getSymbol())
+        .isGreaterThan(failures0);
+  }
+
+  @Test
+  public void testRequestHandlerBinding() throws Throwable {
+    describe("Verify that extra request handlers can be added and that they"
+        + " will be invoked during request execution");
+    final long baseCount = SimpleAWSRequestHandler.getInvocationCount();
+    final S3AFileSystem fs = getFileSystem();
+    final long exec0 = lookupCounterStatistic(iostats(),
+        AUDIT_REQUEST_EXECUTION.getSymbol());
+    // API call
+    fs.getBucketLocation();
+    // which MUST have ended up calling the extension request handler
+    Assertions.assertThat(SimpleAWSRequestHandler.getInvocationCount())
+        .describedAs("Invocatin count of plugged in request handler")

Review comment:
       typo: "invocation"

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.net.URISyntaxException;
+import java.util.Map;
+import java.util.regex.Matcher;
+
+import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
+import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER;
+import static org.apache.hadoop.fs.s3a.audit.S3LogParser.*;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER;
+import static 
org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader.maybeStripWrappedQuotes;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_FILESYSTEM_ID;
+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.PARAM_PRINCIPAL;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for referrer audit header generation/parsing.
+ */
+public class TestHttpReferrerAuditHeader extends AbstractAuditingTest {
+
+  /**
+   * Logging.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestHttpReferrerAuditHeader.class);
+
+  private LoggingAuditor auditor;
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+
+    auditor = (LoggingAuditor) getManager().getAuditor();
+  }
+
+  /**
+   * Creaate the config from {@link AuditTestSupport#loggingAuditConfig()}
+   * and patch in filtering for fields x1, x2, x3.
+   * @return a logging configuration.
+   */
+  protected Configuration createConfig() {
+    final Configuration conf = loggingAuditConfig();
+    conf.set(REFERRER_HEADER_FILTER, "x1, x2, x3");
+    return conf;
+  }
+
+  /**
+   * This verifies that passing a request through the audit manager
+   * causes the http referrer header to be added, that it can
+   * be split to query parameters, and that those parameters match
+   * those of the active wrapped span.
+   */
+  @Test
+  public void testHttpReferrerPatchesTheRequest() throws Throwable {
+    AuditSpan span = span();
+    long ts = span.getTimestamp();
+    GetObjectMetadataRequest request = head();
+    Map<String, String> headers
+        = request.getCustomRequestHeaders();
+    assertThat(headers)
+        .describedAs("Custom headers")
+        .containsKey(HEADER_REFERRER);
+    String header = headers.get(HEADER_REFERRER);
+    LOG.info("Header is {}", header);
+    Map<String, String> params
+        = HttpReferrerAuditHeader.extractQueryParameters(header);
+    assertMapContains(params, PARAM_PRINCIPAL,
+        UserGroupInformation.getCurrentUser().getUserName());
+    assertMapContains(params, PARAM_FILESYSTEM_ID, auditor.getAuditorId());
+    assertMapContains(params, PARAM_OP, OPERATION);
+    assertMapContains(params, PARAM_PATH, PATH_1);
+    assertMapContains(params, PARAM_PATH2, PATH_2);
+    String threadID = CommonAuditContext.currentThreadID();
+    assertMapContains(params, PARAM_THREAD0, threadID);
+    assertMapContains(params, PARAM_THREAD1, threadID);
+    assertMapContains(params, PARAM_ID, span.getSpanId());
+    assertThat(span.getTimestamp())
+        .describedAs("Timestamp of " + span)
+        .isEqualTo(ts);
+
+    assertMapContains(params, PARAM_TIMESTAMP,
+        Long.toString(ts));
+  }
+
+  @Test
+  public void testHeaderComplexPaths() throws Throwable {
+    String p1 = "s3a://dotted.bucket/path: value/subdir";
+    String p2 = "s3a://key/";
+    AuditSpan span = getManager().createSpan(OPERATION, p1, p2);
+    long ts = span.getTimestamp();
+    Map<String, String> params = issueRequestAndExtractParameters();
+    assertMapContains(params, PARAM_PRINCIPAL,
+        UserGroupInformation.getCurrentUser().getUserName());
+    assertMapContains(params, PARAM_FILESYSTEM_ID, auditor.getAuditorId());
+    assertMapContains(params, PARAM_OP, OPERATION);
+    assertMapContains(params, PARAM_PATH, p1);
+    assertMapContains(params, PARAM_PATH2, p2);
+    String threadID = CommonAuditContext.currentThreadID();
+    assertMapContains(params, PARAM_THREAD0, threadID);
+    assertMapContains(params, PARAM_THREAD1, threadID);
+    assertMapContains(params, PARAM_ID, span.getSpanId());
+    assertThat(span.getTimestamp())
+        .describedAs("Timestamp of " + span)
+        .isEqualTo(ts);
+
+    assertMapContains(params, PARAM_TIMESTAMP,
+        Long.toString(ts));
+  }
+
+  /**
+   * Issue a request, then get the header field and parse it to the parameter.
+   * @return map of query params on the referrer header.
+   * @throws URISyntaxException failure to parse the header as a URI.
+   */
+  private Map<String, String> issueRequestAndExtractParameters()
+      throws URISyntaxException {
+    head();
+    return HttpReferrerAuditHeader.extractQueryParameters(
+        auditor.getLastHeader());
+  }
+
+
+  /**
+   * Test that headers are filtered out if configured.
+   */
+  @Test
+  public void testHeaderFiltering() throws Throwable {
+    // add two attributes, x2 will be filtered.
+    AuditSpan span = getManager().createSpan(OPERATION, null, null);
+    auditor.addAttribute("x0", "x0");
+    auditor.addAttribute("x2", "x2");
+    final Map<String, String> params
+        = issueRequestAndExtractParameters();
+    assertThat(params)
+        .doesNotContainKey("x2");
+
+  }
+
+  /**
+   * A real log entry.
+   * This is derived from a real log entry on a test run.
+   * If this needs to be updated, please do it from a real log.
+   * Splitting this up across lines has a tendency to break things, so
+   * be careful making changes.
+   */
+  public static final String SAMPLE_LOG_ENTRY =
+      "183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000"
+          + " bucket-london"
+          + " [13/May/2021:11:26:06 +0000]"
+          + " 109.157.171.174"
+          + " arn:aws:iam::152813717700:user/dev"
+          + " M7ZB7C4RTKXJKTM9"
+          + " REST.PUT.OBJECT"
+          + " fork-0001/test/testParseBrokenCSVFile"
+          + " \"PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1\""
+          + " 200"
+          + " -"
+          + " -"
+          + " 794"
+          + " 55"
+          + " 17"
+          + " \"https://audit.example.org/op_create/";
+          + "e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/"
+          + "?op=op_create"
+          + "&p1=fork-0001/test/testParseBrokenCSVFile"
+          + "&pr=alice"
+          + "&ps=2eac5a04-2153-48db-896a-09bc9a2fd132"
+          + "&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154"
+          + "&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&"
+          + "ts=1620905165700\""
+          + " \"Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK\""
+          + " -"
+          + " TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0="
+          + " SigV4"
+          + " ECDHE-RSA-AES128-GCM-SHA256"
+          + " AuthHeader"
+          + " bucket-london.s3.eu-west-2.amazonaws.com"
+          + " TLSv1.2";
+
+  private static final String DESCRIPTION = String.format(
+      "log entry %s split by %s", SAMPLE_LOG_ENTRY,
+      LOG_ENTRY_PATTERN);
+
+  /**
+   * Match the log entry and validate the results.
+   */
+  @Test
+  public void testMatchAWSLogEntry() throws Throwable {
+
+    LOG.info("Matcher pattern is\n'{}'", LOG_ENTRY_PATTERN);
+    LOG.info("Log entry is\n'{}'", SAMPLE_LOG_ENTRY);
+    final Matcher matcher = LOG_ENTRY_PATTERN.matcher(SAMPLE_LOG_ENTRY);
+
+    // match the pattern against the entire log entry.
+    assertThat(matcher.matches())

Review comment:
       Don't think we'll see the error message if this assert fails, better to 
use ```assertTrue(msg, matcher.matches())```

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.handlers.RequestHandler2;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+
+import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Unit tests related to span lifecycle.
+ */
+public class TestAuditSpanLifecycle extends AbstractAuditingTest {
+
+  private Configuration conf;
+
+  private AuditSpan resetSpan;
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    resetSpan = getManager().getActiveAuditSpan();
+  }
+
+  protected Configuration createConfig() {
+    return noopAuditConfig();
+  }
+
+  /**
+   * Core lifecycle (remember: the service has already been started).
+   */
+  @Test
+  public void testStop() throws Throwable {
+    getManager().stop();
+  }
+
+  @Test
+  public void testCreateRequestHandlers() throws Throwable {
+    List<RequestHandler2> handlers
+        = getManager().createRequestHandlers();
+    assertThat(handlers).isNotEmpty();
+  }
+
+  @Test
+  public void testInitialSpanIsInvalid() throws Throwable {
+    assertThat(resetSpan)
+        .matches(f -> !f.isValidSpan(), "is invalid");
+  }
+
+  @Test
+  public void testCreateCloseSpan() throws Throwable {
+    AuditSpan span = getManager().createSpan("op", null, null);
+    assertThat(span)
+        .matches(AuditSpan::isValidSpan, "is valid");
+    assertActiveSpan(span);
+    // activation when already active is no-op
+    span.activate();
+    assertActiveSpan(span);
+    // close the span
+    span.close();
+    // the original span is restored.
+    assertActiveSpan(resetSpan);
+  }
+
+  @Test
+  public void testSpanActivation() throws Throwable {
+    // real activation switches spans in the current thead.
+
+    AuditSpan span1 = getManager().createSpan("op1", null, null);
+    AuditSpan span2 = getManager().createSpan("op2", null, null);
+    assertActiveSpan(span2);
+    // switch back to span 1
+    span1.activate();
+    assertActiveSpan(span1);
+    // then to span 2
+    span2.activate();
+    assertActiveSpan(span2);
+    span2.close();
+

Review comment:
       should we close span1 here? Maybe some assertion regarding span1's 
lifecycle after span2 was closed?

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.FileNotFoundException;
+import java.io.IOException;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.security.AccessControlException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_ACCESS_CHECK_FAILURE;
+import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION;
+import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_ACCESS;
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME;
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions;
+import static 
org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_ALL_PROBES;
+import static 
org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_FILE_PROBE;
+import static 
org.apache.hadoop.fs.s3a.performance.OperationCost.ROOT_FILE_STATUS_PROBE;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
+
+/**
+ * Test S3A FS Access permit/deny is passed through all the way to the
+ * auditor.
+ * Uses {@link AccessCheckingAuditor} to enable/disable access.
+ * There are not currently any contract tests for this; behaviour
+ * based on base FileSystem implementation.
+ */
+public class ITestAuditAccessChecks extends AbstractS3ACostTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestAuditAccessChecks.class);
+
+  private AccessCheckingAuditor auditor;
+
+  public ITestAuditAccessChecks() {
+    super(true);
+  }
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    resetAuditOptions(conf);
+    conf.set(AUDIT_SERVICE_CLASSNAME, AccessCheckingAuditor.CLASS);
+    return conf;
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    auditor = (AccessCheckingAuditor) getFileSystem().getAuditor();
+  }
+
+  @Test
+  public void testFileAccessAllowed() throws Throwable {
+    describe("Enable checkaccess and verify it works with expected"
+        + " statitics");

Review comment:
       typo: "statistics"

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.net.URISyntaxException;
+import java.util.Map;
+import java.util.regex.Matcher;
+
+import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
+import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER;
+import static org.apache.hadoop.fs.s3a.audit.S3LogParser.*;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER;
+import static 
org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader.maybeStripWrappedQuotes;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_FILESYSTEM_ID;
+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.PARAM_PRINCIPAL;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for referrer audit header generation/parsing.
+ */
+public class TestHttpReferrerAuditHeader extends AbstractAuditingTest {
+
+  /**
+   * Logging.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestHttpReferrerAuditHeader.class);
+
+  private LoggingAuditor auditor;
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+
+    auditor = (LoggingAuditor) getManager().getAuditor();
+  }
+
+  /**
+   * Creaate the config from {@link AuditTestSupport#loggingAuditConfig()}
+   * and patch in filtering for fields x1, x2, x3.
+   * @return a logging configuration.
+   */
+  protected Configuration createConfig() {
+    final Configuration conf = loggingAuditConfig();
+    conf.set(REFERRER_HEADER_FILTER, "x1, x2, x3");
+    return conf;
+  }
+
+  /**
+   * This verifies that passing a request through the audit manager
+   * causes the http referrer header to be added, that it can
+   * be split to query parameters, and that those parameters match
+   * those of the active wrapped span.
+   */
+  @Test
+  public void testHttpReferrerPatchesTheRequest() throws Throwable {
+    AuditSpan span = span();
+    long ts = span.getTimestamp();
+    GetObjectMetadataRequest request = head();
+    Map<String, String> headers
+        = request.getCustomRequestHeaders();
+    assertThat(headers)
+        .describedAs("Custom headers")
+        .containsKey(HEADER_REFERRER);
+    String header = headers.get(HEADER_REFERRER);
+    LOG.info("Header is {}", header);
+    Map<String, String> params
+        = HttpReferrerAuditHeader.extractQueryParameters(header);
+    assertMapContains(params, PARAM_PRINCIPAL,
+        UserGroupInformation.getCurrentUser().getUserName());
+    assertMapContains(params, PARAM_FILESYSTEM_ID, auditor.getAuditorId());
+    assertMapContains(params, PARAM_OP, OPERATION);
+    assertMapContains(params, PARAM_PATH, PATH_1);
+    assertMapContains(params, PARAM_PATH2, PATH_2);
+    String threadID = CommonAuditContext.currentThreadID();
+    assertMapContains(params, PARAM_THREAD0, threadID);
+    assertMapContains(params, PARAM_THREAD1, threadID);
+    assertMapContains(params, PARAM_ID, span.getSpanId());
+    assertThat(span.getTimestamp())
+        .describedAs("Timestamp of " + span)
+        .isEqualTo(ts);
+
+    assertMapContains(params, PARAM_TIMESTAMP,
+        Long.toString(ts));
+  }
+
+  @Test
+  public void testHeaderComplexPaths() throws Throwable {

Review comment:
       javadoc for this test would be helpful.

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.nio.file.AccessDeniedException;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.WriteOperationHelper;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+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.AuditTestSupport.enableLoggingAuditor;
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
+import static 
org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Test S3A FS Access permit/deny is passed through all the way to the

Review comment:
       JavaDocs doesn't seem right for this test.

##########
File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
##########
@@ -1214,21 +1359,76 @@ private FSDataInputStream open(
           fileStatus,
           policy,
           changeDetectionPolicy,
-          readAheadRange2);
+          readAheadRange2,
+          auditSpan);
     } else {
       readContext = createReadContext(
           fileStatus,
           inputPolicy,
           changeDetectionPolicy,
-          readAhead);
+          readAhead,
+          auditSpan);
     }
     LOG.debug("Opening '{}'", readContext);
 
     return new FSDataInputStream(
         new S3AInputStream(
             readContext,
             createObjectAttributes(fileStatus),
-            s3));
+            createInputStreamCallbacks(auditSpan)));
+  }
+
+  /**
+   * Overrride point: create the callbacks for S3AInputStream.

Review comment:
       typo: "Override"

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.net.URISyntaxException;
+import java.util.Map;
+import java.util.regex.Matcher;
+
+import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
+import org.apache.hadoop.fs.audit.CommonAuditContext;
+import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static 
org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig;
+import static 
org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER;
+import static org.apache.hadoop.fs.s3a.audit.S3LogParser.*;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER;
+import static 
org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader.maybeStripWrappedQuotes;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_FILESYSTEM_ID;
+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.PARAM_PRINCIPAL;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD0;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_THREAD1;
+import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_TIMESTAMP;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for referrer audit header generation/parsing.
+ */
+public class TestHttpReferrerAuditHeader extends AbstractAuditingTest {
+
+  /**
+   * Logging.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestHttpReferrerAuditHeader.class);
+
+  private LoggingAuditor auditor;
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+
+    auditor = (LoggingAuditor) getManager().getAuditor();
+  }
+
+  /**
+   * Creaate the config from {@link AuditTestSupport#loggingAuditConfig()}

Review comment:
       typo: "Create"




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

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



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

Reply via email to