|
@@ -29,6 +29,7 @@ import java.util.Locale;
|
|
|
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.InterfaceAudience;
|
|
|
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 @@ import static org.apache.hadoop.fs.audit.AuditConstants.REFERRER_ORIGIN_HOST;
|
|
|
* {@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.
|
|
|
+ * <p>
|
|
|
+ * 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.
|
|
|
+ * <p>
|
|
|
+ * 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.
|
|
|
- *
|
|
|
+ * <p>
|
|
|
+ * All maps/enums passed down are copied into thread safe equivalents.
|
|
|
+ * as their origin is unknown and cannot be guaranteed to
|
|
|
+ * not be shared.
|
|
|
+ * <p>
|
|
|
* 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 @@ public final class HttpReferrerAuditHeader {
|
|
|
* per entry, and "" returned.
|
|
|
* @return a referrer string or ""
|
|
|
*/
|
|
|
- public String buildHttpReferrer() {
|
|
|
+ public synchronized String buildHttpReferrer() {
|
|
|
|
|
|
String header;
|
|
|
try {
|
|
|
+ Map<String, String> 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 final class HttpReferrerAuditHeader {
|
|
|
} 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 final class HttpReferrerAuditHeader {
|
|
|
* @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);
|