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
index b2684e758892a..01a36b24fb2f6 100644
--- 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
@@ -29,6 +29,7 @@
import java.util.Map;
import java.util.Set;
import java.util.StringJoiner;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -40,6 +41,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.audit.CommonAuditContext;
import org.apache.hadoop.fs.store.LogExactlyOnce;
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
import org.apache.http.NameValuePair;
import org.apache.http.client.utils.URLEncodedUtils;
@@ -57,6 +59,13 @@
* {@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.
+ *
+ * This header may be shared across multiple threads at the same time.
+ * so some methods are marked as synchronized, specifically those reading
+ * or writing the attribute map.
+ *
+ * For the same reason, maps and lists passed down during construction are
+ * copied into thread safe structures.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@@ -81,6 +90,14 @@ public final class HttpReferrerAuditHeader {
private static final LogExactlyOnce WARN_OF_URL_CREATION =
new LogExactlyOnce(LOG);
+ /**
+ * Log for warning of an exception raised when building
+ * the referrer header, including building the evaluated
+ * attributes.
+ */
+ private static final LogExactlyOnce ERROR_BUILDING_REFERRER_HEADER =
+ new LogExactlyOnce(LOG);
+
/** Context ID. */
private final String contextId;
@@ -122,7 +139,11 @@ public final class HttpReferrerAuditHeader {
/**
* Instantiate.
- *
+ *
+ * All maps/enums passed down are copied into thread safe equivalents.
+ * as their origin is unknown and cannot be guaranteed to
+ * not be shared.
+ *
* 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.
@@ -130,15 +151,15 @@ public final class HttpReferrerAuditHeader {
private HttpReferrerAuditHeader(
final Builder builder) {
this.contextId = requireNonNull(builder.contextId);
- this.evaluated = builder.evaluated;
- this.filter = builder.filter;
+ this.evaluated = new ConcurrentHashMap<>(builder.evaluated);
+ this.filter = ImmutableSet.copyOf(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;
+ attributes = new ConcurrentHashMap<>(builder.attributes);
addAttribute(PARAM_OP, operationName);
addAttribute(PARAM_PATH, path1);
@@ -166,17 +187,18 @@ private HttpReferrerAuditHeader(
* per entry, and "" returned.
* @return a referrer string or ""
*/
- public String buildHttpReferrer() {
+ public synchronized String buildHttpReferrer() {
String header;
try {
+ Map requestAttrs = new HashMap<>(attributes);
String queries;
// Update any params which are dynamically evaluated
evaluated.forEach((key, eval) ->
- addAttribute(key, eval.get()));
+ requestAttrs.put(key, eval.get()));
// now build the query parameters from all attributes, static and
// evaluated, stripping out any from the filter
- queries = attributes.entrySet().stream()
+ queries = requestAttrs.entrySet().stream()
.filter(e -> !filter.contains(e.getKey()))
.map(e -> e.getKey() + "=" + e.getValue())
.collect(Collectors.joining("&"));
@@ -189,7 +211,14 @@ public String buildHttpReferrer() {
} catch (URISyntaxException e) {
WARN_OF_URL_CREATION.warn("Failed to build URI for auditor: " + e, e);
header = "";
+ } catch (RuntimeException e) {
+ // do not let failure to build the header stop the request being
+ // issued.
+ ERROR_BUILDING_REFERRER_HEADER.warn("Failed to construct referred header {}", e.toString());
+ LOG.debug("Full stack", e);
+ header = "";
}
+
return header;
}
@@ -200,7 +229,7 @@ public String buildHttpReferrer() {
* @param key query key
* @param value query value
*/
- private void addAttribute(String key,
+ private synchronized void addAttribute(String key,
String value) {
if (StringUtils.isNotEmpty(value)) {
attributes.put(key, value);
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
index 9dd04af68e8a9..e8e989efaa141 100644
--- 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
@@ -700,7 +700,8 @@ public SdkResponse modifyResponse(Context.ModifyResponse context,
* span is deactivated.
* Package-private for testing.
*/
- private final class WrappingAuditSpan extends AbstractAuditSpanImpl {
+ @VisibleForTesting
+ final class WrappingAuditSpan extends AbstractAuditSpanImpl {
/**
* Inner span.
@@ -792,6 +793,15 @@ public boolean isValidSpan() {
return isValid && span.isValidSpan();
}
+ /**
+ * Get the inner span.
+ * @return the span.
+ */
+ @VisibleForTesting
+ AuditSpanS3A getSpan() {
+ return span;
+ }
+
/**
* Forward to the inner span.
* {@inheritDoc}
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
index 5f93454795a94..16bae4b816457 100644
--- 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
@@ -38,6 +38,7 @@
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.audit.AuditConstants;
import org.apache.hadoop.fs.audit.CommonAuditContext;
@@ -252,6 +253,17 @@ private void setLastHeader(final String lastHeader) {
this.lastHeader = lastHeader;
}
+ /**
+ * Get the referrer provided the span is an instance or
+ * subclass of LoggingAuditSpan.
+ * @param span span
+ * @return the referrer
+ * @throws ClassCastException if a different span type was passed in
+ */
+ @VisibleForTesting
+ HttpReferrerAuditHeader getReferrer(AuditSpanS3A span) {
+ return ((LoggingAuditSpan) span).getReferrer();
+ }
/**
* Span which logs at debug and sets the HTTP referrer on
* invocations.
@@ -441,10 +453,10 @@ public String toString() {
}
/**
- * Get the referrer; visible for tests.
+ * Get the referrer.
* @return the referrer.
*/
- HttpReferrerAuditHeader getReferrer() {
+ private HttpReferrerAuditHeader getReferrer() {
return referrer;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
index 1dfa94cc5adaf..43155fe239f79 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java
@@ -24,6 +24,7 @@
import java.util.Map;
import java.util.regex.Matcher;
+import org.assertj.core.api.Assertions;
import software.amazon.awssdk.http.SdkHttpRequest;
import org.junit.Before;
import org.junit.Test;
@@ -32,6 +33,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor;
+import org.apache.hadoop.fs.s3a.audit.impl.ReferrerExtractor;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.fs.audit.CommonAuditContext;
import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader;
@@ -417,4 +419,33 @@ private void expectStrippedField(final String str,
.describedAs("Stripped <%s>", str)
.isEqualTo(ex);
}
-}
\ No newline at end of file
+
+ /**
+ * Verify that exceptions raised when building referrer headers
+ * do not result in failures, just an empty header.
+ */
+ @Test
+ public void testSpanResilience() throws Throwable {
+ final CommonAuditContext auditContext = CommonAuditContext.currentAuditContext();
+ final String failing = "failing";
+ auditContext.put(failing, () -> {
+ throw new RuntimeException("raised");
+ });
+ try {
+ final HttpReferrerAuditHeader referrer = ReferrerExtractor.getReferrer(auditor, span());
+ Assertions.assertThat(referrer.buildHttpReferrer())
+ .describedAs("referrer header")
+ .isBlank();
+ // repeat
+ LOG.info("second attempt: there should be no second warning below");
+ Assertions.assertThat(referrer.buildHttpReferrer())
+ .describedAs("referrer header 2")
+ .isBlank();
+ referrer.buildHttpReferrer();
+ } finally {
+ // critical to remove this so it doesn't interfere with any other
+ // tests
+ auditContext.remove(failing);
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/ReferrerExtractor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/ReferrerExtractor.java
new file mode 100644
index 0000000000000..80d6512c976be
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/ReferrerExtractor.java
@@ -0,0 +1,52 @@
+/*
+ * 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.HttpReferrerAuditHeader;
+
+/**
+ * Extract the referrer from a LoggingAuditor through a package-private
+ * method.
+ */
+public final class ReferrerExtractor {
+
+ private ReferrerExtractor() {
+ }
+
+ /**
+ * Get the referrer provided the span is an instance or
+ * subclass of LoggingAuditSpan.
+ * If wrapped by a {@code WrappingAuditSpan}, it will be extracted.
+ * @param auditor the auditor.
+ * @param span span
+ * @return the referrer
+ * @throws ClassCastException if a different span type was passed in
+ */
+ public static HttpReferrerAuditHeader getReferrer(LoggingAuditor auditor,
+ AuditSpanS3A span) {
+ AuditSpanS3A sp;
+ if (span instanceof ActiveAuditManagerS3A.WrappingAuditSpan) {
+ sp = ((ActiveAuditManagerS3A.WrappingAuditSpan) span).getSpan();
+ } else {
+ sp = span;
+ }
+ return auditor.getReferrer(sp);
+ }
+}