From 99cda77b3888107fd280c584c6534202e21dc792 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 2 Oct 2024 19:54:10 +0100 Subject: [PATCH 1/4] HADOOP-19299. HttpReferrerAuditHeader resilience Catch all exceptions raised when building logger header log message at info; stack at debug Change-Id: I119abd167978dbc515554a36189462599870d1f9 --- .../store/audit/HttpReferrerAuditHeader.java | 34 ++++++++++--- .../s3a/audit/impl/ActiveAuditManagerS3A.java | 12 ++++- .../fs/s3a/audit/impl/LoggingAuditor.java | 16 +++++- .../audit/TestHttpReferrerAuditHeader.java | 22 +++++++++ .../fs/s3a/audit/impl/ReferrerExtractor.java | 49 +++++++++++++++++++ 5 files changed, 122 insertions(+), 11 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/ReferrerExtractor.java 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..3770980d8066e 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,10 @@ * {@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. */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -122,7 +128,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 +140,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 +176,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 +200,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. + LOG.warn("Failed to construct referred header " + e , e); + LOG.debug("Full stack", e); + header = ""; } + return header; } @@ -200,7 +218,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..95a0c0abf73cd 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,24 @@ private void expectStrippedField(final String str, .describedAs("Stripped <%s>", str) .isEqualTo(ex); } + + @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()); + final String header = referrer.buildHttpReferrer(); + Assertions.assertThat(header) + .describedAs("referrer header") + .isBlank(); + } finally { + // critical to remove this so it doesn't interfere with any other + // tests + auditContext.remove(failing); + } + } } \ No newline at end of file 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..84d01c2645774 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/impl/ReferrerExtractor.java @@ -0,0 +1,49 @@ +/* + * 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 { + + /** + * 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); + } +} From 0be63c1b42a55ee8c3c47b2306031b6df9a913b1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 3 Oct 2024 13:37:05 +0100 Subject: [PATCH 2/4] HADOOP-19299. HttpReferrerAuditHeader resilience style checks + extra javadocs Change-Id: I1fbe381c49f51f9f4df62fd674a693c89ce9e9ed --- .../hadoop/fs/store/audit/HttpReferrerAuditHeader.java | 5 ++++- .../hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java | 2 +- .../apache/hadoop/fs/s3a/audit/impl/ReferrerExtractor.java | 3 +++ 3 files changed, 8 insertions(+), 2 deletions(-) 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 3770980d8066e..025682bcf84e2 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 @@ -63,6 +63,9 @@ * 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 @@ -203,7 +206,7 @@ public synchronized String buildHttpReferrer() { } catch (RuntimeException e) { // do not let failure to build the header stop the request being // issued. - LOG.warn("Failed to construct referred header " + e , e); + LOG.warn("Failed to construct referred header {}", e.toString(), e); LOG.debug("Full stack", e); header = ""; } 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 95a0c0abf73cd..062a8cc8c3a76 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 @@ -439,4 +439,4 @@ public void testSpanResilience() throws Throwable { auditContext.remove(failing); } } -} \ No newline at end of file +} 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 index 84d01c2645774..80d6512c976be 100644 --- 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 @@ -27,6 +27,9 @@ */ public final class ReferrerExtractor { + private ReferrerExtractor() { + } + /** * Get the referrer provided the span is an instance or * subclass of LoggingAuditSpan. From fd284537ce4fae0a2e886d5c7eddc095e1fd19c2 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 4 Oct 2024 15:04:41 +0100 Subject: [PATCH 3/4] HADOOP-19299. review feedback Change-Id: If7a1ff24de449be395667f550341389a5083097c --- .../apache/hadoop/fs/store/audit/HttpReferrerAuditHeader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 025682bcf84e2..40cd63e6bb012 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 @@ -206,7 +206,7 @@ public synchronized String buildHttpReferrer() { } catch (RuntimeException e) { // do not let failure to build the header stop the request being // issued. - LOG.warn("Failed to construct referred header {}", e.toString(), e); + LOG.warn("Failed to construct referred header {}", e.toString()); LOG.debug("Full stack", e); header = ""; } From 0883106fa3a7ab7f379892db7eb07f8246b1d618 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 4 Oct 2024 18:31:35 +0100 Subject: [PATCH 4/4] HADOOP-19299. log only once Let's no flood the logs if there is a recurrent problem, ok? Change-Id: I189a5b9cb7c6d264bc4835a631e3da5ec3bb22de --- .../fs/store/audit/HttpReferrerAuditHeader.java | 10 +++++++++- .../fs/s3a/audit/TestHttpReferrerAuditHeader.java | 13 +++++++++++-- 2 files changed, 20 insertions(+), 3 deletions(-) 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 40cd63e6bb012..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 @@ -90,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; @@ -206,7 +214,7 @@ public synchronized String buildHttpReferrer() { } catch (RuntimeException e) { // do not let failure to build the header stop the request being // issued. - LOG.warn("Failed to construct referred header {}", e.toString()); + ERROR_BUILDING_REFERRER_HEADER.warn("Failed to construct referred header {}", e.toString()); LOG.debug("Full stack", e); header = ""; } 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 062a8cc8c3a76..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 @@ -420,6 +420,10 @@ private void expectStrippedField(final String str, .isEqualTo(ex); } + /** + * 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(); @@ -429,10 +433,15 @@ public void testSpanResilience() throws Throwable { }); try { final HttpReferrerAuditHeader referrer = ReferrerExtractor.getReferrer(auditor, span()); - final String header = referrer.buildHttpReferrer(); - Assertions.assertThat(header) + 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