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
This commit is contained in:
Steve Loughran 2021-05-25 10:25:41 +01:00
parent ad7071c93b
commit 464bbd5b7c
No known key found for this signature in database
GPG Key ID: D22CF846DBB162A0
134 changed files with 11986 additions and 1156 deletions

View File

@ -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";
}

View File

@ -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.audit;
/**
* Statistic Names for Auditing.
*/
public final class AuditStatisticNames {
private AuditStatisticNames() {
}
/**
* 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";
}

View File

@ -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();
}
}
}

View File

@ -0,0 +1,29 @@
/*
* 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.
*/
/**
* Public classes for adding information to any auditing information
* picked up by filesystem clients.
*
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
package org.apache.hadoop.fs.audit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -38,6 +38,8 @@
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 @@ protected void processPaths(PathData parent,
}
}
}
cleanupRemoteIterator(itemsIterator);
}
private void processPathInternal(PathData item) throws IOException {

View File

@ -39,6 +39,8 @@
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 RemoteIterator<PathData> getDirectoryContentsIterator()
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));
}
/**

View File

@ -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() {
}

View File

@ -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) {
}
}

View File

@ -28,6 +28,7 @@
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;
@ -97,6 +98,15 @@ public static IOStatistics emptyStatistics() {
return EmptyIOStatistics.getInstance();
}
/**
* 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.
@ -573,6 +583,38 @@ public static <B> Callable<B> trackDurationOfCallable(
};
}
/**
* 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.

View File

@ -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 void warn(String format, Object...args) {
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);
}
}
}

View File

@ -0,0 +1,38 @@
/*
* 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;
/**
* 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 interface ActiveThreadSpanSource<T extends AuditSpan> {
/**
* 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();
}

View File

@ -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 {
}

View File

@ -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) { }
}

View File

@ -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;
}

View File

@ -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();
};
}
}

View File

@ -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;
}
}
}

View File

@ -0,0 +1,29 @@
/*
* 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.
*/
/**
* 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}.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.store.audit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -35,6 +35,7 @@
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 @@ protected int coreServiceLaunch(Configuration conf,
}
String name = getServiceName();
LOG.debug("Launched service {}", name);
CommonAuditContext.noteEntryPoint(service);
LaunchableService launchableService = null;
if (service instanceof LaunchableService) {

View File

@ -23,6 +23,7 @@
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 static int run(Configuration conf, Tool tool, String[] args)
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();

View File

@ -189,6 +189,7 @@ public static <S> RemoteIterator<S> closingRemoteIterator(
/**
* 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 static <T> List<T> toList(RemoteIterator<T> source)
/**
* 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,18 +246,28 @@ public static <T> long foreach(
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.

View File

@ -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;
}
}

View File

@ -399,9 +399,7 @@ public static boolean rm(FileSystem fileSystem,
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 static void assertDeleted(FileSystem fs,
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);
}

View File

@ -20,12 +20,21 @@
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);

View File

@ -34,6 +34,7 @@
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 @@
* 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 @@
* 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 static <T> T once(String action, String path,
* @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 static void ignoreIOExceptions(
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 void retry(String action,
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 void maybeRetry(boolean doRetry,
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 void maybeRetry(boolean doRetry,
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 void maybeRetry(
String action,
String path,
boolean idempotent,
VoidOperation operation)
InvocationRaisingIOE operation)
throws IOException {
maybeRetry(doRetry, action, path, idempotent, retryCallback, operation);
}
@ -475,7 +477,7 @@ public <T> T retryUntranslated(
*/
public static void quietly(String action,
String path,
VoidOperation operation) {
InvocationRaisingIOE operation) {
try {
once(action, path, operation);
} catch (Exception e) {
@ -515,14 +517,6 @@ private static String toDescription(String action, @Nullable String path) {
(StringUtils.isNotEmpty(path) ? (" on " + path) : "");
}
/**
* 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

View File

@ -22,6 +22,7 @@
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.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 @@
/**
* 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 static RemoteIterator<S3AFileStatus> toProvidedFileStatusIterator(
* @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 FileStatusListingIterator createFileStatusListingIterator(
* 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 FileStatusListingIterator createFileStatusListingIterator(
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 FileStatusListingIterator createFileStatusListingIterator(
* 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 RemoteIterator<S3ALocatedFileStatus> createSingleStatusIterator(
* @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 RemoteIterator<S3ALocatedFileStatus> getListFilesAssumingDir(
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 RemoteIterator<S3ALocatedFileStatus> getListFilesAssumingDir(
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 RemoteIterator<S3ALocatedFileStatus> getListFilesAssumingDir(
* 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 RemoteIterator<S3ALocatedFileStatus> getLocatedFileStatusIteratorForDir(
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 RemoteIterator<S3ALocatedFileStatus> getLocatedFileStatusIteratorForDir(
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 RemoteIterator<S3ALocatedFileStatus> getLocatedFileStatusIteratorForDir(
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 String toString() {
* 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 @@ class ObjectListingIterator implements RemoteIterator<S3ListResult>,
* 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 @@ class ObjectListingIterator implements RemoteIterator<S3ListResult>,
.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 @@ private void fetchNextBatchAsyncIfPresent() throws IOException {
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 Path getListPath() {
public int getListingCount() {
return listingCount;
}
/**
* Close, if actually called, will close the span
* this listing was created with.
*/
@Override
public void close() {
}
}
/**

View File

@ -31,11 +31,20 @@
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 @@ private 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 @@ static class ListingIterator implements
*/
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 MultipartUploadListing next() throws IOException {
@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 @@ private void requestNextBatch() throws IOException {
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 static class UploadIterator
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();
}

View File

@ -31,6 +31,7 @@
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.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 @@ public void close() throws IOException {
final List<PartETag> partETags =
multiPartUpload.waitForAllPartUploads();
bytes = bytesSubmitted;
// then complete the operation
if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
partETags,
@ -777,6 +781,12 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block)
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);

View File

@ -20,14 +20,11 @@
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.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 S3AInputStream(S3AReadOpContext ctx,
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 @@ private synchronized void reopen(String reason, long targetPos, long length,
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 synchronized void close() throws IOException {
// 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 @@ boolean isObjectStreamOpen() {
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);
}
}

View File

@ -28,15 +28,16 @@
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;
@ -468,6 +469,14 @@ public DurationTracker trackDuration(final String key, final long count) {
return durationTrackerFactory.trackDuration(key, count);
}
/**
* 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.
@ -547,10 +556,24 @@ public void errorIgnored() {
* @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 Map<String, Long> getMap() {
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);
}
}
}

View File

@ -66,6 +66,7 @@ public int hashCode() {
/**
* Generate an S3AFileStatus instance, including etag and
* version ID, if present.
* @return the S3A status.
*/
public S3AFileStatus toS3AFileStatus() {
return new S3AFileStatus(

View File

@ -23,6 +23,7 @@
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 S3AReadOpContext(
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 long getReadahead() {
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(

View File

@ -34,7 +34,6 @@
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.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.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 static IOException translateException(@Nullable String operation,
}
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 @@ private static void initUserAgent(Configuration conf,
* 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;
}
/**

View File

@ -115,6 +115,11 @@ final class S3ClientCreationParameters {
*/
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;
}

View File

@ -22,6 +22,7 @@
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_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);

View File

@ -23,7 +23,6 @@
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.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 @@
* </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
@ -116,22 +127,50 @@ 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 @@ void operationRetried(String text, Exception ex, int retries,
/**
* 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,10 +204,33 @@ public <T> T retry(String action,
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
@ -176,15 +239,18 @@ public <T> T retry(String action,
* @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 PutObjectRequest createPutObjectRequest(String destKey,
* @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 void writeFailed(Exception ex) {
* @return a new metadata instance
*/
public ObjectMetadata newObjectMetadata(long length) {
return owner.newObjectMetadata(length);
return getRequestFactory().newObjectMetadata(length);
}
/**
@ -240,15 +309,16 @@ public ObjectMetadata newObjectMetadata(long length) {
@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 @@ private CompleteMultipartUploadResult finalizeMultipartUpload(
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 void abortMultipartUpload(String destKey, String uploadId,
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 void abortMultipartUpload(String destKey, String uploadId,
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 int abortMultipartUploadsUnderPath(String prefix)
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 int abortMultipartUploadsUnderPath(String prefix)
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 int abortMultipartUploadsUnderPath(String prefix)
* @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 void abortMultipartCommit(String destKey, String uploadId)
* 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 void abortMultipartCommit(String destKey, String uploadId)
* @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 UploadPartRequest newUploadPartRequest(
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 PutObjectResult putObject(PutObjectRequest putObjectRequest)
throws IOException {
return retry("Writing Object",
putObjectRequest.getKey(), true,
() -> owner.putObjectDirect(putObjectRequest));
withinAuditSpan(getAuditSpan(), () ->
owner.putObjectDirect(putObjectRequest)));
}
/**
@ -529,7 +578,8 @@ public UploadResult uploadObject(PutObjectRequest putObjectRequest)
// 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 UploadResult uploadObject(PutObjectRequest putObjectRequest)
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 BulkOperationState initiateOperation(final Path path,
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 Configuration getConf() {
* @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 SelectObjectContentResult select(
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 SelectObjectContentResult select(
action,
source.toString(),
true,
() -> {
withinAuditSpan(getAuditSpan(), () -> {
try (DurationInfo ignored =
new DurationInfo(LOG, "S3 Select operation")) {
try {
@ -691,11 +743,35 @@ public SelectObjectContentResult select(
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;
}
}

View File

@ -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.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 @@
* use `WriteOperationHelper` directly.
* @since Hadoop 3.3.0
*/
public interface WriteOperations {
public interface WriteOperations extends AuditSpanSource, Closeable {
/**
* Execute a function with retry processing.
@ -184,6 +186,16 @@ void abortMultipartUpload(MultipartUpload upload)
int abortMultipartUploadsUnderPath(String prefix)
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
@ -210,7 +222,7 @@ void abortMultipartCommit(String destKey, String uploadId)
* @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 @@ UploadPartRequest newUploadPartRequest(
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).

View File

@ -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);
}

View File

@ -0,0 +1,34 @@
/*
* 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.
*/
/**
* 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.
*/
@InterfaceAudience.LimitedPrivate("extensions")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.api;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -0,0 +1,41 @@
/*
* 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.fs.s3a.CredentialInitializationException;
/**
* 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 AuditFailureException extends CredentialInitializationException {
public AuditFailureException(final String message, final Throwable t) {
super(message, t);
}
public AuditFailureException(final String message) {
super(message);
}
}

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -0,0 +1,29 @@
/*
* 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.fs.store.audit.AuditSpan;
/**
* An Audit Span with S3A integration, specifically
* callbacks from the AWS client SDK.
*/
public interface AuditSpanS3A extends AuditSpan,
AWSAuditEventCallbacks {
}

View File

@ -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();
}

View File

@ -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();
}
}

View File

@ -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";
}

View File

@ -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);
}

View File

@ -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";
}

View File

@ -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();
}
}

View File

@ -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());
}
}

View File

@ -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();
}
}
}

View File

@ -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);
}
}
}

View File

@ -0,0 +1,185 @@
/*
* 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.ArrayList;
import java.util.List;
import java.util.UUID;
import com.amazonaws.handlers.RequestHandler2;
import com.amazonaws.services.s3.transfer.Transfer;
import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener;
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.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.service.CompositeService;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
/**
* Simple No-op audit manager for use before a real
* audit chain is set up, and for testing.
* Audit spans always have a unique ID and the activation/deactivation
* operations on them will update this audit manager's active span.
* It does have the service lifecycle, so do
* create a unique instance whenever used.
*/
@InterfaceAudience.Private
public class NoopAuditManagerS3A extends CompositeService
implements AuditManagerS3A, NoopSpan.SpanActivationCallbacks {
private static final NoopAuditor NOOP_AUDITOR =
NoopAuditor.createAndStartNoopAuditor(new Configuration(), null);
/**
* The inner auditor.
*/
private NoopAuditor auditor = NOOP_AUDITOR;
/**
* Thread local span. This defaults to being
* the unbonded span.
*/
private final ThreadLocal<AuditSpanS3A> activeSpan =
ThreadLocal.withInitial(this::getUnbondedSpan);
/**
* ID which is returned as a span ID in the audit event
* callbacks.
*/
private final String id;
/**
* Constructor.
* Will create and start a new instance of the auditor.
*/
public NoopAuditManagerS3A() {
super("NoopAuditManagerS3A");
id = UUID.randomUUID().toString();
}
@Override
protected void serviceInit(final Configuration conf) throws Exception {
super.serviceInit(conf);
NoopAuditor audit = new NoopAuditor(this);
final OperationAuditorOptions options =
OperationAuditorOptions.builder()
.withConfiguration(conf)
.withIoStatisticsStore(iostatisticsStore().build());
addService(audit);
audit.init(options);
}
@Override
public String getSpanId() {
return id;
}
@Override
public String getOperationName() {
return getName();
}
@Override
public OperationAuditor getAuditor() {
return auditor;
}
/**
* Unbonded span to use after deactivation.
*/
private AuditSpanS3A getUnbondedSpan() {
return auditor.getUnbondedSpan();
}
@Override
public AuditSpanS3A getActiveAuditSpan() {
return NoopSpan.INSTANCE;
}
@Override
public AuditSpanS3A createSpan(final String operation,
@Nullable final String path1,
@Nullable final String path2) throws IOException {
return createNewSpan(operation, path1, path2);
}
@Override
public List<RequestHandler2> createRequestHandlers() throws IOException {
return new ArrayList<>();
}
@Override
public TransferStateChangeListener createStateChangeListener() {
return new TransferStateChangeListener() {
public void transferStateChanged(final Transfer transfer,
final Transfer.TransferState state) {
}
};
}
/**
* Forward to the auditor.
* @param path path to check
* @param status status of the path.
* @param mode access mode.
* @throws IOException failure
*/
@Override
public boolean checkAccess(final Path path,
final S3AFileStatus status,
final FsAction mode)
throws IOException {
return auditor.checkAccess(path, status, mode);
}
@Override
public void activate(final AuditSpanS3A span) {
activeSpan.set(span);
}
@Override
public void deactivate(final AuditSpanS3A span) {
activate(getUnbondedSpan());
}
/**
* A static source of no-op spans, using the same span ID
* source as managed spans.
* @param name operation name.
* @param path1 first path of operation
* @param path2 second path of operation
* @return a span for the audit
*/
public static AuditSpanS3A createNewSpan(
final String name,
final String path1,
final String path2) {
return NOOP_AUDITOR.createSpan(name, path1, path2);
}
}

View File

@ -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 javax.annotation.Nullable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
/**
* An audit service which returns the {@link NoopSpan}.
* Even though the spans are no-ops, each span is still
* created with a unique span ID.
*/
public class NoopAuditor extends AbstractOperationAuditor {
/**
* unbonded span created in constructor.
*/
private final AuditSpanS3A unbondedSpan;
/**
* Activation callbacks.
*/
private final NoopSpan.SpanActivationCallbacks activationCallbacks;
/**
* Constructor.
* This will be used when the auditor is created through
* configuration and classloading.
*/
public NoopAuditor() {
this(null);
}
/**
* Constructor when explicitly created within
* the {@link NoopAuditManagerS3A}.
* @param activationCallbacks Activation callbacks.
*/
public NoopAuditor(
NoopSpan.SpanActivationCallbacks activationCallbacks) {
super("NoopAuditor");
this.unbondedSpan = createSpan("unbonded", null, null);
this.activationCallbacks = activationCallbacks;
}
@Override
public AuditSpanS3A createSpan(
final String operation,
@Nullable final String path1,
@Nullable final String path2) {
return new NoopSpan(createSpanID(), operation, path1, path2,
activationCallbacks);
}
@Override
public AuditSpanS3A getUnbondedSpan() {
return unbondedSpan;
}
/**
* Create, init and start an instance.
* @param conf configuration.
* @param activationCallbacks Activation callbacks.
* @return a started instance.
*/
public static NoopAuditor createAndStartNoopAuditor(Configuration conf,
NoopSpan.SpanActivationCallbacks activationCallbacks) {
NoopAuditor noop = new NoopAuditor(activationCallbacks);
final OperationAuditorOptions options =
OperationAuditorOptions.builder()
.withConfiguration(conf)
.withIoStatisticsStore(iostatisticsStore().build());
noop.init(options);
noop.start();
return noop;
}
}

View File

@ -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.s3a.audit.impl;
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
/**
* A minimal span with no direct side effects.
* It does have an ID and, if given callbacks,
* will notify the callback implementation
* of activation and deactivation.
* Subclassable for tests.
*/
public class NoopSpan extends AbstractAuditSpanImpl {
private final String path1;
private final String path2;
/** Activation callbacks. */
private final SpanActivationCallbacks activationCallbacks;
/**
* Static public instance.
*/
public static final NoopSpan INSTANCE = new NoopSpan();
/**
* Create a no-op span.
* @param spanId span ID
* @param operationName operation name
* @param path1 path
* @param path2 path 2
* @param activationCallbacks Activation callbacks.
*/
protected NoopSpan(String spanId,
final String operationName,
final String path1,
final String path2,
final SpanActivationCallbacks activationCallbacks) {
super(spanId, operationName);
this.path1 = path1;
this.path2 = path2;
this.activationCallbacks = activationCallbacks;
}
protected NoopSpan() {
this("", "no-op", null, null, null);
}
@Override
public AuditSpanS3A activate() {
if (activationCallbacks != null) {
activationCallbacks.activate(this);
}
return this;
}
@Override
public void deactivate() {
if (activationCallbacks != null) {
activationCallbacks.deactivate(this);
}
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("NoopSpan{");
sb.append("id='").append(getSpanId()).append('\'');
sb.append("name='").append(getOperationName()).append('\'');
sb.append(", path1='").append(path1).append('\'');
sb.append(", path2='").append(path2).append('\'');
sb.append('}');
return sb.toString();
}
/** Activation callbacks. */
public interface SpanActivationCallbacks {
/**
* Span was activated.
* @param span span reference.
*/
void activate(AuditSpanS3A span);
/**
* Span was deactivated.
* @param span span reference.
*/
void deactivate(AuditSpanS3A span);
}
}

View File

@ -0,0 +1,44 @@
/*
* 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 com.amazonaws.handlers.HandlerContextKey;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks;
/**
* Internal constants; not intended for public use, or
* for use by any external implementations.
*/
@InterfaceAudience.Private
public final class S3AInternalAuditConstants {
private S3AInternalAuditConstants() {
}
/**
* Handler key for audit span callbacks.
* This is used to bind the handler in the AWS code.
*/
public static final HandlerContextKey<AWSAuditEventCallbacks>
AUDIT_SPAN_HANDLER_CONTEXT =
new HandlerContextKey<>(
"org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks");
}

View File

@ -0,0 +1,28 @@
/*
* 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.
*/
/**
* The internal implementations of auditing support.
* This package is not for use by extensions.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.audit.impl;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -0,0 +1,41 @@
/*
* 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.
*/
/**
* Support for auditing and ultimately tracing operations.
* This is a public API for extension points, e.g. opentracing.
* However, it is very unstable as we evolve how best to audit/trace
* operation.
*
* An audit service is instantiated when an S3A Filesystem is initialized
* during creation.
* The choice of service is determined in the configuration option
* {@link org.apache.hadoop.fs.s3a.audit.S3AAuditConstants#AUDIT_SERVICE_CLASSNAME}.
* The service MUST implement the interface
* {@link org.apache.hadoop.fs.s3a.audit.OperationAuditor}
* to provide an {@link org.apache.hadoop.fs.store.audit.AuditSpan} whenever
* an operation is started through a public FileSystem API call
* (+some other operations).
*/
@InterfaceAudience.LimitedPrivate("S3A auditing extensions")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.audit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a.commit;
import javax.annotation.Nullable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.text.DateFormat;
@ -30,6 +31,7 @@
import java.util.concurrent.TimeUnit;
import com.amazonaws.services.s3.model.MultipartUpload;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -42,8 +44,11 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.audit.AuditConstants;
import org.apache.hadoop.fs.audit.CommonAuditContext;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.fs.store.audit.AuditSpanSource;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
@ -60,11 +65,13 @@
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.apache.hadoop.util.functional.InvocationRaisingIOE;
import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS;
import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB;
import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
@ -129,6 +136,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter
*/
private final JobUUIDSource uuidSource;
private final CommonAuditContext commonAuditContext;
/**
* Has this instance been used for job setup?
* If so then it is safe for a locally generated
@ -175,6 +184,11 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter
private final CommitterStatistics committerStatistics;
/**
* Source of Audit spans.
*/
private final AuditSpanSource auditSpanSource;
/**
* Create a committer.
* This constructor binds the destination directory and configuration, but
@ -203,6 +217,13 @@ protected AbstractS3ACommitter(
LOG.debug("{} instantiated for job \"{}\" ID {} with destination {}",
role, jobName(context), jobIdString(context), outputPath);
S3AFileSystem fs = getDestS3AFS();
// set this thread's context with the job ID.
// audit spans created in this thread will pick
// up this value.
this.commonAuditContext = currentAuditContext();
updateCommonContext();
// the filesystem is the span source, always.
auditSpanSource = fs.getAuditSpanSource();
this.createJobMarker = context.getConfiguration().getBoolean(
CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
@ -535,6 +556,8 @@ public void setupJob(JobContext context) throws IOException {
@Override
public void setupTask(TaskAttemptContext context) throws IOException {
TaskAttemptID attemptID = context.getTaskAttemptID();
updateCommonContext();
try (DurationInfo d = new DurationInfo(LOG, "Setup Task %s",
attemptID)) {
// reject attempts to set up the task where the output won't be
@ -947,11 +970,11 @@ public void cleanupJob(JobContext context) throws IOException {
protected void maybeIgnore(
boolean suppress,
String action,
Invoker.VoidOperation operation) throws IOException {
InvocationRaisingIOE operation) throws IOException {
if (suppress) {
ignoreIOExceptions(LOG, action, "", operation);
} else {
operation.execute();
operation.apply();
}
}
@ -1361,6 +1384,43 @@ public String toString() {
}
}
/**
* Add jobID to current context.
*/
protected final void updateCommonContext() {
currentAuditContext().put(AuditConstants.PARAM_JOB_ID, uuid);
}
/**
* Remove JobID from the current thread's context.
*/
protected final void resetCommonContext() {
currentAuditContext().remove(AuditConstants.PARAM_JOB_ID);
}
protected AuditSpanSource getAuditSpanSource() {
return auditSpanSource;
}
/**
* Start an operation; retrieve an audit span.
*
* All operation names <i>SHOULD</i> come from
* {@code StoreStatisticNames} or
* {@code StreamStatisticNames}.
* @param name operation name.
* @param path1 first path of operation
* @param path2 second path of operation
* @return a span for the audit
* @throws IOException failure
*/
protected AuditSpan startOperation(String name,
@Nullable String path1,
@Nullable String path2)
throws IOException {
return getAuditSpanSource().createSpan(name, path1, path2);
}
/**
* State of the active commit operation.
*

View File

@ -48,10 +48,11 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
import org.apache.hadoop.fs.s3a.WriteOperations;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation;
import org.apache.hadoop.fs.s3a.impl.HeaderProcessing;
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState;
@ -65,11 +66,13 @@
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB;
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE;
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator;
/**
* The implementation of the various actions a committer needs.
@ -81,7 +84,8 @@
* duplicate that work.
*
*/
public class CommitOperations implements IOStatisticsSource {
public class CommitOperations extends AbstractStoreOperation
implements IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(
CommitOperations.class);
@ -96,7 +100,7 @@ public class CommitOperations implements IOStatisticsSource {
/**
* Write operations for the destination fs.
*/
private final WriteOperationHelper writeOperations;
private final WriteOperations writeOperations;
/**
* Filter to find all {code .pendingset} files.
@ -113,21 +117,29 @@ public class CommitOperations implements IOStatisticsSource {
/**
* Instantiate.
* @param fs FS to bind to
* @throws IOException failure to bind.
*/
public CommitOperations(S3AFileSystem fs) {
public CommitOperations(S3AFileSystem fs) throws IOException {
this(requireNonNull(fs), fs.newCommitterStatistics());
}
/**
* Instantiate.
* Instantiate. This creates a new audit span for
* the commit operations.
* @param fs FS to bind to
* @param committerStatistics committer statistics
* @throws IOException failure to bind.
*/
public CommitOperations(S3AFileSystem fs,
CommitterStatistics committerStatistics) {
this.fs = requireNonNull(fs);
CommitterStatistics committerStatistics) throws IOException {
super(requireNonNull(fs).createStoreContext());
this.fs = fs;
statistics = requireNonNull(committerStatistics);
writeOperations = fs.getWriteOperationHelper();
// create a span
writeOperations = fs.createWriteOperationHelper(
fs.getAuditSpanSource().createSpan(
COMMITTER_COMMIT_JOB.getSymbol(),
"/", null));
}
/**
@ -365,6 +377,7 @@ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir,
}
}
}
cleanupRemoteIterator(pendingFiles);
return outcome;
}
@ -388,7 +401,7 @@ protected RemoteIterator<LocatedFileStatus> ls(Path path, boolean recursive)
*/
public List<MultipartUpload> listPendingUploadsUnderPath(Path dest)
throws IOException {
return fs.listMultipartUploads(fs.pathToKey(dest));
return writeOperations.listMultipartUploads(fs.pathToKey(dest));
}
/**

View File

@ -125,5 +125,29 @@ public static void validateCollectionClass(Iterable it, Class classname)
}
}
/**
* Extract the job ID from a configuration.
* @param conf configuration
* @return a job ID or null.
*/
public static String extractJobID(Configuration conf) {
String jobUUID = conf.getTrimmed(FS_S3A_COMMITTER_UUID, "");
if (!jobUUID.isEmpty()) {
return jobUUID;
}
// there is no job UUID.
// look for one from spark
jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, "");
if (!jobUUID.isEmpty()) {
return jobUUID;
}
jobUUID = conf.getTrimmed(MR_JOB_ID, "");
if (!jobUUID.isEmpty()) {
return jobUUID;
}
return null;
}
}

View File

@ -124,4 +124,10 @@ private InternalCommitterConstants() {
"Job/task context does not contain a unique ID in "
+ SPARK_WRITE_UUID;
/**
* The MR job ID; copies from MRJobConfig so that it can be
* referred to without needing hadoop-mapreduce on the classpath.
*/
public static final String MR_JOB_ID = "mapreduce.job.id";
}

View File

@ -27,7 +27,7 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation;
import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*;
@ -45,14 +45,12 @@
* <p>Important</p>: must not directly or indirectly import a class which
* uses any datatype in hadoop-mapreduce.
*/
public class MagicCommitIntegration {
public class MagicCommitIntegration extends AbstractStoreOperation {
private static final Logger LOG =
LoggerFactory.getLogger(MagicCommitIntegration.class);
private final S3AFileSystem owner;
private final boolean magicCommitEnabled;
private final StoreContext storeContext;
/**
* Instantiate.
* @param owner owner class
@ -60,9 +58,9 @@ public class MagicCommitIntegration {
*/
public MagicCommitIntegration(S3AFileSystem owner,
boolean magicCommitEnabled) {
super(owner.createStoreContext());
this.owner = owner;
this.magicCommitEnabled = magicCommitEnabled;
this.storeContext = owner.createStoreContext();
}
/**
@ -85,6 +83,9 @@ public String keyOfFinalDestination(List<String> elements, String key) {
* Given a path and a key to that same path, create a tracker for it.
* This specific tracker will be chosen based on whether or not
* the path is a magic one.
* Auditing: the span used to invoke
* this method will be the one used to create the write operation helper
* for the commit tracker.
* @param path path of nominal write
* @param key key of path of nominal write
* @return the tracker for this operation.
@ -98,10 +99,10 @@ public PutTracker createTracker(Path path, String key) {
if (isMagicCommitPath(elements)) {
final String destKey = keyOfFinalDestination(elements, key);
String pendingsetPath = key + CommitConstants.PENDING_SUFFIX;
storeContext.incrementStatistic(
getStoreContext().incrementStatistic(
Statistic.COMMITTER_MAGIC_FILES_CREATED);
tracker = new MagicCommitTracker(path,
storeContext.getBucket(),
getStoreContext().getBucket(),
key,
destKey,
pendingsetPath,

View File

@ -65,7 +65,7 @@ public class MagicCommitTracker extends PutTracker {
* @param originalDestKey the original key, in the magic directory.
* @param destKey key for the destination
* @param pendingsetKey key of the pendingset file
* @param writer writer instance to use for operations
* @param writer writer instance to use for operations; includes audit span
*/
public MagicCommitTracker(Path path,
String bucket,

View File

@ -56,6 +56,7 @@
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*;
import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*;
import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator;
/**
* Committer based on the contributed work of the
@ -828,6 +829,7 @@ protected PathExistsException failDestinationExists(final Path path,
? " dir"
: ("file size " + status.getLen() + " bytes"));
}
cleanupRemoteIterator(lf);
} catch (IOException e) {
LOG.info("Discarding exception raised when listing {}: " + e, path);
LOG.debug("stack trace ", e);

View File

@ -18,24 +18,46 @@
package org.apache.hadoop.fs.s3a.impl;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
/**
* Base class of operations in the store.
* An operation is something which executes against the context to
* perform a single function.
* It is expected to have a limited lifespan.
*/
public abstract class AbstractStoreOperation {
/**
* Store context.
*/
private final StoreContext storeContext;
/**
* constructor.
* Audit Span.
*/
private AuditSpan auditSpan;
/**
* Constructor.
* Picks up the active audit span from the store context and
* stores it for later.
* @param storeContext store context.
*/
protected AbstractStoreOperation(final StoreContext storeContext) {
this(storeContext, storeContext.getActiveAuditSpan());
}
/**
* Constructor.
* @param storeContext store context.
* @param auditSpan active span
*/
protected AbstractStoreOperation(final StoreContext storeContext,
final AuditSpan auditSpan) {
this.storeContext = checkNotNull(storeContext);
this.auditSpan = checkNotNull(auditSpan);
}
/**
@ -46,4 +68,18 @@ public final StoreContext getStoreContext() {
return storeContext;
}
/**
* Get the audit span this object was created with.
* @return the current span
*/
public AuditSpan getAuditSpan() {
return auditSpan;
}
/**
* Activate the audit span.
*/
public void activateAuditSpan() {
auditSpan.activate();
}
}

View File

@ -32,6 +32,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
@ -41,24 +42,48 @@
* raised by the callable and wrapping them as appropriate.
* @param <T> return type.
*/
public final class CallableSupplier<T> implements Supplier {
public final class CallableSupplier<T> implements Supplier<T> {
private static final Logger LOG =
LoggerFactory.getLogger(CallableSupplier.class);
private final Callable<T> call;
/**
* Audit Span; may be null.
*/
private final AuditSpan auditSpan;
/**
* Create.
* @param call call to invoke.
*/
public CallableSupplier(final Callable<T> call) {
this.call = call;
this(null, call);
}
/**
* Create.
* @param auditSpan audit span (or null)
* @param call call to invoke.
*/
public CallableSupplier(
final AuditSpan auditSpan,
final Callable<T> call) {
this.call = call;
this.auditSpan = auditSpan;
}
/**
* Active any span and then call the supplied callable.
* @return the result.
*/
@Override
public Object get() {
public T get() {
try {
if (auditSpan != null) {
auditSpan.activate();
}
return call.call();
} catch (RuntimeException e) {
throw e;
@ -86,11 +111,32 @@ public static <T> CompletableFuture<T> submit(
return CompletableFuture.supplyAsync(
new CallableSupplier<T>(call), executor);
}
/**
* Submit a callable into a completable future.
* RTEs are rethrown.
* Non RTEs are caught and wrapped; IOExceptions to
* {@code RuntimeIOException} instances.
* @param executor executor.
* @param auditSpan audit span (or null)
* @param call call to invoke
* @param <T> type
* @return the future to wait for
*/
@SuppressWarnings("unchecked")
public static <T> CompletableFuture<T> submit(
final Executor executor,
final AuditSpan auditSpan,
final Callable<T> call) {
return CompletableFuture.supplyAsync(
new CallableSupplier<T>(auditSpan, call),
executor);
}
/**
* Wait for a list of futures to complete. If the list is empty,
* return immediately.
* @param futures list of futures.
* @param <T> type
* @throws IOException if one of the called futures raised an IOE.
* @throws RuntimeException if one of the futures raised one.
*/
@ -108,6 +154,7 @@ public static <T> void waitForCompletion(
/**
* Wait for a single of future to complete, extracting IOEs afterwards.
* @param future future to wait for.
* @param <T> type
* @throws IOException if one of the called futures raised an IOE.
* @throws RuntimeException if one of the futures raised one.
*/
@ -127,6 +174,7 @@ public static <T> void waitForCompletion(
/**
* Wait for a single of future to complete, ignoring exceptions raised.
* @param future future to wait for.
* @param <T> type
*/
public static <T> void waitForCompletionIgnoringExceptions(
@Nullable final CompletableFuture<T> future) {

View File

@ -35,6 +35,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.RemoteFileChangedException;
import org.apache.hadoop.fs.store.LogExactlyOnce;
import static org.apache.hadoop.fs.s3a.Constants.*;

View File

@ -22,10 +22,10 @@
import java.io.IOException;
import java.nio.file.AccessDeniedException;
import com.amazonaws.services.s3.model.ObjectMetadata;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.api.RequestFactory;
import org.apache.hadoop.fs.store.audit.AuditSpan;
/**
* An interface to implement for providing accessors to
@ -85,13 +85,16 @@ public interface ContextAccessors {
Path makeQualified(Path path);
/**
* Retrieve the object metadata.
*
* @param key key to retrieve.
* @return metadata
* @throws IOException IO and object access problems.
* Return the active audit span.
* This is thread local -it MUST be picked up and passed into workers.
* Collect and cache the value during construction.
* @return active audit span.
*/
@Retries.RetryTranslated
ObjectMetadata getObjectMetadata(String key) throws IOException;
AuditSpan getActiveAuditSpan();
/**
* Get the request factory.
* @return the factory for requests.
*/
RequestFactory getRequestFactory();
}

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.maybeAwaitCompletion;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
@ -504,13 +505,15 @@ private CompletableFuture<Void> submitDelete(
return null;
}
filesDeleted += keyList.size();
return submit(executor, () -> {
asyncDeleteAction(operationState,
keyList,
pathList,
LOG.isDebugEnabled());
return null;
});
return submit(executor,
callableWithinAuditSpan(
getAuditSpan(), () -> {
asyncDeleteAction(operationState,
keyList,
pathList,
LOG.isDebugEnabled());
return null;
}));
}
/**

View File

@ -23,14 +23,21 @@
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.util.functional.CallableRaisingIOE;
/**
* A subclass of {@link AbstractStoreOperation} which
* provides a method {@link #execute()} that may be invoked
* exactly once.
* It declares itself a {@code CallableRaisingIOE} and
* can be handed straight to methods which take those
* as parameters.
* @param <T> return type of executed operation.
*/
public abstract class ExecutingStoreOperation<T>
extends AbstractStoreOperation {
extends AbstractStoreOperation
implements CallableRaisingIOE<T> {
/**
* Used to stop any re-entrancy of the rename.
@ -39,11 +46,34 @@ public abstract class ExecutingStoreOperation<T>
private final AtomicBoolean executed = new AtomicBoolean(false);
/**
* constructor.
* Constructor.
* Picks up the active audit span from the store context and
* stores it for later.
* @param storeContext store context.
*/
protected ExecutingStoreOperation(final StoreContext storeContext) {
super(storeContext);
this(storeContext, storeContext.getActiveAuditSpan());
}
/**
* Constructor.
* @param storeContext store context.
* @param auditSpan active span
*/
protected ExecutingStoreOperation(
final StoreContext storeContext,
final AuditSpan auditSpan) {
super(storeContext, auditSpan);
}
/**
* Apply calls {@link #execute()}.
* @return the result.
* @throws IOException IO problem
*/
@Override
public final T apply() throws IOException {
return execute();
}
/**
@ -53,17 +83,19 @@ protected ExecutingStoreOperation(final StoreContext storeContext) {
* @return the result.
* @throws IOException IO problem
*/
public abstract T execute() throws IOException ;
public abstract T execute() throws IOException;
/**
* Check that the operation has not been invoked twice.
* This is an atomic check.
* After the check: activates the span.
* @throws IllegalStateException on a second invocation.
*/
protected void executeOnlyOnce() {
Preconditions.checkState(
!executed.getAndSet(true),
"Operation attempted twice");
activateAuditSpan();
}
}

View File

@ -0,0 +1,208 @@
/*
* 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.impl;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
/**
* GetContentSummary operation.
* This is based on {@code FileSystem.get#getContentSummary};
* its still doing sequential treewalk with the efficiency
* issues.
*
* Changes:
* 1. On the recursive calls there
* is no probe to see if the path is a file: we know the
* recursion only happens with a dir.
* 2. If a subdirectory is not found during the walk, that
* does not trigger an error. The directory is clearly
* not part of the content any more.
*
* The Operation serves up IOStatistics; this counts
* the cost of all the list operations, but not the
* initial HEAD probe to see if the path is a file.
*/
public class GetContentSummaryOperation extends
ExecutingStoreOperation<ContentSummary> implements IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(
GetContentSummaryOperation.class);
/**
* Directory to scan.
*/
private final Path path;
/**
* Callbacks to the store.
*/
private final GetContentSummaryCallbacks callbacks;
/**
* IOStatistics to serve up.
*/
private final IOStatisticsSnapshot iostatistics =
new IOStatisticsSnapshot();
/**
* Constructor.
* @param storeContext context.
* @param path path to summarize
* @param callbacks callbacks for S3 access.
*/
public GetContentSummaryOperation(
final StoreContext storeContext,
final Path path,
final GetContentSummaryCallbacks callbacks) {
super(storeContext);
this.path = path;
this.callbacks = callbacks;
}
@Override
public IOStatistics getIOStatistics() {
return iostatistics;
}
/**
* Return the {@link ContentSummary} of a given path.
* @return the summary.
* @throws FileNotFoundException if the path does not resolve
* @throws IOException failure
*/
@Override
@Retries.RetryTranslated
public ContentSummary execute() throws IOException {
FileStatus status = probePathStatusOrNull(path, StatusProbeEnum.FILE);
if (status != null && status.isFile()) {
// f is a file
long length = status.getLen();
return new ContentSummary.Builder().length(length).
fileCount(1).directoryCount(0).spaceConsumed(length).build();
}
final ContentSummary summary = getDirSummary(path);
// Log the IOStatistics at debug so the cost of the operation
// can be made visible.
LOG.debug("IOStatistics of getContentSummary({}):\n{}", path, iostatistics);
return summary;
}
/**
* Return the {@link ContentSummary} of a given directory.
* This is a recursive operation (as the original is);
* it'd be more efficient of stack and heap if it managed its
* own stack.
* @param dir dir to scan
* @throws FileNotFoundException if the path does not resolve
* @throws IOException IO failure
* @return the content summary
* @throws FileNotFoundException the path does not exist
* @throws IOException failure
*/
public ContentSummary getDirSummary(Path dir) throws IOException {
long totalLength = 0;
long fileCount = 0;
long dirCount = 1;
final RemoteIterator<S3AFileStatus> it
= callbacks.listStatusIterator(dir);
while (it.hasNext()) {
final S3AFileStatus s = it.next();
if (s.isDirectory()) {
try {
ContentSummary c = getDirSummary(s.getPath());
totalLength += c.getLength();
fileCount += c.getFileCount();
dirCount += c.getDirectoryCount();
} catch (FileNotFoundException ignored) {
// path was deleted during the scan; exclude from
// summary.
}
} else {
totalLength += s.getLen();
fileCount += 1;
}
}
// Add the list's IOStatistics
iostatistics.aggregate(retrieveIOStatistics(it));
return new ContentSummary.Builder().length(totalLength).
fileCount(fileCount).directoryCount(dirCount).
spaceConsumed(totalLength).build();
}
/**
* Get the status of a path, downgrading FNFE to null result.
* @param p path to probe.
* @param probes probes to exec
* @return the status or null
* @throws IOException failure other than FileNotFound
*/
private S3AFileStatus probePathStatusOrNull(final Path p,
final Set<StatusProbeEnum> probes) throws IOException {
try {
return callbacks.probePathStatus(p, probes);
} catch (FileNotFoundException fnfe) {
return null;
}
}
/**
* Callbacks used by the operation.
*/
public interface GetContentSummaryCallbacks {
/**
* Get the status of a path.
* @param path path to probe.
* @param probes probes to exec
* @return the status
* @throws IOException failure
*/
@Retries.RetryTranslated
S3AFileStatus probePathStatus(Path path,
Set<StatusProbeEnum> probes) throws IOException;
/**
* Incremental list of all entries in a directory.
* @param path path of dir
* @return an iterator
* @throws IOException failure
*/
RemoteIterator<S3AFileStatus> listStatusIterator(Path path)
throws IOException;
}
}

View File

@ -35,6 +35,7 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
@ -188,6 +189,15 @@ public class HeaderProcessing extends AbstractStoreOperation {
public static final String XA_STORAGE_CLASS =
XA_HEADER_PREFIX + Headers.STORAGE_CLASS;
/**
* HTTP Referrer for logs: {@value}.
* This can be found in S3 logs, but is not set as
* an attribute in objects.
* <i>important: </i> the header value is deliberately
* a mis-spelling, as that is defined in RFC-1945.
*/
public static final String HEADER_REFERRER = "Referer";
/**
* Standard headers which are retrieved from HEAD Requests
* and set as XAttrs if the response included the relevant header.
@ -230,12 +240,23 @@ public class HeaderProcessing extends AbstractStoreOperation {
public static final String CONTENT_TYPE_APPLICATION_XML =
"application/xml";
/**
* Directory content type : {@value}.
* Matches use/expectations of AWS S3 console.
*/
public static final String CONTENT_TYPE_X_DIRECTORY =
"application/x-directory";
private final HeaderProcessingCallbacks callbacks;
/**
* Construct.
* @param storeContext store context.
* @param callbacks callbacks to the store
*/
public HeaderProcessing(final StoreContext storeContext) {
public HeaderProcessing(final StoreContext storeContext,
final HeaderProcessingCallbacks callbacks) {
super(storeContext);
this.callbacks = callbacks;
}
/**
@ -253,18 +274,17 @@ private Map<String, byte[]> retrieveHeaders(
final Path path,
final Statistic statistic) throws IOException {
StoreContext context = getStoreContext();
ContextAccessors accessors = context.getContextAccessors();
String objectKey = accessors.pathToKey(path);
String objectKey = context.pathToKey(path);
ObjectMetadata md;
String symbol = statistic.getSymbol();
S3AStatisticsContext instrumentation = context.getInstrumentation();
try {
md = trackDuration(instrumentation, symbol, () ->
accessors.getObjectMetadata(objectKey));
callbacks.getObjectMetadata(objectKey));
} catch (FileNotFoundException e) {
// no entry. It could be a directory, so try again.
md = trackDuration(instrumentation, symbol, () ->
accessors.getObjectMetadata(objectKey + "/"));
callbacks.getObjectMetadata(objectKey + "/"));
}
// all user metadata
Map<String, String> rawHeaders = md.getUserMetadata();
@ -443,7 +463,7 @@ public static Optional<Long> extractXAttrLongValue(byte[] data) {
* @param source the {@link ObjectMetadata} to copy
* @param dest the metadata to update; this is the return value.
*/
public void cloneObjectMetadata(ObjectMetadata source,
public static void cloneObjectMetadata(ObjectMetadata source,
ObjectMetadata dest) {
// Possibly null attributes
@ -497,4 +517,16 @@ public void cloneObjectMetadata(ObjectMetadata source,
.forEach(e -> dest.addUserMetadata(e.getKey(), e.getValue()));
}
public interface HeaderProcessingCallbacks {
/**
* Retrieve the object metadata.
*
* @param key key to retrieve.
* @return metadata
* @throws IOException IO and object access problems.
*/
@Retries.RetryTranslated
ObjectMetadata getObjectMetadata(String key) throws IOException;
}
}

View File

@ -37,6 +37,16 @@
*/
public final class InternalConstants {
/**
* 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 InternalConstants() {
}

View File

@ -30,6 +30,7 @@
import org.apache.hadoop.fs.s3a.S3ListResult;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
import org.apache.hadoop.fs.store.audit.AuditSpan;
/**
* These are all the callbacks which
@ -46,13 +47,15 @@ public interface ListingOperationCallbacks {
* Retry policy: retry untranslated.
* @param request request to initiate
* @param trackerFactory tracker with statistics to update
* @param span audit span for this operation
* @return the results
* @throws IOException if the retry invocation raises one (it shouldn't).
*/
@Retries.RetryRaw
CompletableFuture<S3ListResult> listObjectsAsync(
S3ListRequest request,
DurationTrackerFactory trackerFactory)
S3ListRequest request,
DurationTrackerFactory trackerFactory,
AuditSpan span)
throws IOException;
/**
@ -61,14 +64,16 @@ CompletableFuture<S3ListResult> listObjectsAsync(
* @param request last list objects request to continue
* @param prevResult last paged result to continue from
* @param trackerFactory tracker with statistics to update
* @param span audit span for the IO
* @return the next result object
* @throws IOException none, just there for retryUntranslated.
*/
@Retries.RetryRaw
CompletableFuture<S3ListResult> continueListObjectsAsync(
S3ListRequest request,
S3ListResult prevResult,
DurationTrackerFactory trackerFactory)
S3ListRequest request,
S3ListResult prevResult,
DurationTrackerFactory trackerFactory,
AuditSpan span)
throws IOException;
/**
@ -82,16 +87,19 @@ S3ALocatedFileStatus toLocatedFileStatus(
throws IOException;
/**
* Create a {@code ListObjectsRequest} request against this bucket,
* with the maximum keys returned in a query set by
* with the maximum keys returned in a query set in the FS config.
* The active span for the FS is handed the request to prepare it
* before this method returns.
* {@link #getMaxKeys()}.
* @param key key for request
* @param delimiter any delimiter
* @param span span within which the request takes place.
* @return the request
*/
S3ListRequest createListObjectsRequest(
String key,
String delimiter);
String key,
String delimiter,
AuditSpan span);
/**
* Return the number of bytes that large input files should be optimally

View File

@ -0,0 +1,184 @@
/*
* 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.impl;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.file.AccessDeniedException;
import java.util.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
/**
* The mkdir operation.
* A walk up the ancestor list halting as soon as a directory (good)
* or file (bad) is found.
* Optimized with the expectation that there is a marker up the path
* or (ultimately) a sibling of the path being created.
* It performs the directory listing probe ahead of the simple object HEAD
* call for this reason -the object is the failure mode which SHOULD NOT
* be encountered on normal execution.
*/
public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
private static final Logger LOG = LoggerFactory.getLogger(
MkdirOperation.class);
private final Path dir;
private final MkdirCallbacks callbacks;
public MkdirOperation(
final StoreContext storeContext,
final Path dir,
final MkdirCallbacks callbacks) {
super(storeContext);
this.dir = dir;
this.callbacks = callbacks;
}
/**
*
* Make the given path and all non-existent parents into
* directories.
* @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
*/
@Override
@Retries.RetryTranslated
public Boolean execute() throws IOException {
LOG.debug("Making directory: {}", dir);
if (dir.isRoot()) {
// fast exit for root.
return true;
}
FileStatus fileStatus = getPathStatusExpectingDir(dir);
if (fileStatus != null) {
if (fileStatus.isDirectory()) {
return true;
} else {
throw new FileAlreadyExistsException("Path is a file: " + dir);
}
}
// dir, walk up tree
// Walk path to root, ensuring closest ancestor is a directory, not file
Path fPart = dir.getParent();
try {
while (fPart != null && !fPart.isRoot()) {
fileStatus = getPathStatusExpectingDir(fPart);
if (fileStatus == null) {
// nothing at this path, so validate the parent
fPart = fPart.getParent();
continue;
}
if (fileStatus.isDirectory()) {
// the parent dir exists. All is good.
break;
}
// there's a file at the parent entry
throw new FileAlreadyExistsException(String.format(
"Can't make directory for path '%s' since it is a file.",
fPart));
}
} catch (AccessDeniedException e) {
LOG.info("mkdirs({}}: Access denied when looking"
+ " for parent directory {}; skipping checks",
dir, fPart);
LOG.debug("{}", e.toString(), e);
}
// if we get here there is no directory at the destination.
// so create one.
String key = getStoreContext().pathToKey(dir);
// this will create the marker file, delete the parent entries
// and update S3Guard
callbacks.createFakeDirectory(key);
return true;
}
/**
* Get the status of a path, downgrading FNFE to null result.
* @param path path to probe.
* @param probes probes to exec
* @return the status or null
* @throws IOException failure other than FileNotFound
*/
private S3AFileStatus probePathStatusOrNull(final Path path,
final Set<StatusProbeEnum> probes) throws IOException {
try {
return callbacks.probePathStatus(path, probes);
} catch (FileNotFoundException fnfe) {
return null;
}
}
/**
* Get the status of a path -optimized for paths
* where there is a directory marker or child entries.
* @param path path to probe.
* @return the status
* @throws IOException failure
*/
private S3AFileStatus getPathStatusExpectingDir(final Path path)
throws IOException {
S3AFileStatus status = probePathStatusOrNull(path,
StatusProbeEnum.DIRECTORIES);
if (status == null) {
status = probePathStatusOrNull(path,
StatusProbeEnum.FILE);
}
return status;
}
/**
* Callbacks used by mkdir.
*/
public interface MkdirCallbacks {
/**
* Get the status of a path.
* @param path path to probe.
* @param probes probes to exec
* @return the status
* @throws IOException failure
*/
@Retries.RetryTranslated
S3AFileStatus probePathStatus(Path path,
Set<StatusProbeEnum> probes) throws IOException;
/**
* Create a fake directory, always ending in "/".
* Retry policy: retrying; translated.
* @param key name of directory object.
* @throws IOException IO failure
*/
@Retries.RetryTranslated
void createFakeDirectory(String key) throws IOException;
}
}

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.OperationDuration;
import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_BLOCK_SIZE;
import static org.apache.hadoop.fs.s3a.S3AUtils.objectRepresentsDirectory;
@ -381,7 +382,7 @@ protected Path renameFileToDest() throws IOException {
* Execute a full recursive rename.
* There is a special handling of directly markers here -only leaf markers
* are copied. This reduces incompatibility "regions" across versions.
Are * @throws IOException failure
* @throws IOException failure
*/
protected void recursiveDirectoryRename() throws IOException {
final StoreContext storeContext = getStoreContext();
@ -596,15 +597,16 @@ protected CompletableFuture<Path> initiateCopy(
source.getVersionId(),
source.getLen());
// queue the copy operation for execution in the thread pool
return submit(getStoreContext().getExecutor(), () ->
copySourceAndUpdateTracker(
childSourcePath,
key,
sourceAttributes,
callbacks.createReadContext(source),
childDestPath,
newDestKey,
true));
return submit(getStoreContext().getExecutor(),
callableWithinAuditSpan(getAuditSpan(), () ->
copySourceAndUpdateTracker(
childSourcePath,
key,
sourceAttributes,
callbacks.createReadContext(source),
childDestPath,
newDestKey,
true)));
}
/**

View File

@ -0,0 +1,695 @@
/*
* 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.impl;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import com.amazonaws.AmazonWebServiceRequest;
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.thirdparty.com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.api.RequestFactory;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecretOperations;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull;
/**
* The standard implementation of the request factory.
* This creates AWS SDK request classes for the specific bucket,
* with standard options/headers set.
* It is also where custom setting parameters can take place.
*
* All creation of AWS S3 requests MUST be through this class so that
* common options (encryption etc.) can be added here,
* and so that any chained transformation of requests can be applied.
*
* This is where audit span information is added to the requests,
* until it is done in the AWS SDK itself.
*
* All created requests will be passed through
* {@link PrepareRequest#prepareRequest(AmazonWebServiceRequest)} before
* being returned to the caller.
*/
public class RequestFactoryImpl implements RequestFactory {
public static final Logger LOG = LoggerFactory.getLogger(
RequestFactoryImpl.class);
/**
* Target bucket.
*/
private final String bucket;
/**
* Encryption secrets.
*/
private EncryptionSecrets encryptionSecrets;
/**
* ACL For new objects.
*/
private final CannedAccessControlList cannedACL;
/**
* Max number of multipart entries allowed in a large
* upload. Tunable for testing only.
*/
private final long multipartPartCountLimit;
/**
* Requester Pays.
* This is to be wired up in a PR with its
* own tests and docs.
*/
private final boolean requesterPays;
/**
* Callback to prepare requests.
*/
private final PrepareRequest requestPreparer;
/**
* Constructor.
* @param builder builder with all the configuration.
*/
protected RequestFactoryImpl(
final RequestFactoryBuilder builder) {
this.bucket = builder.bucket;
this.cannedACL = builder.cannedACL;
this.encryptionSecrets = builder.encryptionSecrets;
this.multipartPartCountLimit = builder.multipartPartCountLimit;
this.requesterPays = builder.requesterPays;
this.requestPreparer = builder.requestPreparer;
}
/**
* Preflight preparation of AWS request.
* @param <T> web service request
* @return prepared entry.
*/
@Retries.OnceRaw
private <T extends AmazonWebServiceRequest> T prepareRequest(T t) {
return requestPreparer != null
? requestPreparer.prepareRequest(t)
: t;
}
/**
* Get the canned ACL of this FS.
* @return an ACL, if any
*/
@Override
public CannedAccessControlList getCannedACL() {
return cannedACL;
}
/**
* Get the target bucket.
* @return the bucket.
*/
protected String getBucket() {
return bucket;
}
/**
* 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
*/
@Override
public 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.
*/
@Override
public Optional<SSECustomerKey> generateSSECustomerKey() {
return EncryptionSecretOperations.createSSECustomerKey(
encryptionSecrets);
}
/**
* Get the encryption algorithm of this endpoint.
* @return the encryption algorithm.
*/
@Override
public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
return encryptionSecrets.getEncryptionMethod();
}
/**
* 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);
}
/**
* Sets server side encryption parameters to the GET reuquest.
* request when encryption is enabled.
* @param request upload part request
*/
protected void setOptionalGetObjectMetadataParameters(
GetObjectMetadataRequest request) {
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
}
/**
* 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);
}
/**
* Set the optional parameters for a PUT request.
* @param request request to patch.
*/
protected void setOptionalPutRequestParameters(PutObjectRequest request) {
generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
}
/**
* Set the optional metadata for an object being created or copied.
* @param metadata to update.
*/
protected void setOptionalObjectMetadata(ObjectMetadata metadata) {
final S3AEncryptionMethods algorithm
= getServerSideEncryptionAlgorithm();
if (S3AEncryptionMethods.SSE_S3 == algorithm) {
metadata.setSSEAlgorithm(algorithm.getMethod());
}
}
/**
* 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
*/
@Override
public ObjectMetadata newObjectMetadata(long length) {
final ObjectMetadata om = new ObjectMetadata();
setOptionalObjectMetadata(om);
if (length >= 0) {
om.setContentLength(length);
}
return om;
}
@Override
public CopyObjectRequest newCopyObjectRequest(String srcKey,
String dstKey,
ObjectMetadata srcom) {
CopyObjectRequest copyObjectRequest =
new CopyObjectRequest(getBucket(), srcKey, getBucket(), dstKey);
ObjectMetadata dstom = newObjectMetadata(srcom.getContentLength());
HeaderProcessing.cloneObjectMetadata(srcom, dstom);
setOptionalObjectMetadata(dstom);
copyEncryptionParameters(srcom, copyObjectRequest);
copyObjectRequest.setCannedAccessControlList(cannedACL);
copyObjectRequest.setNewObjectMetadata(dstom);
Optional.ofNullable(srcom.getStorageClass())
.ifPresent(copyObjectRequest::setStorageClass);
return prepareRequest(copyObjectRequest);
}
/**
* Propagate encryption parameters from source file if set else use the
* current filesystem encryption settings.
* @param srcom source object metadata.
* @param copyObjectRequest copy object request body.
*/
protected void copyEncryptionParameters(
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()) {
case SSE_S3:
/* no-op; this is set in destination object metadata */
break;
case SSE_C:
generateSSECustomerKey().ifPresent(customerKey -> {
copyObjectRequest.setSourceSSECustomerKey(customerKey);
copyObjectRequest.setDestinationSSECustomerKey(customerKey);
});
break;
case SSE_KMS:
generateSSEAwsKeyParams().ifPresent(
copyObjectRequest::setSSEAwsKeyManagementParams);
break;
default:
}
}
/**
* 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
*/
@Override
public PutObjectRequest newPutObjectRequest(String key,
ObjectMetadata metadata, File srcfile) {
Preconditions.checkNotNull(srcfile);
PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key,
srcfile);
setOptionalPutRequestParameters(putObjectRequest);
putObjectRequest.setCannedAcl(cannedACL);
putObjectRequest.setMetadata(metadata);
return prepareRequest(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
*/
@Override
public PutObjectRequest newPutObjectRequest(String key,
ObjectMetadata metadata,
InputStream inputStream) {
Preconditions.checkNotNull(inputStream);
Preconditions.checkArgument(isNotEmpty(key), "Null/empty key");
PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key,
inputStream, metadata);
setOptionalPutRequestParameters(putObjectRequest);
putObjectRequest.setCannedAcl(cannedACL);
return prepareRequest(putObjectRequest);
}
@Override
public PutObjectRequest newDirectoryMarkerRequest(String directory) {
String key = directory.endsWith("/")
? directory
: (directory + "/");
// an input stream which is laways empty
final InputStream im = new InputStream() {
@Override
public int read() throws IOException {
return -1;
}
};
// preparation happens in here
final ObjectMetadata md = newObjectMetadata(0L);
md.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY);
PutObjectRequest putObjectRequest =
newPutObjectRequest(key, md, im);
return putObjectRequest;
}
@Override
public ListMultipartUploadsRequest
newListMultipartUploadsRequest(String prefix) {
ListMultipartUploadsRequest request = new ListMultipartUploadsRequest(
getBucket());
if (prefix != null) {
request.setPrefix(prefix);
}
return prepareRequest(request);
}
@Override
public AbortMultipartUploadRequest newAbortMultipartUploadRequest(
String destKey,
String uploadId) {
return prepareRequest(new AbortMultipartUploadRequest(getBucket(),
destKey,
uploadId));
}
@Override
public InitiateMultipartUploadRequest newMultipartUploadRequest(
String destKey) {
final InitiateMultipartUploadRequest initiateMPURequest =
new InitiateMultipartUploadRequest(getBucket(),
destKey,
newObjectMetadata(-1));
initiateMPURequest.setCannedACL(getCannedACL());
setOptionalMultipartUploadRequestParameters(initiateMPURequest);
return prepareRequest(initiateMPURequest);
}
@Override
public CompleteMultipartUploadRequest newCompleteMultipartUploadRequest(
String destKey,
String uploadId,
List<PartETag> partETags) {
// a copy of the list is required, so that the AWS SDK doesn't
// attempt to sort an unmodifiable list.
return prepareRequest(new CompleteMultipartUploadRequest(bucket,
destKey, uploadId, new ArrayList<>(partETags)));
}
@Override
public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) {
GetObjectMetadataRequest request =
new GetObjectMetadataRequest(getBucket(), key);
//SSE-C requires to be filled in if enabled for object metadata
setOptionalGetObjectMetadataParameters(request);
return prepareRequest(request);
}
@Override
public GetObjectRequest newGetObjectRequest(String key) {
GetObjectRequest request = new GetObjectRequest(bucket, key);
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
return prepareRequest(request);
}
@Override
public UploadPartRequest newUploadPartRequest(
String destKey,
String uploadId,
int partNumber,
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);
final String pathErrorMsg = "Number of parts in multipart upload exceeded."
+ " Current part count = %s, Part count limit = %s ";
if (partNumber > multipartPartCountLimit) {
throw new PathIOException(destKey,
String.format(pathErrorMsg, partNumber, multipartPartCountLimit));
}
UploadPartRequest request = new UploadPartRequest()
.withBucketName(getBucket())
.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(sourceFile.isFile(),
"Source is not a file: %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);
}
setOptionalUploadPartRequestParameters(request);
return prepareRequest(request);
}
@Override
public SelectObjectContentRequest newSelectRequest(String key) {
SelectObjectContentRequest request = new SelectObjectContentRequest();
request.setBucketName(bucket);
request.setKey(key);
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
return prepareRequest(request);
}
@Override
public ListObjectsRequest newListObjectsV1Request(
final String key,
final String delimiter,
final int maxKeys) {
ListObjectsRequest request = new ListObjectsRequest()
.withBucketName(bucket)
.withMaxKeys(maxKeys)
.withPrefix(key);
if (delimiter != null) {
request.setDelimiter(delimiter);
}
return prepareRequest(request);
}
@Override
public ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest(
ObjectListing prev) {
return prepareRequest(new ListNextBatchOfObjectsRequest(prev));
}
@Override
public ListObjectsV2Request newListObjectsV2Request(
final String key,
final String delimiter,
final int maxKeys) {
final ListObjectsV2Request request = new ListObjectsV2Request()
.withBucketName(bucket)
.withMaxKeys(maxKeys)
.withPrefix(key);
if (delimiter != null) {
request.setDelimiter(delimiter);
}
return prepareRequest(request);
}
@Override
public DeleteObjectRequest newDeleteObjectRequest(String key) {
return prepareRequest(new DeleteObjectRequest(bucket, key));
}
@Override
public DeleteObjectsRequest newBulkDeleteRequest(
List<DeleteObjectsRequest.KeyVersion> keysToDelete,
boolean quiet) {
return prepareRequest(
new DeleteObjectsRequest(bucket)
.withKeys(keysToDelete)
.withQuiet(quiet));
}
@Override
public void setEncryptionSecrets(final EncryptionSecrets secrets) {
encryptionSecrets = secrets;
}
/**
* Create a builder.
* @return new builder.
*/
public static RequestFactoryBuilder builder() {
return new RequestFactoryBuilder();
}
/**
* Builder.
*/
public static final class RequestFactoryBuilder {
/**
* Target bucket.
*/
private String bucket;
/**
* Encryption secrets.
*/
private EncryptionSecrets encryptionSecrets = new EncryptionSecrets();
/**
* ACL For new objects.
*/
private CannedAccessControlList cannedACL = null;
/** Requester Pays flag. */
private boolean requesterPays = false;
/**
* Multipart limit.
*/
private long multipartPartCountLimit = DEFAULT_UPLOAD_PART_COUNT_LIMIT;
/**
* Callback to prepare requests.
*/
private PrepareRequest requestPreparer;
private RequestFactoryBuilder() {
}
/**
* Build the request factory.
* @return the factory
*/
public RequestFactory build() {
return new RequestFactoryImpl(this);
}
/**
* Target bucket.
* @param value new value
* @return the builder
*/
public RequestFactoryBuilder withBucket(final String value) {
bucket = value;
return this;
}
/**
* Encryption secrets.
* @param value new value
* @return the builder
*/
public RequestFactoryBuilder withEncryptionSecrets(
final EncryptionSecrets value) {
encryptionSecrets = value;
return this;
}
/**
* ACL For new objects.
* @param value new value
* @return the builder
*/
public RequestFactoryBuilder withCannedACL(
final CannedAccessControlList value) {
cannedACL = value;
return this;
}
/**
* Requester Pays flag.
* @param value new value
* @return the builder
*/
public RequestFactoryBuilder withRequesterPays(
final boolean value) {
requesterPays = value;
return this;
}
/**
* Multipart limit.
* @param value new value
* @return the builder
*/
public RequestFactoryBuilder withMultipartPartCountLimit(
final long value) {
multipartPartCountLimit = value;
return this;
}
/**
* Callback to prepare requests.
*
* @param value new value
* @return the builder
*/
public RequestFactoryBuilder withRequestPreparer(
final PrepareRequest value) {
this.requestPreparer = value;
return this;
}
}
/**
* This is a callback for anything to "prepare" every request
* after creation. The S3AFileSystem's Audit Manager is expected
* to be wired up via this call so can audit/prepare requests
* after their creation.
*/
@FunctionalInterface
public interface PrepareRequest {
/**
* Post-creation preparation of AWS request.
* @param t request
* @param <T> request type.
* @return prepared entry.
*/
@Retries.OnceRaw
<T extends AmazonWebServiceRequest> T prepareRequest(T t);
}
}

View File

@ -32,6 +32,8 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.api.RequestFactory;
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
@ -40,6 +42,9 @@
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.fs.store.audit.AuditSpanSource;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.LambdaUtils;
import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
@ -59,7 +64,7 @@
*/
@InterfaceAudience.LimitedPrivate("S3A Filesystem and extensions")
@InterfaceStability.Unstable
public class StoreContext {
public class StoreContext implements ActiveThreadSpanSource<AuditSpan> {
/** Filesystem URI. */
private final URI fsURI;
@ -117,13 +122,17 @@ public class StoreContext {
/**
* Source of time.
*/
private ITtlTimeProvider timeProvider;
/** Time source for S3Guard TTLs. */
private final ITtlTimeProvider timeProvider;
/** Operation Auditor. */
private final AuditSpanSource<AuditSpanS3A> auditor;
/**
* Instantiate.
* @deprecated as public method: use {@link StoreContextBuilder}.
*/
public StoreContext(
StoreContext(
final URI fsURI,
final String bucket,
final Configuration configuration,
@ -140,13 +149,17 @@ public StoreContext(
final MetadataStore metadataStore,
final boolean useListV1,
final ContextAccessors contextAccessors,
final ITtlTimeProvider timeProvider) {
final ITtlTimeProvider timeProvider,
final AuditSpanSource<AuditSpanS3A> auditor) {
this.fsURI = fsURI;
this.bucket = bucket;
this.configuration = configuration;
this.username = username;
this.owner = owner;
this.executor = MoreExecutors.listeningDecorator(executor);
// some mock tests have a null executor pool
this.executor = executor !=null
? MoreExecutors.listeningDecorator(executor)
: null;
this.executorCapacity = executorCapacity;
this.invoker = invoker;
this.instrumentation = instrumentation;
@ -158,11 +171,7 @@ public StoreContext(
this.useListV1 = useListV1;
this.contextAccessors = contextAccessors;
this.timeProvider = timeProvider;
}
@Override
protected Object clone() throws CloneNotSupportedException {
return super.clone();
this.auditor = auditor;
}
public URI getFsURI() {
@ -391,4 +400,33 @@ public <T> CompletableFuture<T> submit(
LambdaUtils.eval(future, call));
return future;
}
/**
* Get the auditor.
* @return auditor.
*/
public AuditSpanSource<AuditSpanS3A> getAuditor() {
return auditor;
}
/**
* Return the active audit span.
* This is thread local -it MUST be passed into workers.
* To ensure the correct span is used, it SHOULD be
* collected as early as possible, ideally during construction/
* or service init/start.
* @return active audit span.
*/
@Override
public AuditSpan getActiveAuditSpan() {
return contextAccessors.getActiveAuditSpan();
}
/**
* Get the request factory.
* @return the factory for requests.
*/
public RequestFactory getRequestFactory() {
return contextAccessors.getRequestFactory();
}
}

View File

@ -25,9 +25,11 @@
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.s3a.s3guard.ITtlTimeProvider;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.store.audit.AuditSpanSource;
import org.apache.hadoop.security.UserGroupInformation;
/**
@ -69,6 +71,8 @@ public class StoreContextBuilder {
private ITtlTimeProvider timeProvider;
private AuditSpanSource<AuditSpanS3A> auditor;
public StoreContextBuilder setFsURI(final URI fsURI) {
this.fsURI = fsURI;
return this;
@ -165,6 +169,17 @@ public StoreContextBuilder setTimeProvider(
return this;
}
/**
* Set builder value.
* @param value new value
* @return the builder
*/
public StoreContextBuilder setAuditor(
final AuditSpanSource<AuditSpanS3A> value) {
auditor = value;
return this;
}
@SuppressWarnings("deprecation")
public StoreContext build() {
return new StoreContext(fsURI,
@ -183,6 +198,7 @@ public StoreContext build() {
metadataStore,
useListV1,
contextAccessors,
timeProvider);
timeProvider,
auditor);
}
}

View File

@ -52,6 +52,7 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
import org.apache.hadoop.fs.s3a.S3ListRequest;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.service.Service;
import org.apache.hadoop.service.launcher.LauncherExitCodes;
import org.apache.hadoop.service.launcher.ServiceLaunchException;
@ -347,21 +348,26 @@ protected long listStatusFilesystem(
protected long dumpRawS3ObjectStore(
final CsvFile csv) throws IOException {
S3AFileSystem fs = getFilesystem();
Path rootPath = fs.qualify(new Path("/"));
Listing listing = fs.getListing();
S3ListRequest request = listing.createListObjectsRequest("", null);
long count = 0;
RemoteIterator<S3AFileStatus> st =
listing.createFileStatusListingIterator(rootPath, request,
ACCEPT_ALL,
new Listing.AcceptAllButSelfAndS3nDirs(rootPath));
while (st.hasNext()) {
count++;
S3AFileStatus next = st.next();
LOG.debug("[{}] {}", count, next);
csv.entry(next);
Path rootPath = fs.qualify(new Path("/"));
try (AuditSpan span = fs.createSpan("DumpS3GuardDynamoTable",
rootPath.toString(), null)) {
Listing listing = fs.getListing();
S3ListRequest request = listing.createListObjectsRequest("", null, span);
count = 0;
RemoteIterator<S3AFileStatus> st =
listing.createFileStatusListingIterator(rootPath, request,
ACCEPT_ALL,
new Listing.AcceptAllButSelfAndS3nDirs(rootPath),
span);
while (st.hasNext()) {
count++;
S3AFileStatus next = st.next();
LOG.debug("[{}] {}", count, next);
csv.entry(next);
}
LOG.info("entry count: {}", count);
}
LOG.info("entry count: {}", count);
return count;
}

View File

@ -63,6 +63,8 @@
import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
import com.amazonaws.services.dynamodbv2.model.TableDescription;
import com.amazonaws.services.dynamodbv2.model.WriteRequest;
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
@ -593,7 +595,7 @@ private void innerDelete(final Path path,
}
// the policy on whether repeating delete operations is based
// on that of S3A itself
boolean idempotent = S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT;
boolean idempotent = InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
if (tombstone) {
Preconditions.checkArgument(ttlTimeProvider != null, "ttlTimeProvider "
+ "must not be null");

View File

@ -30,7 +30,6 @@
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@ -55,11 +54,11 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageStatistics;
import org.apache.hadoop.fs.s3a.MultipartUtils;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AUtils;
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
import org.apache.hadoop.fs.s3a.auth.RolePolicies;
import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
@ -69,6 +68,8 @@
import org.apache.hadoop.fs.s3a.select.SelectTool;
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
import org.apache.hadoop.fs.shell.CommandFormat;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ExitCodeProvider;
@ -84,6 +85,9 @@
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.FILESYSTEM_TEMP_PATH;
import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStoreTableManager.SSE_DEFAULT_MASTER_KEY;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_ABORTED;
import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
/**
@ -531,16 +535,13 @@ protected void dumpFileSystemStatistics(PrintStream stream) {
if (fs == null) {
return;
}
println(stream, "%nStorage Statistics for %s%n", fs.getUri());
StorageStatistics st = fs.getStorageStatistics();
Iterator<StorageStatistics.LongStatistic> it
= st.getLongStatistics();
while (it.hasNext()) {
StorageStatistics.LongStatistic next = it.next();
long value = next.getValue();
if (value != 0) {
println(stream, "%s\t%s", next.getName(), value);
}
println(stream, "%nIO Statistics for %s%n", fs.getUri());
final IOStatistics iostats = retrieveIOStatistics(fs);
if (iostats != null) {
println(stream, ioStatisticsToPrettyString(iostats));
} else {
println(stream, "FileSystem does not provide IOStatistics");
}
println(stream, "");
}
@ -890,15 +891,18 @@ public int run(String[] args, PrintStream out) throws Exception {
final CommandFormat commandFormat = getCommandFormat();
final boolean verbose = commandFormat.getOpt(VERBOSE);
final ImportOperation importer = new ImportOperation(
getFilesystem(),
getStore(),
status,
commandFormat.getOpt(AUTH_FLAG),
commandFormat.getOpt(VERBOSE));
verbose);
long items = importer.execute();
println(out, "Inserted %d items into Metadata Store", items);
if (verbose) {
dumpFileSystemStatistics(out);
}
return SUCCESS;
}
@ -1584,9 +1588,12 @@ public int run(String[] args, PrintStream out)
throw invalidArgs("No options specified");
}
processArgs(paths, out);
println(out, "Listing uploads under path \"%s\"", prefix);
promptBeforeAbort(out);
processUploads(out);
if (verbose) {
dumpFileSystemStatistics(out);
}
out.flush();
return SUCCESS;
}
@ -1605,8 +1612,15 @@ private void promptBeforeAbort(PrintStream out) throws IOException {
}
private void processUploads(PrintStream out) throws IOException {
MultipartUtils.UploadIterator uploads;
uploads = getFilesystem().listUploads(prefix);
final S3AFileSystem fs = getFilesystem();
MultipartUtils.UploadIterator uploads = fs.listUploads(prefix);
// create a span so that the write operation helper
// is within one
AuditSpan span =
fs.createSpan(MULTIPART_UPLOAD_ABORTED,
prefix, null);
final WriteOperationHelper writeOperationHelper
= fs.getWriteOperationHelper();
int count = 0;
while (uploads.hasNext()) {
@ -1620,18 +1634,20 @@ private void processUploads(PrintStream out) throws IOException {
upload.getKey(), upload.getUploadId());
}
if (mode == Mode.ABORT) {
getFilesystem().getWriteOperationHelper()
writeOperationHelper
.abortMultipartUpload(upload.getKey(), upload.getUploadId(),
true, LOG_EVENT);
}
}
span.deactivate();
if (mode != Mode.EXPECT || verbose) {
println(out, "%s %d uploads %s.", TOTAL, count,
mode == Mode.ABORT ? "deleted" : "found");
}
if (mode == Mode.EXPECT) {
if (count != expectedCount) {
throw badState("Expected %d uploads, found %d", expectedCount, count);
throw badState("Expected upload count under %s: %d, found %d",
prefix, expectedCount, count);
}
}
}
@ -1643,6 +1659,9 @@ private void processUploads(PrintStream out) throws IOException {
* @return true iff u was created at least age milliseconds ago.
*/
private boolean olderThan(MultipartUpload u, long msec) {
if (msec == 0) {
return true;
}
Date ageDate = new Date(System.currentTimeMillis() - msec);
return ageDate.compareTo(u.getInitiated()) >= 0;
}

View File

@ -20,7 +20,6 @@
import java.io.IOException;
import java.util.Locale;
import java.util.Optional;
import com.amazonaws.services.s3.model.CSVInput;
import com.amazonaws.services.s3.model.CSVOutput;
@ -28,7 +27,6 @@
import com.amazonaws.services.s3.model.InputSerialization;
import com.amazonaws.services.s3.model.OutputSerialization;
import com.amazonaws.services.s3.model.QuoteFields;
import com.amazonaws.services.s3.model.SSECustomerKey;
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.slf4j.Logger;
@ -55,6 +53,7 @@
* This class is intended to be instantiated by the owning S3AFileSystem
* instance to handle the construction of requests: IO is still done exclusively
* in the filesystem.
*
*/
public class SelectBinding {
@ -70,12 +69,12 @@ public class SelectBinding {
/**
* Constructor.
* @param operations owning FS.
* @param operations callback to owner FS, with associated span.
*/
public SelectBinding(final WriteOperationHelper operations) {
this.operations = checkNotNull(operations);
Configuration conf = getConf();
this.enabled = conf.getBoolean(FS_S3A_SELECT_ENABLED, true);
this.enabled = isSelectEnabled(conf);
this.errorsIncludeSql = conf.getBoolean(SELECT_ERRORS_INCLUDE_SQL, false);
}
@ -91,12 +90,20 @@ public boolean isEnabled() {
return enabled;
}
/**
* Static probe for select being enabled.
* @param conf configuration
* @return true iff select is enabled.
*/
public static boolean isSelectEnabled(Configuration conf) {
return conf.getBoolean(FS_S3A_SELECT_ENABLED, true);
}
/**
* Build and execute a select request.
* @param readContext the read context, which includes the source path.
* @param expression the SQL expression.
* @param builderOptions query options
* @param sseKey optional SSE customer key
* @param objectAttributes object attributes from a HEAD request
* @return an FSDataInputStream whose wrapped stream is a SelectInputStream
* @throws IllegalArgumentException argument failure
@ -108,7 +115,6 @@ public FSDataInputStream select(
final S3AReadOpContext readContext,
final String expression,
final Configuration builderOptions,
final Optional<SSECustomerKey> sseKey,
final S3ObjectAttributes objectAttributes) throws IOException {
return new FSDataInputStream(
@ -118,8 +124,8 @@ public FSDataInputStream select(
buildSelectRequest(
readContext.getPath(),
expression,
builderOptions,
sseKey)));
builderOptions
)));
}
/**
@ -127,7 +133,6 @@ public FSDataInputStream select(
* @param path source path.
* @param expression the SQL expression.
* @param builderOptions config to extract other query options from
* @param sseKey optional SSE customer key
* @return the request to serve
* @throws IllegalArgumentException argument failure
* @throws IOException problem building/validating the request
@ -135,16 +140,13 @@ public FSDataInputStream select(
public SelectObjectContentRequest buildSelectRequest(
final Path path,
final String expression,
final Configuration builderOptions,
final Optional<SSECustomerKey> sseKey)
final Configuration builderOptions)
throws IOException {
Preconditions.checkState(isEnabled(),
"S3 Select is not enabled for %s", path);
SelectObjectContentRequest request = operations.newSelectRequest(path);
buildRequest(request, expression, builderOptions);
// optionally set an SSE key in the input
sseKey.ifPresent(request::withSSECustomerKey);
return request;
}
@ -428,4 +430,5 @@ static String expandBackslashChars(String src) {
.replace("\\\\", "\\");
}
}

View File

@ -0,0 +1,186 @@
/*
* 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.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 org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
/**
* This may seem odd having an IOStatisticsStore which does nothing
* but forward to a wrapped store, but it's designed to
* assist in subclassing of selective methods, such
* as those to increment counters, get durations etc.
*/
public class ForwardingIOStatisticsStore implements IOStatisticsStore {
private final IOStatisticsStore innerStatistics;
public ForwardingIOStatisticsStore(
final IOStatisticsStore innerStatistics) {
this.innerStatistics = innerStatistics;
}
protected IOStatisticsStore getInnerStatistics() {
return innerStatistics;
}
@Override
public Map<String, Long> counters() {
return getInnerStatistics().counters();
}
@Override
public Map<String, Long> gauges() {
return getInnerStatistics().gauges();
}
@Override
public Map<String, Long> minimums() {
return getInnerStatistics().minimums();
}
@Override
public Map<String, Long> maximums() {
return getInnerStatistics().maximums();
}
@Override
public Map<String, MeanStatistic> meanStatistics() {
return getInnerStatistics().meanStatistics();
}
@Override
public boolean aggregate(@Nullable final IOStatistics statistics) {
return getInnerStatistics().aggregate(statistics);
}
@Override
public long incrementCounter(final String key, final long value) {
return getInnerStatistics().incrementCounter(key, value);
}
@Override
public void setCounter(final String key, final long value) {
getInnerStatistics().setCounter(key, value);
}
@Override
public void setGauge(final String key, final long value) {
getInnerStatistics().setGauge(key, value);
}
@Override
public long incrementGauge(final String key, final long value) {
return getInnerStatistics().incrementGauge(key, value);
}
@Override
public void setMaximum(final String key, final long value) {
getInnerStatistics().setMaximum(key, value);
}
@Override
public long incrementMaximum(final String key, final long value) {
return getInnerStatistics().incrementMaximum(key, value);
}
@Override
public void setMinimum(final String key, final long value) {
getInnerStatistics().setMinimum(key, value);
}
@Override
public long incrementMinimum(final String key, final long value) {
return getInnerStatistics().incrementMinimum(key, value);
}
@Override
public void addMinimumSample(final String key, final long value) {
getInnerStatistics().addMinimumSample(key, value);
}
@Override
public void addMaximumSample(final String key, final long value) {
getInnerStatistics().addMaximumSample(key, value);
}
@Override
public void setMeanStatistic(final String key, final MeanStatistic value) {
getInnerStatistics().setMeanStatistic(key, value);
}
@Override
public void addMeanStatisticSample(final String key, final long value) {
getInnerStatistics().addMeanStatisticSample(key, value);
}
@Override
public void reset() {
getInnerStatistics().reset();
}
@Override
public AtomicLong getCounterReference(final String key) {
return getInnerStatistics().getCounterReference(key);
}
@Override
public AtomicLong getMaximumReference(final String key) {
return getInnerStatistics().getMaximumReference(key);
}
@Override
public AtomicLong getMinimumReference(final String key) {
return getInnerStatistics().getMinimumReference(key);
}
@Override
public AtomicLong getGaugeReference(final String key) {
return getInnerStatistics().getGaugeReference(key);
}
@Override
public MeanStatistic getMeanStatistic(final String key) {
return getInnerStatistics().getMeanStatistic(key);
}
@Override
public void addTimedOperation(final String prefix,
final long durationMillis) {
getInnerStatistics().addTimedOperation(prefix, durationMillis);
}
@Override
public void addTimedOperation(final String prefix,
final Duration duration) {
getInnerStatistics().addTimedOperation(prefix, duration);
}
}

View File

@ -363,7 +363,6 @@ ScanResult execute(final ScanArgs scanArgs)
// extract the callbacks needed for the rest of the work
storeContext = fs.createStoreContext();
operations = fs.createMarkerToolOperations();
// filesystem policy.
// if the -nonauth option is set, this is used to filter
// out surplus markers from the results.
@ -417,13 +416,15 @@ ScanResult execute(final ScanArgs scanArgs)
minMarkerCount = maxMarkerCount;
maxMarkerCount = m;
}
ScanResult result = scan(target,
// extract the callbacks needed for the rest of the work
operations = fs.createMarkerToolOperations(
target.toString());
return scan(target,
scanArgs.isDoPurge(),
minMarkerCount,
maxMarkerCount,
scanArgs.getLimit(),
filterPolicy);
return result;
}
/**

View File

@ -0,0 +1,389 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# Object Store Auditing
The S3A connector provides an extension point for auditing requests to S3.
The auditing can take place at the entry point to every FS operation,
and inside the AWS S3 SDK, immediately before the request is executed.
The full architecture is covered in [Auditing Architecture](auditing_architecture.html);
this document covers its use.
## Auditing workflow
1. An _Auditor Service_ can be instantiated for each S3A FileSystem instance,
created during FS initialization and closed when the FS instance is closed.
1. The S3A FS will request from the Auditor Service an _Audit Span_ for each Hadoop FileSystem API call.
1. The audit span will have callbacks invoked during each of the S3 operations
invoked during the execution of the API call, *from within the AWS SDK*
1. This allows the Auditor Service to log requests made and associate with users and operations.
1. And/or reject operations.
1. The bundled "Logging Auditor" logs operations and attaches information about calls to the HTTP Referrer header.
1. So aiding debugging of issues related to performance, bucket load, S3 costs...etc.
Thus: an Auditor Service can be plugged in to provide (best-effort) auditing as well
as hinted allow/deny security.
* Why best effort: coverage is not complete. See limitations below.
* Why "hinted" security? Any custom code running in the JVM could retrieve the AWS
credential chain and so bypass this auditing mechanism.
## Limitations
This is not a means of controlling access to S3 resources. It is a best-effort
attempt at supporting logging of FileSystem operations API calls, and, in
particular, correlating S3 multiple object requests with a single FS API call,
ideally even identifying the process/job generating load.
* Low-level code using public S3A methods intended only for internal use may not
create spans.
* Code which asks for the AWS S3 client may bypass span creation.
* Application code can also create a new S3 client (reusing any existing
credentials)
and so have unaudited access to S3.
* There's no tie-up with OpenTelemetry.
* Uploads and copy operations through the TransferManager do not pick up an
active span because work is executed in threads which the S3A code cannot
update.
* There's a limit to how long an http referer header can be; operations on long
paths may be incompletely logged.
## Using Auditing
The Logging Auditor is enabled by default; it annotates the S3 logs.
### Auditor Options
| Option | Meaning | Default Value |
|--------|---------|---------------|
| `fs.s3a.audit.service.classname` | Auditor classname | `org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor` |
| `fs.s3a.audit.request.handlers` | List of extra subclasses of AWS SDK RequestHandler2 to include in handler chain | `""` |
| `fs.s3a.audit.referrer.enabled` | Logging auditor to publish the audit information in the HTTP Referrer header | `true` |
| `fs.s3a.audit.referrer.filter` | List of audit fields to filter | `""` |
| `fs.s3a.audit.reject.out.of.span.operations` | Auditor to reject operations "outside of a span" | `false` |
### Disabling Auditing with the No-op Auditor
The No-op auditor does not perform any logging of audit events.
```xml
<property>
<name>fs.s3a.audit.service.classname</name>
<value>org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor</value>
</property>
```
### Auditing with the Logging Auditor
The "Logging Auditor" is the default auditor.
It provides two forms of logging
1. Logging of operations in the client via Log4J.
1. Dynamic generation of the HTTP Referrer header for S3 requests.
The Logging Auditor is enabled by providing its classname in the option
`fs.s3a.audit.service.classname`.
```xml
<property>
<name>fs.s3a.audit.service.classname</name>
<value>org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor</value>
</property>
```
To print auditing events in the local client logs, set the associated Log4J log
to log at debug:
```
# Auditing
log4j.logger.org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor=DEBUG
```
### Integration with S3 Server Access Logging
An AWS S3 bucket can be configured to store logs of all HTTP requests made of a bucket
into a different S3 bucket,
[S3 Server Access Logging](https://docs.aws.amazon.com/AmazonS3/latest/userguide/ServerLogs.html)
In the logging auditor the HTTP `referer` field of every AWS S3 request is built
up into a URL which provides context and span information. As this field is
saved in the S3 logs, if S3 bucket logging is enabled, the logs will be able to
correlate access by S3 clients to the actual operations taking place.
Note: this logging is described as "Best Effort". There's no guarantee as to
when logs arrive.
### Rejecting out-of-span operations
The logging auditor can be configured to raise an exception whenever
a request is made to S3 outside an audited span -that is: the thread
interacting with S3 through the `S3AFileSystem` instance which created
the auditor does not have any span activated.
This is primarily for development, as it can be used to guarantee
spans are being entered through the public API calls.
```xml
<property>
<name>fs.s3a.audit.reject.out.of.span.operations</name>
<value>true</value>
</property>
```
This rejection process is disabled for some AWS S3 Request classes,
which are created within the AWS SDK as part of larger operations
and for which spans cannot be attached.
| AWS Request Always allowed | Reason |
|----------------------------|--------|
| `GetBucketLocationRequest` | Used in AWS SDK to determine S3 endpoint |
| `CopyPartRequest` | Used in AWS SDK during copy operations |
| `CompleteMultipartUploadRequest` | Used in AWS SDK to complete copy operations |
The request to initiate a copy/multipart upload is always audited,
therefore the auditing process does have coverage of rename and multipart
IO. However, the AWS S3 logs will not include full trace information
in the referrer header of the associated copy/complete calls.
## Auditing and the HTTP Referrer header
The HTTP referrer header is attached by the logging auditor.
If the S3 Bucket is configured to log requests to another bucket, then these logs
entries will include the audit information _as the referrer_.
This can be parsed (consult AWS documentation for a regular expression)
and the http referrer header extracted.
```
https://audit.example.org/hadoop/1/op_rename/3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3/
?op=op_rename
&p1=s3a://alice-london/path1
&pr=alice
&p2=s3a://alice-london/path2
&ps=235865a0-d399-4696-9978-64568db1b51c
&id=3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3
&t0=12
&fs=af5943a9-b6f6-4eec-9c58-008982fc492a
&t1=12
&ts=1617116985923
```
Here are the fields which may be found in a request.
If any of the field values were `null`, the field is omitted.
| Name | Meaning | Example |
|------|---------|---------|
| `cm` | Command | `S3GuardTool$BucketInfo` |
| `fs` | FileSystem ID | `af5943a9-b6f6-4eec-9c58-008982fc492a` |
| `id` | Span ID | `3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3` |
| `ji` | Job ID | `(Generated by query engine)` |
| `op` | Filesystem API call | `op_rename` |
| `p1` | Path 1 of operation | `s3a://alice-london/path1` |
| `p2` | Path 2 of operation | `s3a://alice-london/path2` |
| `pr` | Principal | `alice` |
| `ps` | Unique process UUID | `235865a0-d399-4696-9978-64568db1b51c` |
| `t0` | Thread 0: thread span was created in | `100` |
| `t1` | Thread 1: thread this operation was executed in | `200` |
| `ts` | Timestamp (UTC epoch millis) | `1617116985923` |
Thread IDs are from the current thread in the JVM.
```java
Long.toString(Thread.currentThread().getId())
```
When `t0` and `t1` are different it means that the span
has been handed off to another thread for work on
behalf of the original operation.
This can be correlated with log entries on the client
to isolate work to specific threads.
### Limitations of the HTTP Referrer header
There is a size limit on the length of the header;
operations on long paths may exceed it.
In such situations the audit log is incomplete.
This is why the span ID is always passed in as part of the URL,
rather than just an HTTP query parameter: even if
the header is chopped, the span ID will always be present.
## Privacy Implications of HTTP Referrer auditing
When the S3A client makes requests of an S3 bucket, the auditor
adds span information to the header, which is then
stored in the logs
If the S3 bucket is owned by the same organization as the client,
this span information is internal to the organization.
If the S3 bucket is owned/managed by a different entity,
then the span information is visible in any S3 bucket logs
collected by that entity. This includes the principal name
and the command executed if the application is launched via the `Tools` or
service launcher APIs.
Sharing this information can be disabled by either filtering specific
headers, or by explicitly disabling referrer header generation entirely.
Note: even when the HTTP Referrer is disabled by or the principal filtered,
AWS S3 logs include ARN of the user or IAM role making the request.
### Filtering Referrer headers
Specific fields can be filtered from the referrer header, and so are not
included in the S3A logs.
```xml
<property>
<name>fs.s3a.audit.referrer.filter</name>
<value>pr, cm</value>
<description>Strip out principal and command from referrer headers</description>
</property>
```
### Disabling Referrer headers
The logging auditor can be configured to not add the referrer header
by setting the option `fs.s3a.audit.referrer.enabled` to `false`,
either globally or for specific buckets:
```xml
<property>
<name>fs.s3a.audit.referrer.enabled</name>
<value>false</value>
<description>Disable referrer for all buckets</description>
</property>
<property>
<name>fs.s3a.bucket.landsat-pds.audit.referrer.enabled</name>
<value>false</value>
<description>Do not add the referrer header to landsat operations</description>
</property>
```
## Collecting AWS S3 Logs for Analysis
The S3 Bucket(s) must be set up for
[Server Access Logging](https://docs.aws.amazon.com/AmazonS3/latest/userguide/ServerLogs.html).
This will tell AWS S3 to collect access logs of all HTTP requests
and store them in a different bucket in the same region.
The logs arrive as files containing a few seconds worth
of log data, stored under the configured path.
### Enabling logging: Source bucket
1. Create a separate bucket for logs in the same region, if you do not already have one.
1. In the S3 console, locate the bucket you wish to act as a source for logs,
and go to the "properties".
1. Scroll down to "Server access logging"
1. Select "edit" and then enable logging, entering a path in a nearby bucket for logs.
(Tip: for ease of logging multiple buckets to the same log bucket, use a prefix like
`logs/$BUCKET/log-` to isolate different bucket's logs.
For example, the path log data from `dev data london` could be
`s3://london-log-bucket/logs/dev-data-lon/log-`
1. Save this.
There's a lag of about an hour between S3 requests being made and the logs
appearing; don't worry during setup if things do not appear to be working.
Enable the log, work with the bucket through the "hadoop fs" command line, wait
an hour, then go and look in the log bucket for the entries.
The log filename includes the time at which these logs
began
### Keeping costs down by deleting old logs.
As logs are stored in an S3 bucket, they too run up charges.
Keep costs down by deleting logs after a period of time, and/or
set up a workflow to load and coalesce log entries into a compressed
format and larger files.
It is straightforward to set up a rule to automatically delete old log files.
1. In the S3 console, bring up the bucket which is the destination for the logs,
e.g. `london-log-bucket`.
1. Go to the "Management" tab.
1. Add a lifecycle rule (alongside the "abort pending uploads" rule you should already have).
1. Add rule name "Delete old log files".
1. Select "Limit the scope".
1. Add the prefix `logs/` to have it delete all logs of all buckets.
Important: you _must not_ have any leading "/", such as `/logs/` -there will be no
match and the rule will not work.
1. In "Lifecycle rule actions", select "Expire current versions"
This will delete log entries.
1. In "Expire current versions of objects", set the number of days to keep
log entries.
1. Finish by pressing the "Create Rule" button
Keep an eye on the bucket to make sure the deletion is working; it's easy to
make an error in the prefix, and as logs will be created without limit,
costs will ramp up.
## Parsing AWS S3 Logs to extract the referrer header
The [AWS S3 Documentation](https://docs.aws.amazon.com/AmazonS3/latest/userguide/LogFormat.html)
covers the log format and includes a hive external table declaration to work with it.
The Java pattern regular expression used in the `hadoop-aws` test suites to
extract headers is defined as:
```
(?<owner>[^ ]*) (?<bucket>[^ ]*) (?<timestamp>\[(.*?)\]) (?<remoteip>[^ ]*) (?<requester>[^ ]*) (?<requestid>[^ ]*) (?<operation>[^ ]*) (?<key>[^ ]*) (?<requesturi>(-|"[^"]*")) (?<http>(-|[0-9]*)) (?<awserrorcode>[^ ]*) (?<bytessent>[^ ]*) (?<objectsize>[^ ]*) (?<totaltime>[^ ]*) (?<turnaroundtime>[^ ]*) (?<referrer>(-|"[^"]*")) (?<useragent>(-|"[^"]*")) (?<version>[^ ]*) (?<hostid>[^ ]*) (?<sigv>[^ ]*) (?<cypher>[^ ]*) (?<auth>[^ ]*) (?<endpoint>[^ ]*) (?<tls>[^ ]*)*$
```
The class `org.apache.hadoop.fs.s3a.audit.S3LogParser` provides this pattern
as well as constants for each group. It is declared as `Public/Unstable`.
## Debugging
The `org.apache.hadoop.fs.s3a.audit` log context contains logs for the different
components implementing auditing.
Logging of requests audited with the `LoggingAuditService` can be enabled by
setting that log to debug.
```
# Log before a request is made to S3
log4j.logger.org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor=DEBUG
```
This adds one log line per request -and does provide some insight into
communications between the S3A client and AWS S3.
For low-level debugging of the Auditing system, such as when when spans are
entered and exited, set the log to `TRACE`:
```
# log request creation, span lifecycle and other low-level details
log4j.logger.org.apache.hadoop.fs.s3a.audit=TRACE
```
This is very noisy and not recommended in normal operation.
## Integration with S3A Committers
Work submitted through the S3A committer will have the job (query) ID associated
with S3 operations taking place against all S3A filesystems in that thread.
For this to be useful, the work performed in a task MUST be in the same thread
which called `jobSetup()` or `taskSetup()` on the committer.

View File

@ -0,0 +1,323 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# Object Store Auditing: Architecture
This the architecture document of the S3A [Auditing](auditing.html) component.
The S3A auditing subsystem is defined in the package `org.apache.hadoop.fs.s3a.audit`.
This package is declared `LimitedPrivate`; some classes inside are explicitly
declared `@Public` (e.g `AuditConstants`) while others `@Private`. If declared
`@Private`, external auditing modules MUST NOT use them.
```java
@InterfaceAudience.LimitedPrivate("auditing extensions")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.audit;
```
The auditing implementation classes are all in the package
package `org.apache.hadoop.fs.s3a.audit.impl`.
These MUST NOT be subclassed or invoked directly by external code.
Audit classes/interfaces which are intended to be used across object store
clients and manipulated from other parts of hadoop are in `hadoop-common` JAR in
the package `org.apache.hadoop.fs.store`.
### Interface `org.apache.hadoop.fs.store.audit.AuditSpan`
An AuditSpan audits a single Hadoop FileSystem API operation such as
`open(Path)`, `rename(Path, Path)` or `listFiles(Path, Boolean)`.
```java
public interface AuditSpan extends Closeable {
String getSpanId();
String getOperationName();
long getTimestamp();
AuditSpan activate();
void deactivate();
default void close() {
deactivate();
}
boolean isValidSpan();
void set(String key, String value);
}
```
Audit Spans are intended for use _within_ FileSystem clients; that is
not visible to applications invoking them.
1. One `AuditSpan` is created per Hadoop FS API call.
1. Each span has the name of the operation and optionally source and destination paths.
1. A span may be `activate()`d or `deactivate()`d. Between these two operations a span is _active_.
1. Activation is on a per-thread basis. A single span can be active in multiple threads
simultaneously; other spans may be active in other threads.
1. A single filesystem can have only one active span per thread, but different filesystem
instances MAY have different active spans.
1. All store operations performed on a thread are considered _within_
the active span.
1. Spans do not explicitly terminate; they just stop being invoked; eventually
Garbage Collection should dispose of them.
1. Every `AuditSpan` has an ID, which *must* be unique. A UUID and a counter is
the base implementation.
1. The `AuditSpan` class does extend `Closeable`; calling `close()` simply deactivates
the span _for that thread_.
1. All FS API calls which return objects which go on to perform FS operations
(`create()`, `open()`, incremental list calls which return `RemoteIterator` etc) pass
the span into the objects which they return.
1. As a result, any store IO performed by the returned streams and iterators MUST activate
the span before that IO and deactivate it afterwards.
1. There is also the "Unbonded Span" which is the effective span of an FS when there
is no active span.
1. Calling a store within the unbonded span is generally considered an bug. In the S3A codebase
this should never happen outside copy/rename operations, and will be logged at
warning level in the Logging Auditor.
### interface `org.apache.hadoop.fs.store.audit.AuditSpanSource`
This interface is implemented by sources of audit spans.
```java
public interface AuditSpanSource<T extends AuditSpan> {
T createSpan(String operation,
@Nullable String path1,
@Nullable String path2)
throws IOException;
}
```
All S3 Auditors implement this interface, as does the `AuditManagerS3A`.
(Implementation note: so do `S3AFileSystem` and `WriteOperationHelper`)
When a Hadoop FS API call is made of an `S3AFileSystem` instance, it
calls `startOperation` on its audit manager; this will relay it to
the auditor is bound to.
The auditor then creates and returns a span for the specific operation.
The AuditManagerS3A will automatically activate the span returned by the auditor
(i.e. assign it the thread local variable tracking the active span in each thread)
### Class `org.apache.hadoop.fs.audit.CommonAuditContext`
This is a class in `hadoop-common` which provides a context to auditing operations
across all instrumented filesystems.
It's Global Context values are a map of string keys and values, which are
constant across all threads. This is where global values such as a process
UUID and the class executed by `ToolRunner` are noted.
The `CommonAuditContext.currentAuditContext()` call returns a thread local
`CommonAuditContext` which is a thread-local map of keys to string values.
It also supports a map of _evaluated entries_.
This is a map of type `Map&lt;String, Supplier&lt;String>>`.
supplier methods/lambda expressions set here are dynamically evaluated when
auditors retrieve the values.
Spans may be used on different thread from that which they were created.
Spans MUST always use the values from the `currentAuditContext()` in the creation
thread.
### class `NoopAuditor`
This auditor creates spans which perform no auditing.
It is very efficient and reliable.
### class `LoggingAuditor`
The logging auditor logs operations to the console at DEBUG level (to keep the noise down),
and attaches the operation details in the HTTP "referer" header.
It can be configured to raise an exception whenever an S3 API call is made
from within the unbonded span.
This option primarily for development, as it is how we can verify that all
calls are audited/identify where this is not possible.
### class `ActiveAuditManager` interface `ActiveAuditManager`
The class `ActiveAuditManager` provides all the support needed for
`S3AFileSystem` to support spans, including
* Loading and starting the auditor declared in a Hadoop configuration.
* Maintaining a per-thread record of the active audit span
* Switching spans on `AuditSpan.activate()` and reverting to the
unbonded span in `deactivate()` and `close()`.
* Providing binding classes to be passed into the AWS SDK so as to
invoke audit operations prior to requests being issued. This is essential to
guarantee that all AWS S3 operations will be audited.
It's a YARN composite service which follows the standard lifecycle.
The actual auditor is instantiated initialized and started in its service
start phase; closed when the Audit Manager is stopped.
```java
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;
}
```
## Using Audit Spans within the S3A Connector
1. All public FS API calls must be marked as `@AuditEntryPoint` and initiate a span.
1. All interfaces which provided a subset of the store API to another class
(e.g. listing) MUST pick up the current span, store it, and activate/deactivate
the span when invoked. This ensures use across threads.
1. Methods/classes which operate across threads must store the audit span which
was active on their creation/invocation, and activate it in all threads which
interact with the FS. This should be automatic if callback interfaces
do this.
1. All S3 SDK request objects MUST be created in the request factory. Add new
methods if need be.
## Implementing a custom `OperationAuditor`
_This extension point is `@Unstable`_
```java
@InterfaceAudience.LimitedPrivate("S3A auditing extensions")
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.audit;
```
A custom `OperationAuditor` auditor is a class which implements the interface
`org.apache.hadoop.fs.s3a.audit.OperationAuditor`. This SHOULD be done by
subclassing
`org.apache.hadoop.fs.s3a.audit.AbstractOperationAuditor`.
It is a YARN service and follows the lifecycle:
configured in `serviceInit()`; start any worker threads/perform startup
operations in `serviceStart()` and shutdown in `serviceStop()`.
In use, it will be instantiated in `S3AFileSystem.initialize()`
and shutdown when the FS instance is closed.
It will be instantiated before the AWS S3 Client is built -it may provide a
request handler to be part of the handler chain of the S3 request pipeline.
It will be closed in the `FileSystem.close()` operation, after the S3 Client is
itself closed.
### Design Decisions/Review questions
### Why use https://audit.example.org/ as referrer host?
IETF requires *.example.org to be unresolvable through DNS, so with a well configured DNS there's never any host to probe.
It guarantees that there will never be real HTTP requests coming in from that host.
## And why `hadoop/1/` in the referrer path?
Provenance and versioning.
### Why no explicit end to an AuditSpan?
While most API calls have a bounded duration, e.g. `getFileStatus()`,
some calls have a very long lifespan (input and output streams).
List iterators are never formally terminated, they just "fall out of scope",
Thus, they'd never end.
Having a uniform "Audit Spans are never explicitly terminated" design
means that it is consistent everywhere.
### Can you activate an already active audit span?
It's a no-op.
It does mean that if you deactivate the span the first time, then the thread
reverts immediately to the unbonded span.
### Why does `AuditSpan.deactivate()` switches to the unbound span, rather than the span which was active before
Again, it gets complicated fast, especially when audit spans our shared across threads.
Because of the sharing you cannot store the previous span in a field within the AuditSpan itself.
Instead you need to have a thread local stack per FileSystem instance of active audit spans.
And you had better be confident that audit spans are correctly activated and deactivated,
with no span deactivated more than once -else the stack will become confused.
Having a simple "In Span" or "Out of Span" model avoids this problem.
However, it does prevent the S3A FileSystem implementation methods from
calling other methods which create new spans.
Hence the annotation of all span entry points as `@AuditEntryPoint` and
a need for rigorous review of the invocations.
As with the need to make sure that we never call retry() around a method tagged `@Retry`,
making sure that an audit entry point doesn't invoke another audit entry point
is going to become another piece of maintenance overhead.
### History
* 2021-02 Creation. [HADOOP-17511](https://issues.apache.org/jira/browse/HADOOP-17511) _Add an Audit plugin point for S3A auditing/context_.

View File

@ -44,6 +44,8 @@ full details.
* [Working with IAM Assumed Roles](./assumed_roles.html)
* [S3A Delegation Token Support](./delegation_tokens.html)
* [S3A Delegation Token Architecture](delegation_token_architecture.html).
* [Auditing](./auditing.html).
* [Auditing Architecture](./auditing_architecture.html).
* [Testing](./testing.html)
## <a name="overview"></a> Overview

Binary file not shown.

After

Width:  |  Height:  |  Size: 54 KiB

View File

@ -141,11 +141,11 @@ protected AbstractFSContract createContract(Configuration conf) {
@Override
public void teardown() throws Exception {
super.teardown();
S3AFileSystem fs = getFileSystem();
if (fs.getConf().getBoolean(FS_S3A_IMPL_DISABLE_CACHE, false)) {
fs.close();
}
super.teardown();
}
/**

View File

@ -27,6 +27,8 @@
import org.apache.hadoop.fs.contract.s3a.S3AContract;
import org.apache.hadoop.fs.s3a.tools.MarkerTool;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.fs.store.audit.AuditSpanSource;
import org.apache.hadoop.io.IOUtils;
import org.junit.AfterClass;
@ -59,6 +61,27 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS =
snapshotIOStatistics();
/**
* Source of audit spans.
*/
private AuditSpanSource spanSource;
/**
* Get the source.
* @return span source
*/
protected AuditSpanSource getSpanSource() {
return spanSource;
}
/**
* Set the span source.
* @param spanSource new value.
*/
protected void setSpanSource(final AuditSpanSource spanSource) {
this.spanSource = spanSource;
}
@Override
protected AbstractFSContract createContract(Configuration conf) {
return new S3AContract(conf, false);
@ -75,6 +98,7 @@ public void setup() throws Exception {
// static initializers. See: HADOOP-17385
S3AFileSystem.initializeClass();
super.setup();
setSpanSource(getFileSystem());
}
@Override
@ -213,4 +237,26 @@ protected void writeThenReadFile(Path path, int len) throws IOException {
protected String getTestTableName(String suffix) {
return getTestDynamoTablePrefix(getConfiguration()) + suffix;
}
/**
* Create a span from the source; returns a no-op if
* creation fails or the source is null.
* Uses the test method name for the span.
* @return a span.
*/
protected AuditSpan span() throws IOException {
return span(getSpanSource());
}
/**
* Create a span from the source; returns a no-op if
* creation fails or the source is null.
* Uses the test method name for the span.
* @param source source of spans; can be an S3A FS
* @return a span.
*/
protected AuditSpan span(AuditSpanSource source) throws IOException {
return source.createSpan(getMethodName(), null, null);
}
}

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.junit.Test;
@ -79,8 +80,10 @@ public void testDirectoryBecomesNonEmpty() throws Exception {
private S3AFileStatus getS3AFileStatus(S3AFileSystem fs, Path p) throws
IOException {
return fs.innerGetFileStatus(p, true,
StatusProbeEnum.ALL);
try (AuditSpan span = span()) {
return fs.innerGetFileStatus(p, true,
StatusProbeEnum.ALL);
}
}
}

View File

@ -29,6 +29,8 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport;
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.junit.Test;
import org.slf4j.Logger;
@ -78,7 +80,9 @@ public void testMultiObjectDeleteNoFile() throws Throwable {
private void removeKeys(S3AFileSystem fileSystem, String... keys)
throws IOException {
fileSystem.removeKeys(buildDeleteRequest(keys), false, null);
try (AuditSpan span = span()) {
fileSystem.removeKeys(buildDeleteRequest(keys), false, null);
}
}
private List<DeleteObjectsRequest.KeyVersion> buildDeleteRequest(
@ -119,6 +123,9 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable {
Path csvPath = maybeGetCsvPath();
S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem(
getConfiguration());
// create a span, expect it to be activated.
fs.getAuditSpanSource().createSpan(StoreStatisticNames.OP_DELETE,
csvPath.toString(), null);
List<DeleteObjectsRequest.KeyVersion> keys
= buildDeleteRequest(
new String[]{
@ -163,7 +170,9 @@ public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable {
S3AFileSystem fs = getFileSystem();
List<DeleteObjectsRequest.KeyVersion> keys = keysToDelete(
Lists.newArrayList(new Path(base, "1"), new Path(base, "2")));
fs.removeKeys(keys, false, null);
try (AuditSpan span = span()) {
fs.removeKeys(keys, false, null);
}
}
private String join(final Iterable iterable) {

View File

@ -38,6 +38,7 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.fs.store.EtagChecksum;
import org.apache.hadoop.test.LambdaTestUtils;
@ -111,16 +112,18 @@ public void testCreateNonRecursiveParentIsFile() throws IOException {
@Test
public void testPutObjectDirect() throws Throwable {
final S3AFileSystem fs = getFileSystem();
ObjectMetadata metadata = fs.newObjectMetadata(-1);
metadata.setContentLength(-1);
Path path = path("putDirect");
final PutObjectRequest put = new PutObjectRequest(fs.getBucket(),
path.toUri().getPath(),
new ByteArrayInputStream("PUT".getBytes()),
metadata);
LambdaTestUtils.intercept(IllegalStateException.class,
() -> fs.putObjectDirect(put));
assertPathDoesNotExist("put object was created", path);
try (AuditSpan span = span()) {
ObjectMetadata metadata = fs.newObjectMetadata(-1);
metadata.setContentLength(-1);
Path path = path("putDirect");
final PutObjectRequest put = new PutObjectRequest(fs.getBucket(),
path.toUri().getPath(),
new ByteArrayInputStream("PUT".getBytes()),
metadata);
LambdaTestUtils.intercept(IllegalStateException.class,
() -> fs.putObjectDirect(put));
assertPathDoesNotExist("put object was created", path);
}
}
private FSDataOutputStream createNonRecursive(Path path) throws IOException {

View File

@ -21,6 +21,8 @@
import com.amazonaws.services.s3.model.MultipartUpload;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.junit.Test;
import java.io.IOException;
@ -57,7 +59,7 @@ protected Configuration createConfiguration() {
public void testListMultipartUploads() throws Exception {
S3AFileSystem fs = getFileSystem();
Set<MultipartTestUtils.IdKey> keySet = new HashSet<>();
try {
try (AuditSpan span = span()) {
// 1. Create NUM_KEYS pending upload parts
for (int i = 0; i < NUM_KEYS; i++) {
Path filePath = getPartFilename(i);

View File

@ -1352,7 +1352,7 @@ private void skipIfVersionPolicyAndNoVersionId(Path testpath)
// enabled
Assume.assumeTrue(
"Target filesystem does not support versioning",
fs.getObjectMetadata(fs.pathToKey(testpath)).getVersionId() != null);
fs.getObjectMetadata(testpath).getVersionId() != null);
}
}

View File

@ -20,14 +20,12 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.util.stream.Stream;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.ListObjectsV2Request;
import com.amazonaws.services.s3.model.ListObjectsV2Result;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import org.assertj.core.api.Assertions;
import org.junit.Test;
@ -40,9 +38,11 @@
import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome;
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_MKDIRS;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeFilesystemHasMetadatastore;
@ -73,7 +73,9 @@ public void testRenameEmptyDir() throws Throwable {
String destDirMarker = fs.pathToKey(destDir) + "/";
// set things up.
mkdirs(sourceDir);
// there's source directory marker
// create a span for all the low level operations
span();
// there's source directory marker+
fs.getObjectMetadata(sourceDirMarker);
S3AFileStatus srcStatus = getEmptyDirStatus(sourceDir);
assertEquals("Must be an empty dir: " + srcStatus, Tristate.TRUE,
@ -89,6 +91,7 @@ public void testRenameEmptyDir() throws Throwable {
// and verify that there's no dir marker hidden under a tombstone
intercept(FileNotFoundException.class,
() -> Invoker.once("HEAD", sourceDirMarker, () -> {
span();
ObjectMetadata md = fs.getObjectMetadata(sourceDirMarker);
return String.format("Object %s of length %d",
sourceDirMarker, md.getInstanceLength());
@ -99,11 +102,14 @@ public void testRenameEmptyDir() throws Throwable {
assertEquals("Must not be an empty dir: " + baseStatus, Tristate.FALSE,
baseStatus.isEmptyDirectory());
// and verify the dest dir has a marker
span();
fs.getObjectMetadata(destDirMarker);
}
private S3AFileStatus getEmptyDirStatus(Path dir) throws IOException {
return getFileSystem().innerGetFileStatus(dir, true, StatusProbeEnum.ALL);
try (AuditSpan span = span()) {
return getFileSystem().innerGetFileStatus(dir, true, StatusProbeEnum.ALL);
}
}
@Test
@ -128,7 +134,7 @@ public void testEmptyDirs() throws Exception {
fs.setMetadataStore(configuredMs); // "start cluster"
Path newFile = path("existing-dir/new-file");
touch(fs, newFile);
span();
S3AFileStatus status = fs.innerGetFileStatus(existingDir, true,
StatusProbeEnum.ALL);
assertEquals("Should not be empty dir", Tristate.FALSE,
@ -137,6 +143,7 @@ public void testEmptyDirs() throws Exception {
// 3. Assert that removing the only file the MetadataStore witnessed
// being created doesn't cause it to think the directory is now empty.
fs.delete(newFile, false);
span();
status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL);
assertEquals("Should not be empty dir", Tristate.FALSE,
status.isEmptyDirectory());
@ -144,6 +151,7 @@ public void testEmptyDirs() throws Exception {
// 4. Assert that removing the final file, that existed "before"
// MetadataStore started, *does* cause the directory to be marked empty.
fs.delete(existingFile, false);
span();
status = fs.innerGetFileStatus(existingDir, true, StatusProbeEnum.ALL);
assertEquals("Should be empty dir now", Tristate.TRUE,
status.isEmptyDirectory());
@ -198,11 +206,9 @@ public void testTombstonesAndEmptyDirectories() throws Throwable {
createEmptyObject(fs, childKey);
// Do a list
ListObjectsV2Request listReq = new ListObjectsV2Request()
.withBucketName(bucket)
.withPrefix(baseKey)
.withMaxKeys(10)
.withDelimiter("/");
span();
ListObjectsV2Request listReq = ctx.getRequestFactory()
.newListObjectsV2Request(baseKey, "/", 10);
ListObjectsV2Result listing = s3.listObjectsV2(listReq);
// the listing has the first path as a prefix, because of the child
@ -243,6 +249,7 @@ public void testTombstonesAndEmptyDirectories() throws Throwable {
} finally {
// try to recover from the defective state.
span();
s3.deleteObject(bucket, childKey);
fs.delete(lastPath, true);
ddbMs.forgetMetadata(firstPath);
@ -272,19 +279,13 @@ private DynamoDBMetadataStore getRequiredDDBMetastore(S3AFileSystem fs) {
* @param fs filesystem
* @param key key
*/
private void createEmptyObject(S3AFileSystem fs, String key) {
final InputStream im = new InputStream() {
@Override
public int read() {
return -1;
}
};
private void createEmptyObject(S3AFileSystem fs, String key)
throws IOException {
PutObjectRequest putObjectRequest = fs.newPutObjectRequest(key,
fs.newObjectMetadata(0L),
im);
AmazonS3 s3 = fs.getAmazonS3ClientForTesting("PUT");
s3.putObject(putObjectRequest);
try (AuditSpan span = fs.getAuditSpanSource()
.createSpan(INVOCATION_MKDIRS.getSymbol(), key, null)) {
fs.createMkdirOperationCallbacks().createFakeDirectory(key);
}
}
@Test

Some files were not shown because too many files have changed in this diff Show More