HADOOP-19221. S3A: Unable to recover from failure of multipart block upload attempt (#6938)

This is a major change which handles 400 error responses when uploading
large files from memory heap/buffer (or staging committer) and the remote S3
store returns a 500 response from a upload of a block in a multipart upload.

The SDK's own streaming code seems unable to fully replay the upload;
at attempts to but then blocks and the S3 store returns a 400 response

    "Your socket connection to the server was not read from or written to
     within the timeout period. Idle connections will be closed.
     (Service: S3, Status Code: 400...)"

There is an option to control whether or not the S3A client itself
attempts to retry on a 50x error other than 503 throttling events
(which are independently processed as before)

Option:  fs.s3a.retry.http.5xx.errors
Default: true

500 errors are very rare from standard AWS S3, which has a five nines
SLA. It may be more common against S3 Express which has lower
guarantees.

Third party stores have unknown guarantees, and the exception may
indicate a bad server configuration. Consider setting
fs.s3a.retry.http.5xx.errors to false when working with
such stores.

Signification Code changes:

There is now a custom set of implementations of
software.amazon.awssdk.http.ContentStreamProvidercontent in
the class org.apache.hadoop.fs.s3a.impl.UploadContentProviders.

These:

* Restart on failures
* Do not copy buffers/byte buffers into new private byte arrays,
  so avoid exacerbating memory problems..

There new IOStatistics for specific http error codes -these are collected
even when all recovery is performed within the SDK.
  
S3ABlockOutputStream has major changes, including handling of
Thread.interrupt() on the main thread, which now triggers and briefly
awaits cancellation of any ongoing uploads.

If the writing thread is interrupted in close(), it is mapped to
an InterruptedIOException. Applications like Hive and Spark must
catch these after cancelling a worker thread.

Contributed by Steve Loughran
This commit is contained in:
Steve Loughran 2024-09-13 20:02:14 +01:00 committed by GitHub
parent c835adb3a8
commit ea6e0f7cd5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
53 changed files with 4252 additions and 987 deletions

View File

@ -384,6 +384,47 @@ public final class StoreStatisticNames {
public static final String ACTION_HTTP_PATCH_REQUEST
= "action_http_patch_request";
/**
* HTTP error response: {@value}.
*/
public static final String HTTP_RESPONSE_400
= "http_response_400";
/**
* HTTP error response: {@value}.
* Returned by some stores for throttling events.
*/
public static final String HTTP_RESPONSE_429
= "http_response_429";
/**
* Other 4XX HTTP response: {@value}.
* (404 responses are excluded as they are rarely 'errors'
* and will be reported differently if they are.
*/
public static final String HTTP_RESPONSE_4XX
= "http_response_4XX";
/**
* HTTP error response: {@value}.
* Sign of server-side problems, possibly transient
*/
public static final String HTTP_RESPONSE_500
= "http_response_500";
/**
* HTTP error response: {@value}.
* AWS Throttle.
*/
public static final String HTTP_RESPONSE_503
= "http_response_503";
/**
* Other 5XX HTTP response: {@value}.
*/
public static final String HTTP_RESPONSE_5XX
= "http_response_5XX";
/**
* An HTTP POST request was made: {@value}.
*/

View File

@ -0,0 +1,199 @@
/*
* 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;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.util.Preconditions;
/**
* Provide an input stream from a byte buffer; supporting
* {@link #mark(int)}.
*/
public final class ByteBufferInputStream extends InputStream {
private static final Logger LOG =
LoggerFactory.getLogger(ByteBufferInputStream.class);
/** Size of the buffer. */
private final int size;
/**
* Not final so that in close() it will be set to null, which
* may result in faster cleanup of the buffer.
*/
private ByteBuffer byteBuffer;
public ByteBufferInputStream(int size,
ByteBuffer byteBuffer) {
LOG.debug("Creating ByteBufferInputStream of size {}", size);
this.size = size;
this.byteBuffer = byteBuffer;
}
/**
* After the stream is closed, set the local reference to the byte
* buffer to null; this guarantees that future attempts to use
* stream methods will fail.
*/
@Override
public synchronized void close() {
LOG.debug("ByteBufferInputStream.close()");
byteBuffer = null;
}
/**
* Is the stream open?
* @return true if the stream has not been closed.
*/
public synchronized boolean isOpen() {
return byteBuffer != null;
}
/**
* Verify that the stream is open.
* @throws IOException if the stream is closed
*/
private void verifyOpen() throws IOException {
if (byteBuffer == null) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
}
/**
* Check the open state.
* @throws IllegalStateException if the stream is closed.
*/
private void checkOpenState() {
Preconditions.checkState(isOpen(),
FSExceptionMessages.STREAM_IS_CLOSED);
}
public synchronized int read() throws IOException {
if (available() > 0) {
return byteBuffer.get() & 0xFF;
} else {
return -1;
}
}
@Override
public synchronized long skip(long offset) throws IOException {
verifyOpen();
long newPos = position() + offset;
if (newPos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
}
if (newPos > size) {
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
}
byteBuffer.position((int) newPos);
return newPos;
}
@Override
public synchronized int available() {
checkOpenState();
return byteBuffer.remaining();
}
/**
* Get the current buffer position.
* @return the buffer position
*/
public synchronized int position() {
checkOpenState();
return byteBuffer.position();
}
/**
* Check if there is data left.
* @return true if there is data remaining in the buffer.
*/
public synchronized boolean hasRemaining() {
checkOpenState();
return byteBuffer.hasRemaining();
}
@Override
public synchronized void mark(int readlimit) {
LOG.debug("mark at {}", position());
checkOpenState();
byteBuffer.mark();
}
@Override
public synchronized void reset() throws IOException {
LOG.debug("reset");
checkOpenState();
byteBuffer.reset();
}
@Override
public boolean markSupported() {
return true;
}
/**
* Read in data.
* @param b destination buffer.
* @param offset offset within the buffer.
* @param length length of bytes to read.
* @throws EOFException if the position is negative
* @throws IndexOutOfBoundsException if there isn't space for the
* amount of data requested.
* @throws IllegalArgumentException other arguments are invalid.
*/
@SuppressWarnings("NullableProblems")
public synchronized int read(byte[] b, int offset, int length)
throws IOException {
Preconditions.checkArgument(length >= 0, "length is negative");
Preconditions.checkArgument(b != null, "Null buffer");
if (b.length - offset < length) {
throw new IndexOutOfBoundsException(
FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+ ": request length =" + length
+ ", with offset =" + offset
+ "; buffer capacity =" + (b.length - offset));
}
verifyOpen();
if (!hasRemaining()) {
return -1;
}
int toRead = Math.min(length, available());
byteBuffer.get(b, offset, toRead);
return toRead;
}
@Override
public String toString() {
return "ByteBufferInputStream{" +
"size=" + size +
", byteBuffer=" + byteBuffer +
((byteBuffer != null) ? ", available=" + byteBuffer.remaining() : "") +
"} " + super.toString();
}
}

View File

@ -22,7 +22,6 @@
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
import java.io.EOFException;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
@ -40,7 +39,6 @@
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.DirectBufferPool;
@ -777,158 +775,8 @@ public String toString() {
'}';
}
/**
* Provide an input stream from a byte buffer; supporting
* {@link #mark(int)}, which is required to enable replay of failed
* PUT attempts.
*/
class ByteBufferInputStream extends InputStream {
private final int size;
private ByteBuffer byteBuffer;
ByteBufferInputStream(int size,
ByteBuffer byteBuffer) {
LOG.debug("Creating ByteBufferInputStream of size {}", size);
this.size = size;
this.byteBuffer = byteBuffer;
}
/**
* After the stream is closed, set the local reference to the byte
* buffer to null; this guarantees that future attempts to use
* stream methods will fail.
*/
@Override
public synchronized void close() {
LOG.debug("ByteBufferInputStream.close() for {}",
ByteBufferBlock.super.toString());
byteBuffer = null;
}
/**
* Verify that the stream is open.
*
* @throws IOException if the stream is closed
*/
private void verifyOpen() throws IOException {
if (byteBuffer == null) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
}
public synchronized int read() throws IOException {
if (available() > 0) {
return byteBuffer.get() & 0xFF;
} else {
return -1;
}
}
@Override
public synchronized long skip(long offset) throws IOException {
verifyOpen();
long newPos = position() + offset;
if (newPos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
}
if (newPos > size) {
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
}
byteBuffer.position((int) newPos);
return newPos;
}
@Override
public synchronized int available() {
Preconditions.checkState(byteBuffer != null,
FSExceptionMessages.STREAM_IS_CLOSED);
return byteBuffer.remaining();
}
/**
* Get the current buffer position.
*
* @return the buffer position
*/
public synchronized int position() {
return byteBuffer.position();
}
/**
* Check if there is data left.
*
* @return true if there is data remaining in the buffer.
*/
public synchronized boolean hasRemaining() {
return byteBuffer.hasRemaining();
}
@Override
public synchronized void mark(int readlimit) {
LOG.debug("mark at {}", position());
byteBuffer.mark();
}
@Override
public synchronized void reset() throws IOException {
LOG.debug("reset");
byteBuffer.reset();
}
@Override
public boolean markSupported() {
return true;
}
/**
* Read in data.
*
* @param b destination buffer.
* @param offset offset within the buffer.
* @param length length of bytes to read.
* @throws EOFException if the position is negative
* @throws IndexOutOfBoundsException if there isn't space for the
* amount of data requested.
* @throws IllegalArgumentException other arguments are invalid.
*/
@SuppressWarnings("NullableProblems")
public synchronized int read(byte[] b, int offset, int length)
throws IOException {
Preconditions.checkArgument(length >= 0, "length is negative");
Preconditions.checkArgument(b != null, "Null buffer");
if (b.length - offset < length) {
throw new IndexOutOfBoundsException(
FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+ ": request length =" + length
+ ", with offset =" + offset
+ "; buffer capacity =" + (b.length - offset));
}
verifyOpen();
if (!hasRemaining()) {
return -1;
}
int toRead = Math.min(length, available());
byteBuffer.get(b, offset, toRead);
return toRead;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"ByteBufferInputStream{");
sb.append("size=").append(size);
ByteBuffer buf = this.byteBuffer;
if (buf != null) {
sb.append(", available=").append(buf.remaining());
}
sb.append(", ").append(ByteBufferBlock.super.toString());
sb.append('}');
return sb.toString();
}
}
}
}
// ====================================================================
@ -1124,4 +972,5 @@ void closeBlock() {
}
}
}
}

View File

@ -26,6 +26,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
@ -33,10 +34,14 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSBuilder;
import org.apache.hadoop.util.Time;
/**
* Future IO Helper methods.
@ -53,12 +58,18 @@
* {@code UncheckedIOException} raised in the future.
* This makes it somewhat easier to execute IOException-raising
* code inside futures.
* </p>
* <p>
* Important: any {@code CancellationException} raised by the future
* is rethrown unchanged. This has been the implicit behavior since
* this code was first written, and is now explicitly documented.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class FutureIO {
private static final Logger LOG =
LoggerFactory.getLogger(FutureIO.class);
private FutureIO() {
}
@ -68,17 +79,28 @@ private FutureIO() {
* Any exception generated in the future is
* extracted and rethrown.
* </p>
* If this thread is interrupted while waiting for the future to complete,
* an {@code InterruptedIOException} is raised.
* However, if the future is cancelled, a {@code CancellationException}
* is raised in the {code Future.get()} call. This is
* passed up as is -so allowing the caller to distinguish between
* thread interruption (such as when speculative task execution is aborted)
* and future cancellation.
* @param future future to evaluate
* @param <T> type of the result.
* @return the result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws InterruptedIOException waiting for future completion was interrupted
* @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
public static <T> T awaitFuture(final Future<T> future)
throws InterruptedIOException, IOException, RuntimeException {
throws InterruptedIOException, IOException, CancellationException, RuntimeException {
try {
return future.get();
} catch (CancellationException e) {
LOG.debug("Future {} was cancelled", future, e);
throw e;
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
@ -94,11 +116,12 @@ public static <T> T awaitFuture(final Future<T> future)
* extracted and rethrown.
* </p>
* @param future future to evaluate
* @param timeout timeout to wait
* @param timeout timeout to wait.
* @param unit time unit.
* @param <T> type of the result.
* @return the result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws InterruptedIOException waiting for future completion was interrupted
* @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
* @throws TimeoutException the future timed out.
@ -106,10 +129,13 @@ public static <T> T awaitFuture(final Future<T> future)
public static <T> T awaitFuture(final Future<T> future,
final long timeout,
final TimeUnit unit)
throws InterruptedIOException, IOException, RuntimeException,
throws InterruptedIOException, IOException, CancellationException, RuntimeException,
TimeoutException {
try {
return future.get(timeout, unit);
} catch (CancellationException e) {
LOG.debug("Future {} was cancelled", future, e);
throw e;
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
@ -128,12 +154,13 @@ public static <T> T awaitFuture(final Future<T> future,
* @param collection collection of futures to be evaluated
* @param <T> type of the result.
* @return the list of future's result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws InterruptedIOException waiting for future completion was interrupted
* @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
public static <T> List<T> awaitAllFutures(final Collection<Future<T>> collection)
throws InterruptedIOException, IOException, RuntimeException {
throws InterruptedIOException, IOException, CancellationException, RuntimeException {
List<T> results = new ArrayList<>();
for (Future<T> future : collection) {
results.add(awaitFuture(future));
@ -148,19 +175,19 @@ public static <T> List<T> awaitAllFutures(final Collection<Future<T>> collection
* This method blocks until all futures in the collection have completed or
* the timeout expires, whichever happens first. If any future throws an
* exception during its execution, this method extracts and rethrows that exception.
* </p>
* @param collection collection of futures to be evaluated
* @param duration timeout duration
* @param <T> type of the result.
* @return the list of future's result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws InterruptedIOException waiting for future completion was interrupted
* @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
* @throws TimeoutException the future timed out.
*/
public static <T> List<T> awaitAllFutures(final Collection<Future<T>> collection,
final Duration duration)
throws InterruptedIOException, IOException, RuntimeException,
throws InterruptedIOException, IOException, CancellationException, RuntimeException,
TimeoutException {
List<T> results = new ArrayList<>();
for (Future<T> future : collection) {
@ -169,6 +196,48 @@ public static <T> List<T> awaitAllFutures(final Collection<Future<T>> collection
return results;
}
/**
* Cancels a collection of futures and awaits the specified duration for their completion.
* <p>
* This method blocks until all futures in the collection have completed or
* the timeout expires, whichever happens first.
* All exceptions thrown by the futures are ignored. as is any TimeoutException.
* @param collection collection of futures to be evaluated
* @param interruptIfRunning should the cancel interrupt any active futures?
* @param duration total timeout duration
* @param <T> type of the result.
* @return all futures which completed successfully.
*/
public static <T> List<T> cancelAllFuturesAndAwaitCompletion(
final Collection<Future<T>> collection,
final boolean interruptIfRunning,
final Duration duration) {
for (Future<T> future : collection) {
future.cancel(interruptIfRunning);
}
// timeout is relative to the start of the operation
long timeout = duration.toMillis();
List<T> results = new ArrayList<>();
for (Future<T> future : collection) {
long start = Time.now();
try {
results.add(awaitFuture(future, timeout, TimeUnit.MILLISECONDS));
} catch (CancellationException | IOException | TimeoutException e) {
// swallow
LOG.debug("Ignoring exception of cancelled future", e);
}
// measure wait time and reduce timeout accordingly
long waited = Time.now() - start;
timeout -= waited;
if (timeout < 0) {
// very brief timeout always
timeout = 0;
}
}
return results;
}
/**
* From the inner cause of an execution exception, extract the inner cause
* if it is an IOE or RTE.

View File

@ -22,12 +22,19 @@
/**
* A 5xx response came back from a service.
* The 500 error considered retriable by the AWS SDK, which will have already
* tried it {@code fs.s3a.attempts.maximum} times before reaching s3a
* <p>
* The 500 error is considered retryable by the AWS SDK, which will have already
* retried it {@code fs.s3a.attempts.maximum} times before reaching s3a
* code.
* How it handles other 5xx errors is unknown: S3A FS code will treat them
* as unrecoverable on the basis that they indicate some third-party store
* or gateway problem.
* <p>
* These are rare, but can occur; they are considered retryable.
* Note that HADOOP-19221 shows a failure condition where the
* SDK itself did not recover on retry from the error.
* In S3A code, retries happen if the retry policy configuration
* {@code fs.s3a.retry.http.5xx.errors} is {@code true}.
* <p>
* In third party stores it may have a similar meaning -though it
* can often just mean "misconfigured server".
*/
public class AWSStatus500Exception extends AWSServiceIOException {
public AWSStatus500Exception(String operation,
@ -35,8 +42,4 @@ public AWSStatus500Exception(String operation,
super(operation, cause);
}
@Override
public boolean retryable() {
return false;
}
}

View File

@ -1129,6 +1129,22 @@ private Constants() {
*/
public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms";
/**
* Should S3A connector retry on all 5xx errors which don't have
* explicit support: {@value}?
* <p>
* This is in addition to any retries the AWS SDK itself does, which
* is known to retry on many of these (e.g. 500).
*/
public static final String RETRY_HTTP_5XX_ERRORS =
"fs.s3a.retry.http.5xx.errors";
/**
* Default value for {@link #RETRY_HTTP_5XX_ERRORS}: {@value}.
*/
public static final boolean DEFAULT_RETRY_HTTP_5XX_ERRORS = true;
/**
* Should etags be exposed as checksums?
*/

View File

@ -29,21 +29,21 @@
*/
public class ProgressableProgressListener implements TransferListener {
private static final Logger LOG = S3AFileSystem.LOG;
private final S3AFileSystem fs;
private final S3AStore store;
private final String key;
private final Progressable progress;
private long lastBytesTransferred;
/**
* Instantiate.
* @param fs filesystem: will be invoked with statistics updates
* @param store store: will be invoked with statistics updates
* @param key key for the upload
* @param progress optional callback for progress.
*/
public ProgressableProgressListener(S3AFileSystem fs,
public ProgressableProgressListener(S3AStore store,
String key,
Progressable progress) {
this.fs = fs;
this.store = store;
this.key = key;
this.progress = progress;
this.lastBytesTransferred = 0;
@ -51,12 +51,12 @@ public ProgressableProgressListener(S3AFileSystem fs,
@Override
public void transferInitiated(TransferListener.Context.TransferInitiated context) {
fs.incrementWriteOperations();
store.incrementWriteOperations();
}
@Override
public void transferComplete(TransferListener.Context.TransferComplete context) {
fs.incrementWriteOperations();
store.incrementWriteOperations();
}
@Override
@ -68,7 +68,7 @@ public void bytesTransferred(TransferListener.Context.BytesTransferred context)
long transferred = context.progressSnapshot().transferredBytes();
long delta = transferred - lastBytesTransferred;
fs.incrementPutProgressStatistics(key, delta);
store.incrementPutProgressStatistics(key, delta);
lastBytesTransferred = transferred;
}
@ -84,7 +84,7 @@ public long uploadCompleted(ObjectTransfer upload) {
upload.progress().snapshot().transferredBytes() - lastBytesTransferred;
if (delta > 0) {
LOG.debug("S3A write delta changed after finished: {} bytes", delta);
fs.incrementPutProgressStatistics(key, delta);
store.incrementPutProgressStatistics(key, delta);
}
return delta;
}

View File

@ -18,33 +18,41 @@
package org.apache.hadoop.fs.s3a;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.StringJoiner;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import software.amazon.awssdk.core.exception.SdkException;
import javax.annotation.Nonnull;
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.model.CompletedPart;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.ClosedIOException;
import org.apache.hadoop.fs.s3a.impl.ProgressListener;
import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
import org.apache.hadoop.util.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
@ -68,25 +76,59 @@
import org.apache.hadoop.util.Progressable;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.*;
import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.util.functional.FutureIO.awaitAllFutures;
import static org.apache.hadoop.util.functional.FutureIO.cancelAllFuturesAndAwaitCompletion;
/**
* Upload files/parts directly via different buffering mechanisms:
* including memory and disk.
* <p>
* Key Features
* <ol>
* <li>Support single/multipart uploads</li>
* <li>Multiple buffering options</li>
* <li>Magic files are uploaded but not completed</li>
* <li>Implements {@link Abortable} API</li>
* <li>Doesn't implement {@link Syncable}; whether to ignore or reject calls is configurable</li>a
* <li>When multipart uploads are triggered, will queue blocks for asynchronous uploads</li>
* <li>Provides progress information to any supplied {@link Progressable} callback,
* during async uploads and in the {@link #close()} operation.</li>
* <li>If a {@link Progressable} passed in to the create() call implements
* {@link ProgressListener}, it will get detailed callbacks on internal events.
* Important: these may come from different threads.
* </li>
*
* If the stream is closed and no update has started, then the upload
* is instead done as a single PUT operation.
*
* Unstable: statistics and error handling might evolve.
*
* </ol>
* This class is best described as "complicated".
* <ol>
* <li>For "normal" files, data is buffered until either of:
* the limit of {@link #blockSize} is reached or the stream is closed.
* </li>
* <li>If if there are any problems call mukund</li>
* </ol>
* <p>
* The upload will not be completed until {@link #close()}, and
* then only if {@link PutTracker#outputImmediatelyVisible()} is true.
* <p>
* If less than a single block of data has been written before {@code close()}
* then it will uploaded as a single PUT (non-magic files), otherwise
* (larger files, magic files) a multipart upload is initiated and blocks
* uploaded as the data accrued reaches the block size.
* <p>
* The {@code close()} call blocks until all uploads have been completed.
* This may be a slow operation: progress callbacks are made during this
* process to reduce the risk of timeouts.
* <p>
* Syncable is declared as supported so the calls can be
* explicitly rejected.
* explicitly rejected if the filesystem is configured to do so.
* <p>
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
@ -99,6 +141,12 @@ class S3ABlockOutputStream extends OutputStream implements
private static final String E_NOT_SYNCABLE =
"S3A streams are not Syncable. See HADOOP-17597.";
/**
* How long to wait for uploads to complete after being cancelled before
* the blocks themselves are closed: 15 seconds.
*/
private static final Duration TIME_TO_AWAIT_CANCEL_COMPLETION = Duration.ofSeconds(15);
/** Object being uploaded. */
private final String key;
@ -178,8 +226,16 @@ class S3ABlockOutputStream extends OutputStream implements
* An S3A output stream which uploads partitions in a separate pool of
* threads; different {@link S3ADataBlocks.BlockFactory}
* instances can control where data is buffered.
* @throws IOException on any problem
* If the passed in put tracker returns true on
* {@link PutTracker#initialize()} then a multipart upload is
* initiated; this triggers a remote call to the store.
* On a normal upload no such operation takes place; the only
* failures which surface will be related to buffer creation.
* @throws IOException on any problem initiating a multipart upload or creating
* a disk storage buffer.
* @throws OutOfMemoryError lack of space to create any memory buffer
*/
@Retries.RetryTranslated
S3ABlockOutputStream(BlockOutputStreamBuilder builder)
throws IOException {
builder.validate();
@ -224,7 +280,8 @@ class S3ABlockOutputStream extends OutputStream implements
/**
* Demand create a destination block.
* @return the active block; null if there isn't one.
* @throws IOException on any failure to create
* @throws IOException any failure to create a block in the local FS.
* @throws OutOfMemoryError lack of space to create any memory buffer
*/
private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded()
throws IOException {
@ -268,12 +325,13 @@ private void clearActiveBlock() {
}
/**
* Check for the filesystem being open.
* @throws IOException if the filesystem is closed.
* Check for the stream being open.
* @throws ClosedIOException if the stream is closed.
*/
void checkOpen() throws IOException {
@VisibleForTesting
void checkOpen() throws ClosedIOException {
if (closed.get()) {
throw new IOException("Filesystem " + writeOperationHelper + " closed");
throw new ClosedIOException(key, "Stream is closed: " + this);
}
}
@ -281,14 +339,17 @@ void checkOpen() throws IOException {
* The flush operation does not trigger an upload; that awaits
* the next block being full. What it does do is call {@code flush() }
* on the current block, leaving it to choose how to react.
* @throws IOException Any IO problem.
* <p>
* If the stream is closed, a warning is logged but the exception
* is swallowed.
* @throws IOException Any IO problem flushing the active data block.
*/
@Override
public synchronized void flush() throws IOException {
try {
checkOpen();
} catch (IOException e) {
LOG.warn("Stream closed: " + e.getMessage());
} catch (ClosedIOException e) {
LOG.warn("Stream closed: {}", e.getMessage());
return;
}
S3ADataBlocks.DataBlock dataBlock = getActiveBlock();
@ -314,13 +375,17 @@ public synchronized void write(int b) throws IOException {
* buffer to reach its limit, the actual upload is submitted to the
* threadpool and the remainder of the array is written to memory
* (recursively).
* In such a case, if not already initiated, a multipart upload is
* started.
* @param source byte array containing
* @param offset offset in array where to start
* @param len number of bytes to be written
* @throws IOException on any problem
* @throws ClosedIOException if the stream is closed.
*/
@Override
public synchronized void write(byte[] source, int offset, int len)
@Retries.RetryTranslated
public synchronized void write(@Nonnull byte[] source, int offset, int len)
throws IOException {
S3ADataBlocks.validateWriteArgs(source, offset, len);
@ -400,20 +465,23 @@ private void initMultipartUpload() throws IOException {
/**
* Close the stream.
*
* <p>
* This will not return until the upload is complete
* or the attempt to perform the upload has failed.
* or the attempt to perform the upload has failed or been interrupted.
* Exceptions raised in this method are indicative that the write has
* failed and data is at risk of being lost.
* @throws IOException on any failure.
* @throws InterruptedIOException if the wait for uploads to complete was interrupted.
*/
@Override
@Retries.RetryTranslated
public void close() throws IOException {
if (closed.getAndSet(true)) {
// already closed
LOG.debug("Ignoring close() as stream is already closed");
return;
}
progressListener.progressChanged(CLOSE_EVENT, 0);
S3ADataBlocks.DataBlock block = getActiveBlock();
boolean hasBlock = hasActiveBlock();
LOG.debug("{}: Closing block #{}: current block= {}",
@ -431,7 +499,7 @@ public void close() throws IOException {
bytesSubmitted = bytes;
}
} else {
// there's an MPU in progress';
// there's an MPU in progress
// IF there is more data to upload, or no data has yet been uploaded,
// PUT the final block
if (hasBlock &&
@ -440,13 +508,17 @@ public void close() throws IOException {
// Necessary to set this "true" in case of client side encryption.
uploadCurrentBlock(true);
}
// wait for the partial uploads to finish
// wait for the part uploads to finish
// this may raise CancellationException as well as any IOE.
final List<CompletedPart> partETags =
multiPartUpload.waitForAllPartUploads();
bytes = bytesSubmitted;
final String uploadId = multiPartUpload.getUploadId();
LOG.debug("Multipart upload to {} ID {} containing {} blocks",
key, uploadId, partETags.size());
// then complete the operation
if (putTracker.aboutToComplete(multiPartUpload.getUploadId(),
if (putTracker.aboutToComplete(uploadId,
partETags,
bytes,
iostatistics)) {
@ -468,6 +540,14 @@ public void close() throws IOException {
maybeAbortMultipart();
writeOperationHelper.writeFailed(ioe);
throw ioe;
} catch (CancellationException e) {
// waiting for the upload was cancelled.
// abort uploads
maybeAbortMultipart();
writeOperationHelper.writeFailed(e);
// and raise an InterruptedIOException
throw (IOException)(new InterruptedIOException(e.getMessage())
.initCause(e));
} finally {
cleanupOnClose();
}
@ -502,13 +582,19 @@ private void mergeThreadIOStatistics(IOStatistics streamStatistics) {
/**
* Best effort abort of the multipart upload; sets
* the field to null afterwards.
* @return any exception caught during the operation.
* <p>
* Cancels any active uploads on the first invocation.
* @return any exception caught during the operation. If FileNotFoundException
* it means the upload was not found.
*/
@Retries.RetryTranslated
private synchronized IOException maybeAbortMultipart() {
if (multiPartUpload != null) {
final IOException ioe = multiPartUpload.abort();
multiPartUpload = null;
return ioe;
try {
return multiPartUpload.abort();
} finally {
multiPartUpload = null;
}
} else {
return null;
}
@ -519,15 +605,25 @@ private synchronized IOException maybeAbortMultipart() {
* @return the outcome
*/
@Override
@Retries.RetryTranslated
public AbortableResult abort() {
if (closed.getAndSet(true)) {
// already closed
LOG.debug("Ignoring abort() as stream is already closed");
return new AbortableResultImpl(true, null);
}
// abort the upload.
// if not enough data has been written to trigger an upload: this is no-op.
// if a multipart had started: abort it by cancelling all active uploads
// and aborting the multipart upload on s3.
try (DurationTracker d =
statistics.trackDuration(INVOCATION_ABORT.getSymbol())) {
return new AbortableResultImpl(false, maybeAbortMultipart());
// abort. If the upload is not found, report as already closed.
final IOException anyCleanupException = maybeAbortMultipart();
return new AbortableResultImpl(
anyCleanupException instanceof FileNotFoundException,
anyCleanupException);
} finally {
cleanupOnClose();
}
@ -584,59 +680,45 @@ public String toString() {
* Upload the current block as a single PUT request; if the buffer is empty a
* 0-byte PUT will be invoked, as it is needed to create an entry at the far
* end.
* @return number of bytes uploaded. If thread was interrupted while waiting
* for upload to complete, returns zero with interrupted flag set on this
* thread.
* @throws IOException
* any problem.
* @return number of bytes uploaded.
* @throws IOException any problem.
*/
@Retries.RetryTranslated
private long putObject() throws IOException {
LOG.debug("Executing regular upload for {}", writeOperationHelper);
final S3ADataBlocks.DataBlock block = getActiveBlock();
long size = block.dataSize();
final long size = block.dataSize();
final S3ADataBlocks.BlockUploadData uploadData = block.startUpload();
final PutObjectRequest putObjectRequest = uploadData.hasFile() ?
final PutObjectRequest putObjectRequest =
writeOperationHelper.createPutObjectRequest(
key,
uploadData.getFile().length(),
builder.putOptions,
true)
: writeOperationHelper.createPutObjectRequest(
key,
size,
builder.putOptions,
false);
uploadData.getSize(),
builder.putOptions);
clearActiveBlock();
BlockUploadProgress progressCallback =
new BlockUploadProgress(block, progressListener, now());
statistics.blockUploadQueued(size);
ListenableFuture<PutObjectResponse> putObjectResult =
executorService.submit(() -> {
try {
// the putObject call automatically closes the input
// stream afterwards.
PutObjectResponse response =
writeOperationHelper.putObject(putObjectRequest, builder.putOptions, uploadData,
uploadData.hasFile(), statistics);
progressCallback.progressChanged(REQUEST_BYTE_TRANSFER_EVENT);
return response;
} finally {
cleanupWithLogger(LOG, uploadData, block);
}
});
clearActiveBlock();
//wait for completion
try {
putObjectResult.get();
return size;
} catch (InterruptedException ie) {
LOG.warn("Interrupted object upload", ie);
Thread.currentThread().interrupt();
return 0;
} catch (ExecutionException ee) {
throw extractException("regular upload", key, ee);
progressCallback.progressChanged(PUT_STARTED_EVENT);
// the putObject call automatically closes the upload data
writeOperationHelper.putObject(putObjectRequest,
builder.putOptions,
uploadData,
statistics);
progressCallback.progressChanged(REQUEST_BYTE_TRANSFER_EVENT);
progressCallback.progressChanged(PUT_COMPLETED_EVENT);
} catch (InterruptedIOException ioe){
progressCallback.progressChanged(PUT_INTERRUPTED_EVENT);
throw ioe;
} catch (IOException ioe){
progressCallback.progressChanged(PUT_FAILED_EVENT);
throw ioe;
} finally {
cleanupWithLogger(LOG, uploadData, block);
}
return size;
}
@Override
@ -731,6 +813,7 @@ public void hsync() throws IOException {
/**
* Shared processing of Syncable operation reporting/downgrade.
* @throws UnsupportedOperationException if required.
*/
private void handleSyncableInvocation() {
final UnsupportedOperationException ex
@ -763,12 +846,44 @@ protected IOStatisticsAggregator getThreadIOStatistics() {
* Multiple partition upload.
*/
private class MultiPartUpload {
/**
* ID of this upload.
*/
private final String uploadId;
private final List<ListenableFuture<CompletedPart>> partETagsFutures;
/**
* List of completed uploads, in order of blocks written.
*/
private final List<Future<CompletedPart>> partETagsFutures =
Collections.synchronizedList(new ArrayList<>());
/** blocks which need to be closed when aborting a stream. */
private final Map<Integer, S3ADataBlocks.DataBlock> blocksToClose =
new ConcurrentHashMap<>();
/**
* Count of parts submitted, including those queued.
*/
private int partsSubmitted;
/**
* Count of parts which have actually been uploaded.
*/
private int partsUploaded;
/**
* Count of bytes submitted.
*/
private long bytesSubmitted;
/**
* Has this upload been aborted?
* This value is checked when each future is executed.
* and to stop re-entrant attempts to abort an upload.
*/
private final AtomicBoolean uploadAborted = new AtomicBoolean(false);
/**
* Any IOException raised during block upload.
* if non-null, then close() MUST NOT complete
@ -782,7 +897,6 @@ private class MultiPartUpload {
* @param key upload destination
* @throws IOException failure
*/
@Retries.RetryTranslated
MultiPartUpload(String key) throws IOException {
this.uploadId = trackDuration(statistics,
@ -791,9 +905,9 @@ private class MultiPartUpload {
key,
builder.putOptions));
this.partETagsFutures = new ArrayList<>(2);
LOG.debug("Initiated multi-part upload for {} with " +
"id '{}'", writeOperationHelper, uploadId);
progressListener.progressChanged(TRANSFER_MULTIPART_INITIATED_EVENT, 0);
}
/**
@ -852,9 +966,13 @@ public void maybeRethrowUploadFailure() throws IOException {
/**
* Upload a block of data.
* This will take the block
* This will take the block and queue it for upload.
* There is no communication with S3 in this operation;
* it is all done in the asynchronous threads.
* @param block block to upload
* @throws IOException upload failure
* @param isLast this the last block?
* @throws IOException failure to initiate upload or a previous exception
* has been raised -which is then rethrown.
* @throws PathIOException if too many blocks were written
*/
private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
@ -862,33 +980,35 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
throws IOException {
LOG.debug("Queueing upload of {} for upload {}", block, uploadId);
Preconditions.checkNotNull(uploadId, "Null uploadId");
// if another upload has failed, throw it rather than try to submit
// a new upload
maybeRethrowUploadFailure();
partsSubmitted++;
final long size = block.dataSize();
bytesSubmitted += size;
final int currentPartNumber = partETagsFutures.size() + 1;
// this is the request which will be asynchronously uploaded
final UploadPartRequest request;
final S3ADataBlocks.BlockUploadData uploadData;
final RequestBody requestBody;
try {
uploadData = block.startUpload();
requestBody = uploadData.hasFile()
? RequestBody.fromFile(uploadData.getFile())
: RequestBody.fromInputStream(uploadData.getUploadStream(), size);
// get the content provider from the upload data; this allows
// different buffering mechanisms to provide their own
// implementations of efficient and recoverable content streams.
requestBody = RequestBody.fromContentProvider(
uploadData.getContentProvider(),
uploadData.getSize(),
CONTENT_TYPE_OCTET_STREAM);
request = writeOperationHelper.newUploadPartRequestBuilder(
key,
uploadId,
currentPartNumber,
size).build();
} catch (SdkException 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.
// failure to prepare the upload.
noteUploadFailure(e);
throw e;
}
@ -897,6 +1017,8 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
new BlockUploadProgress(block, progressListener, now());
statistics.blockUploadQueued(block.dataSize());
/* BEGIN: asynchronous upload */
ListenableFuture<CompletedPart> partETagFuture =
executorService.submit(() -> {
// this is the queued upload operation
@ -905,66 +1027,146 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
LOG.debug("Uploading part {} for id '{}'",
currentPartNumber, uploadId);
// update statistics
progressCallback.progressChanged(TRANSFER_PART_STARTED_EVENT);
if (uploadAborted.get()) {
// upload was cancelled; record as a failure
LOG.debug("Upload of part {} was cancelled", currentPartNumber);
progressCallback.progressChanged(TRANSFER_PART_ABORTED_EVENT);
// return stub entry.
return CompletedPart.builder()
.eTag("")
.partNumber(currentPartNumber)
.build();
}
// this is potentially slow.
// if the stream is aborted, this will be interrupted.
UploadPartResponse response = writeOperationHelper
.uploadPart(request, requestBody, statistics);
LOG.debug("Completed upload of {} to part {}",
LOG.debug("Completed upload of {} to with etag {}",
block, response.eTag());
LOG.debug("Stream statistics of {}", statistics);
partsUploaded++;
progressCallback.progressChanged(TRANSFER_PART_COMPLETED_EVENT);
progressCallback.progressChanged(TRANSFER_PART_SUCCESS_EVENT);
return CompletedPart.builder()
.eTag(response.eTag())
.partNumber(currentPartNumber)
.build();
} catch (IOException e) {
} catch (Exception e) {
final IOException ex = e instanceof IOException
? (IOException) e
: new IOException(e);
LOG.debug("Failed to upload part {}", currentPartNumber, ex);
// save immediately.
noteUploadFailure(e);
noteUploadFailure(ex);
progressCallback.progressChanged(TRANSFER_PART_FAILED_EVENT);
throw e;
throw ex;
} finally {
progressCallback.progressChanged(TRANSFER_PART_COMPLETED_EVENT);
// close the stream and block
cleanupWithLogger(LOG, uploadData, block);
LOG.debug("closing block");
completeUpload(currentPartNumber, block, uploadData);
}
});
/* END: asynchronous upload */
addSubmission(currentPartNumber, block, partETagFuture);
}
/**
* Add a submission to the list of active uploads and the map of
* blocks to close when interrupted.
* @param currentPartNumber part number
* @param block block
* @param partETagFuture queued upload
*/
private void addSubmission(
final int currentPartNumber,
final S3ADataBlocks.DataBlock block,
final ListenableFuture<CompletedPart> partETagFuture) {
partETagsFutures.add(partETagFuture);
blocksToClose.put(currentPartNumber, block);
}
/**
* Complete an upload.
* <p>
* This closes the block and upload data.
* It removes the block from {@link #blocksToClose}.
* @param currentPartNumber part number
* @param block block
* @param uploadData upload data
*/
private void completeUpload(
final int currentPartNumber,
final S3ADataBlocks.DataBlock block,
final S3ADataBlocks.BlockUploadData uploadData) {
// this may not actually be in the map if the upload executed
// before the relevant submission was noted
blocksToClose.remove(currentPartNumber);
cleanupWithLogger(LOG, uploadData);
cleanupWithLogger(LOG, block);
}
/**
* Block awaiting all outstanding uploads to complete.
* @return list of results
* Any interruption of this thread or a failure in an upload will
* trigger cancellation of pending uploads and an abort of the MPU.
* @return list of results or null if interrupted.
* @throws CancellationException waiting for the uploads to complete was cancelled
* @throws IOException IO Problems
*/
private List<CompletedPart> waitForAllPartUploads() throws IOException {
private List<CompletedPart> waitForAllPartUploads()
throws CancellationException, IOException {
LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size());
try {
return Futures.allAsList(partETagsFutures).get();
} catch (InterruptedException ie) {
LOG.warn("Interrupted partUpload", ie);
Thread.currentThread().interrupt();
return null;
} catch (ExecutionException ee) {
//there is no way of recovering so abort
//cancel all partUploads
LOG.debug("While waiting for upload completion", ee);
//abort multipartupload
this.abort();
throw extractException("Multi-part upload with id '" + uploadId
+ "' to " + key, key, ee);
// wait for the uploads to finish in order.
final List<CompletedPart> completedParts = awaitAllFutures(partETagsFutures);
for (CompletedPart part : completedParts) {
if (StringUtils.isEmpty(part.eTag())) {
// this was somehow cancelled/aborted
// explicitly fail.
throw new CancellationException("Upload of part "
+ part.partNumber() + " was aborted");
}
}
return completedParts;
} catch (CancellationException e) {
// One or more of the futures has been cancelled.
LOG.warn("Cancelled while waiting for uploads to {} to complete", key, e);
throw e;
} catch (RuntimeException | IOException ie) {
// IO failure or low level problem.
LOG.debug("Failure while waiting for uploads to {} to complete;"
+ " uploadAborted={}",
key, uploadAborted.get(), ie);
abort();
throw ie;
}
}
/**
* Cancel all active uploads.
* Cancel all active uploads and close all blocks.
* This waits for {@link #TIME_TO_AWAIT_CANCEL_COMPLETION}
* for the cancellations to be processed.
* All exceptions thrown by the futures are ignored. as is any TimeoutException.
*/
private void cancelAllActiveFutures() {
LOG.debug("Cancelling futures");
for (ListenableFuture<CompletedPart> future : partETagsFutures) {
future.cancel(true);
}
private void cancelAllActiveUploads() {
// interrupt futures if not already attempted
LOG.debug("Cancelling {} futures", partETagsFutures.size());
cancelAllFuturesAndAwaitCompletion(partETagsFutures,
true,
TIME_TO_AWAIT_CANCEL_COMPLETION);
// now close all the blocks.
LOG.debug("Closing blocks");
blocksToClose.forEach((key1, value) ->
cleanupWithLogger(LOG, value));
}
/**
@ -972,8 +1174,9 @@ private void cancelAllActiveFutures() {
* Sometimes it fails; here retries are handled to avoid losing all data
* on a transient failure.
* @param partETags list of partial uploads
* @throws IOException on any problem
* @throws IOException on any problem which did not recover after retries.
*/
@Retries.RetryTranslated
private void complete(List<CompletedPart> partETags)
throws IOException {
maybeRethrowUploadFailure();
@ -994,23 +1197,35 @@ private void complete(List<CompletedPart> partETags)
}
/**
* Abort a multi-part upload. Retries are not attempted on failures.
* Abort a multi-part upload, after first attempting to
* cancel active uploads via {@link #cancelAllActiveUploads()} on
* the first invocation.
* <p>
* IOExceptions are caught; this is expected to be run as a cleanup process.
* @return any caught exception.
*/
@Retries.RetryTranslated
private IOException abort() {
LOG.debug("Aborting upload");
try {
trackDurationOfInvocation(statistics,
OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol(), () -> {
cancelAllActiveFutures();
writeOperationHelper.abortMultipartUpload(key, uploadId,
false, null);
});
// set the cancel flag so any newly scheduled uploads exit fast.
if (!uploadAborted.getAndSet(true)) {
LOG.debug("Aborting upload");
progressListener.progressChanged(TRANSFER_MULTIPART_ABORTED_EVENT, 0);
// an abort is double counted; the outer one also includes time to cancel
// all pending aborts so is important to measure.
trackDurationOfInvocation(statistics,
OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol(), () -> {
cancelAllActiveUploads();
writeOperationHelper.abortMultipartUpload(key, uploadId,
false, null);
});
}
return null;
} catch (FileNotFoundException e) {
// The abort has already taken place
return e;
} catch (IOException e) {
// this point is only reached if the operation failed more than
// the allowed retry count
// this point is only reached if abortMultipartUpload failed
LOG.warn("Unable to abort multipart upload,"
+ " you may need to purge uploaded parts", e);
statistics.exceptionInMultipartAbort();
@ -1047,17 +1262,14 @@ private BlockUploadProgress(S3ADataBlocks.DataBlock block,
this.transferQueueTime = transferQueueTime;
this.size = block.dataSize();
this.nextListener = nextListener;
this.transferStartTime = now(); // will be updated when progress is made
}
public void progressChanged(ProgressListenerEvent eventType) {
switch (eventType) {
case REQUEST_BYTE_TRANSFER_EVENT:
// bytes uploaded
statistics.bytesTransferred(size);
break;
case PUT_STARTED_EVENT:
case TRANSFER_PART_STARTED_EVENT:
transferStartTime = now();
statistics.blockUploadStarted(
@ -1067,6 +1279,7 @@ public void progressChanged(ProgressListenerEvent eventType) {
break;
case TRANSFER_PART_COMPLETED_EVENT:
case PUT_COMPLETED_EVENT:
statistics.blockUploadCompleted(
Duration.between(transferStartTime, now()),
size);
@ -1074,6 +1287,8 @@ public void progressChanged(ProgressListenerEvent eventType) {
break;
case TRANSFER_PART_FAILED_EVENT:
case PUT_FAILED_EVENT:
case PUT_INTERRUPTED_EVENT:
statistics.blockUploadFailed(
Duration.between(transferStartTime, now()),
size);
@ -1092,8 +1307,9 @@ public void progressChanged(ProgressListenerEvent eventType) {
/**
* Bridge from {@link ProgressListener} to Hadoop {@link Progressable}.
* All progress events invoke {@link Progressable#progress()}.
*/
private static class ProgressableListener implements ProgressListener {
private static final class ProgressableListener implements ProgressListener {
private final Progressable progress;
ProgressableListener(Progressable progress) {
@ -1106,11 +1322,12 @@ public void progressChanged(ProgressListenerEvent eventType, long bytesTransferr
progress.progress();
}
}
}
/**
* Create a builder.
* @return
* @return a new builder.
*/
public static BlockOutputStreamBuilder builder() {
return new BlockOutputStreamBuilder();
@ -1323,6 +1540,11 @@ public BlockOutputStreamBuilder withIOStatisticsAggregator(
return this;
}
/**
* Is multipart upload enabled?
* @param value the new value
* @return the builder
*/
public BlockOutputStreamBuilder withMultipartEnabled(
final boolean value) {
isMultipartUploadEnabled = value;

View File

@ -19,29 +19,35 @@
package org.apache.hadoop.fs.s3a;
import java.io.BufferedOutputStream;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
import java.io.EOFException;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import org.apache.hadoop.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.store.DataBlocks;
import org.apache.hadoop.util.DirectBufferPool;
import org.apache.hadoop.util.functional.BiFunctionRaisingIOE;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
import static org.apache.hadoop.fs.s3a.impl.UploadContentProviders.byteArrayContentProvider;
import static org.apache.hadoop.fs.s3a.impl.UploadContentProviders.byteBufferContentProvider;
import static org.apache.hadoop.fs.s3a.impl.UploadContentProviders.fileContentProvider;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.util.Preconditions.checkArgument;
/**
* Set of classes to support output streaming into blocks which are then
@ -63,15 +69,11 @@ private S3ADataBlocks() {
* @param len number of bytes to be written
* @throws NullPointerException for a null buffer
* @throws IndexOutOfBoundsException if indices are out of range
* @throws IOException never but in signature of methods called.
*/
static void validateWriteArgs(byte[] b, int off, int len)
throws IOException {
Preconditions.checkNotNull(b);
if ((off < 0) || (off > b.length) || (len < 0) ||
((off + len) > b.length) || ((off + len) < 0)) {
throw new IndexOutOfBoundsException(
"write (b[" + b.length + "], " + off + ", " + len + ')');
}
DataBlocks.validateWriteArgs(b, off, len);
}
/**
@ -81,7 +83,7 @@ static void validateWriteArgs(byte[] b, int off, int len)
* @return the factory, ready to be initialized.
* @throws IllegalArgumentException if the name is unknown.
*/
static BlockFactory createFactory(S3AFileSystem owner,
static BlockFactory createFactory(StoreContext owner,
String name) {
switch (name) {
case Constants.FAST_UPLOAD_BUFFER_ARRAY:
@ -98,56 +100,77 @@ static BlockFactory createFactory(S3AFileSystem owner,
/**
* The output information for an upload.
* It can be one of a file or an input stream.
* When closed, any stream is closed. Any source file is untouched.
* <p>
* The data is accessed via the content provider; other constructors
* create the appropriate content provider for the data.
* <p>
* When {@link #close()} is called, the content provider is itself closed.
*/
public static final class BlockUploadData implements Closeable {
private final File file;
private final InputStream uploadStream;
/**
* The content provider.
*/
private final UploadContentProviders.BaseContentProvider<?> contentProvider;
public BlockUploadData(final UploadContentProviders.BaseContentProvider<?> contentProvider) {
this.contentProvider = requireNonNull(contentProvider);
}
/**
* The content provider.
* @return the content provider
*/
public UploadContentProviders.BaseContentProvider<?> getContentProvider() {
return contentProvider;
}
/**
* File constructor; input stream will be null.
* @param file file to upload
* @param isOpen optional predicate to check if the stream is open.
*/
public BlockUploadData(File file) {
Preconditions.checkArgument(file.exists(), "No file: " + file);
this.file = file;
this.uploadStream = null;
public BlockUploadData(File file, final Supplier<Boolean> isOpen) {
checkArgument(file.exists(), "No file: " + file);
final long length = file.length();
checkArgument(length <= Integer.MAX_VALUE,
"File %s is too long to upload: %d", file, length);
this.contentProvider = fileContentProvider(file, 0, (int) length, isOpen);
}
/**
* Stream constructor, file field will be null.
* @param uploadStream stream to upload
* Byte array constructor, with support for
* uploading just a slice of the array.
*
* @param bytes buffer to read.
* @param offset offset in buffer.
* @param size size of the data.
* @param isOpen optional predicate to check if the stream is open.
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null.
*/
public BlockUploadData(InputStream uploadStream) {
Preconditions.checkNotNull(uploadStream, "rawUploadStream");
this.uploadStream = uploadStream;
this.file = null;
public BlockUploadData(byte[] bytes, int offset, int size,
final Supplier<Boolean> isOpen) {
this.contentProvider = byteArrayContentProvider(bytes, offset, size, isOpen);
}
/**
* Predicate: does this instance contain a file reference.
* @return true if there is a file.
* Byte array constructor to upload all of the array.
* @param bytes buffer to read.
* @throws IllegalArgumentException if the arguments are invalid.
* @param isOpen optional predicate to check if the stream is open.
* @throws NullPointerException if the buffer is null.
*/
boolean hasFile() {
return file != null;
public BlockUploadData(byte[] bytes, final Supplier<Boolean> isOpen) {
this.contentProvider = byteArrayContentProvider(bytes, isOpen);
}
/**
* Get the file, if there is one.
* @return the file for uploading, or null.
* Size as declared by the content provider.
* @return size of the data
*/
File getFile() {
return file;
}
/**
* Get the raw upload stream, if the object was
* created with one.
* @return the upload stream or null.
*/
InputStream getUploadStream() {
return uploadStream;
int getSize() {
return contentProvider.getSize();
}
/**
@ -156,18 +179,21 @@ InputStream getUploadStream() {
*/
@Override
public void close() throws IOException {
cleanupWithLogger(LOG, uploadStream);
cleanupWithLogger(LOG, contentProvider);
}
}
/**
* Base class for block factories.
*/
static abstract class BlockFactory implements Closeable {
public static abstract class BlockFactory implements Closeable {
private final S3AFileSystem owner;
/**
* Store context; left as "owner" for historical reasons.
*/
private final StoreContext owner;
protected BlockFactory(S3AFileSystem owner) {
protected BlockFactory(StoreContext owner) {
this.owner = owner;
}
@ -179,6 +205,8 @@ protected BlockFactory(S3AFileSystem owner) {
* @param limit limit of the block.
* @param statistics stats to work with
* @return a new block.
* @throws IOException any failure to create a block in the local FS.
* @throws OutOfMemoryError lack of space to create any memory buffer
*/
abstract DataBlock create(long index, long limit,
BlockOutputStreamStatistics statistics)
@ -196,8 +224,9 @@ public void close() throws IOException {
/**
* Owner.
* @return the store context of the factory.
*/
protected S3AFileSystem getOwner() {
protected StoreContext getOwner() {
return owner;
}
}
@ -254,6 +283,14 @@ final DestState getState() {
return state;
}
/**
* Predicate to check if the block is in the upload state.
* @return true if the block is uploading
*/
final boolean isUploading() {
return state == Upload;
}
/**
* Return the current data size.
* @return the size of the data
@ -295,10 +332,10 @@ boolean hasData() {
*/
int write(byte[] buffer, int offset, int length) throws IOException {
verifyState(Writing);
Preconditions.checkArgument(buffer != null, "Null buffer");
Preconditions.checkArgument(length >= 0, "length is negative");
Preconditions.checkArgument(offset >= 0, "offset is negative");
Preconditions.checkArgument(
checkArgument(buffer != null, "Null buffer");
checkArgument(length >= 0, "length is negative");
checkArgument(offset >= 0, "offset is negative");
checkArgument(
!(buffer.length - offset < length),
"buffer shorter than amount of data to write");
return 0;
@ -359,7 +396,7 @@ protected void innerClose() throws IOException {
/**
* A block has been allocated.
*/
protected void blockAllocated() {
protected final void blockAllocated() {
if (statistics != null) {
statistics.blockAllocated();
}
@ -368,7 +405,7 @@ protected void blockAllocated() {
/**
* A block has been released.
*/
protected void blockReleased() {
protected final void blockReleased() {
if (statistics != null) {
statistics.blockReleased();
}
@ -386,7 +423,7 @@ protected BlockOutputStreamStatistics getStatistics() {
*/
static class ArrayBlockFactory extends BlockFactory {
ArrayBlockFactory(S3AFileSystem owner) {
ArrayBlockFactory(StoreContext owner) {
super(owner);
}
@ -394,13 +431,18 @@ static class ArrayBlockFactory extends BlockFactory {
DataBlock create(long index, long limit,
BlockOutputStreamStatistics statistics)
throws IOException {
Preconditions.checkArgument(limit > 0,
checkArgument(limit > 0,
"Invalid block size: %d", limit);
return new ByteArrayBlock(0, limit, statistics);
}
}
/**
* Subclass of JVM {@link ByteArrayOutputStream} which makes the buffer
* accessible; the base class {@code toByteArray()} method creates a copy
* of the data first, which we do not want.
*/
static class S3AByteArrayOutputStream extends ByteArrayOutputStream {
S3AByteArrayOutputStream(int size) {
@ -408,16 +450,14 @@ static class S3AByteArrayOutputStream extends ByteArrayOutputStream {
}
/**
* InputStream backed by the internal byte array.
*
* @return
* Get the buffer.
* This is not a copy.
* @return the buffer.
*/
ByteArrayInputStream getInputStream() {
ByteArrayInputStream bin = new ByteArrayInputStream(this.buf, 0, count);
this.reset();
this.buf = null;
return bin;
public byte[] getBuffer() {
return buf;
}
}
/**
@ -459,9 +499,10 @@ long dataSize() {
BlockUploadData startUpload() throws IOException {
super.startUpload();
dataSize = buffer.size();
ByteArrayInputStream bufferData = buffer.getInputStream();
final byte[] bytes = buffer.getBuffer();
buffer = null;
return new BlockUploadData(bufferData);
return new BlockUploadData(
byteArrayContentProvider(bytes, 0, dataSize, this::isUploading));
}
@Override
@ -511,7 +552,7 @@ static class ByteBufferBlockFactory extends BlockFactory {
private final DirectBufferPool bufferPool = new DirectBufferPool();
private final AtomicInteger buffersOutstanding = new AtomicInteger(0);
ByteBufferBlockFactory(S3AFileSystem owner) {
ByteBufferBlockFactory(StoreContext owner) {
super(owner);
}
@ -519,7 +560,7 @@ static class ByteBufferBlockFactory extends BlockFactory {
ByteBufferBlock create(long index, long limit,
BlockOutputStreamStatistics statistics)
throws IOException {
Preconditions.checkArgument(limit > 0,
checkArgument(limit > 0,
"Invalid block size: %d", limit);
return new ByteBufferBlock(index, limit, statistics);
}
@ -590,11 +631,8 @@ long dataSize() {
BlockUploadData startUpload() throws IOException {
super.startUpload();
dataSize = bufferCapacityUsed();
// set the buffer up from reading from the beginning
blockBuffer.limit(blockBuffer.position());
blockBuffer.position(0);
return new BlockUploadData(
new ByteBufferInputStream(dataSize, blockBuffer));
byteBufferContentProvider(blockBuffer, dataSize, this::isUploading));
}
@Override
@ -642,154 +680,8 @@ public String toString() {
'}';
}
/**
* Provide an input stream from a byte buffer; supporting
* {@link #mark(int)}, which is required to enable replay of failed
* PUT attempts.
*/
class ByteBufferInputStream extends InputStream {
private final int size;
private ByteBuffer byteBuffer;
ByteBufferInputStream(int size,
ByteBuffer byteBuffer) {
LOG.debug("Creating ByteBufferInputStream of size {}", size);
this.size = size;
this.byteBuffer = byteBuffer;
}
/**
* After the stream is closed, set the local reference to the byte
* buffer to null; this guarantees that future attempts to use
* stream methods will fail.
*/
@Override
public synchronized void close() {
LOG.debug("ByteBufferInputStream.close() for {}",
ByteBufferBlock.super.toString());
byteBuffer = null;
}
/**
* Verify that the stream is open.
* @throws IOException if the stream is closed
*/
private void verifyOpen() throws IOException {
if (byteBuffer == null) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
}
public synchronized int read() throws IOException {
if (available() > 0) {
return byteBuffer.get() & 0xFF;
} else {
return -1;
}
}
@Override
public synchronized long skip(long offset) throws IOException {
verifyOpen();
long newPos = position() + offset;
if (newPos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
}
if (newPos > size) {
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
}
byteBuffer.position((int) newPos);
return newPos;
}
@Override
public synchronized int available() {
Preconditions.checkState(byteBuffer != null,
FSExceptionMessages.STREAM_IS_CLOSED);
return byteBuffer.remaining();
}
/**
* Get the current buffer position.
* @return the buffer position
*/
public synchronized int position() {
return byteBuffer.position();
}
/**
* Check if there is data left.
* @return true if there is data remaining in the buffer.
*/
public synchronized boolean hasRemaining() {
return byteBuffer.hasRemaining();
}
@Override
public synchronized void mark(int readlimit) {
LOG.debug("mark at {}", position());
byteBuffer.mark();
}
@Override
public synchronized void reset() throws IOException {
LOG.debug("reset");
byteBuffer.reset();
}
@Override
public boolean markSupported() {
return true;
}
/**
* Read in data.
* @param b destination buffer
* @param offset offset within the buffer
* @param length length of bytes to read
* @throws EOFException if the position is negative
* @throws IndexOutOfBoundsException if there isn't space for the
* amount of data requested.
* @throws IllegalArgumentException other arguments are invalid.
*/
@SuppressWarnings("NullableProblems")
public synchronized int read(byte[] b, int offset, int length)
throws IOException {
Preconditions.checkArgument(length >= 0, "length is negative");
Preconditions.checkArgument(b != null, "Null buffer");
if (b.length - offset < length) {
throw new IndexOutOfBoundsException(
FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+ ": request length =" + length
+ ", with offset =" + offset
+ "; buffer capacity =" + (b.length - offset));
}
verifyOpen();
if (!hasRemaining()) {
return -1;
}
int toRead = Math.min(length, available());
byteBuffer.get(b, offset, toRead);
return toRead;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(
"ByteBufferInputStream{");
sb.append("size=").append(size);
ByteBuffer buf = this.byteBuffer;
if (buf != null) {
sb.append(", available=").append(buf.remaining());
}
sb.append(", ").append(ByteBufferBlock.super.toString());
sb.append('}');
return sb.toString();
}
}
}
}
// ====================================================================
@ -799,8 +691,34 @@ public String toString() {
*/
static class DiskBlockFactory extends BlockFactory {
DiskBlockFactory(S3AFileSystem owner) {
/**
* Function to create a temp file.
*/
private final BiFunctionRaisingIOE<Long, Long, File> tempFileFn;
/**
* Constructor.
* Takes the owner so it can call
* {@link StoreContext#createTempFile(String, long)}
* and {@link StoreContext#getConfiguration()}.
* @param owner owning fs.
*/
DiskBlockFactory(StoreContext owner) {
super(owner);
tempFileFn = (index, limit) ->
owner.createTempFile(
String.format("s3ablock-%04d-", index),
limit);
}
/**
* Constructor for testing.
* @param tempFileFn function to create a temp file
*/
@VisibleForTesting
DiskBlockFactory(BiFunctionRaisingIOE<Long, Long, File> tempFileFn) {
super(null);
this.tempFileFn = requireNonNull(tempFileFn);
}
/**
@ -817,11 +735,9 @@ DataBlock create(long index,
long limit,
BlockOutputStreamStatistics statistics)
throws IOException {
Preconditions.checkArgument(limit != 0,
checkArgument(limit != 0,
"Invalid block size: %d", limit);
File destFile = getOwner()
.createTmpFileForWrite(String.format("s3ablock-%04d-", index),
limit, getOwner().getConf());
File destFile = tempFileFn.apply(index, limit);
return new DiskBlock(destFile, limit, index, statistics);
}
}
@ -838,6 +754,14 @@ static class DiskBlock extends DataBlock {
private BufferedOutputStream out;
private final AtomicBoolean closed = new AtomicBoolean(false);
/**
* A disk block.
* @param bufferFile file to write to
* @param limit block size limit
* @param index index in output stream
* @param statistics statistics to upaste
* @throws FileNotFoundException if the file cannot be created.
*/
DiskBlock(File bufferFile,
long limit,
long index,
@ -845,7 +769,7 @@ static class DiskBlock extends DataBlock {
throws FileNotFoundException {
super(index, statistics);
this.limit = limit;
this.bufferFile = bufferFile;
this.bufferFile = requireNonNull(bufferFile);
blockAllocated();
out = new BufferedOutputStream(new FileOutputStream(bufferFile));
}
@ -898,7 +822,7 @@ BlockUploadData startUpload() throws IOException {
out.close();
out = null;
}
return new BlockUploadData(bufferFile);
return new BlockUploadData(bufferFile, this::isUploading);
}
/**
@ -906,7 +830,6 @@ BlockUploadData startUpload() throws IOException {
* exists.
* @throws IOException IO problems
*/
@SuppressWarnings("UnnecessaryDefault")
@Override
protected void innerClose() throws IOException {
final DestState state = getState();

View File

@ -21,7 +21,6 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.InterruptedIOException;
import java.io.UncheckedIOException;
import java.net.URI;
@ -43,6 +42,7 @@
import java.util.Set;
import java.util.Objects;
import java.util.TreeSet;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutorService;
@ -85,11 +85,8 @@
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import software.amazon.awssdk.transfer.s3.model.CompletedCopy;
import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
import software.amazon.awssdk.transfer.s3.model.Copy;
import software.amazon.awssdk.transfer.s3.model.CopyRequest;
import software.amazon.awssdk.transfer.s3.model.FileUpload;
import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.slf4j.Logger;
@ -149,6 +146,7 @@
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
import org.apache.hadoop.fs.s3a.impl.StoreContextFactory;
import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
@ -248,6 +246,7 @@
import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_PERFORMANCE;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_REQUIRED_EXCEPTION;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.ARN_BUCKET_OPTION;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
@ -356,8 +355,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
/** Log to warn of storage class configuration problems. */
private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG);
private static final Logger PROGRESS =
LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
private LocalDirAllocator directoryAllocator;
private String cannedACL;
@ -734,7 +731,7 @@ public void initialize(URI name, Configuration originalConf)
}
blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
DEFAULT_FAST_UPLOAD_BUFFER);
blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
blockFactory = S3ADataBlocks.createFactory(createStoreContext(), blockOutputBuffer);
blockOutputActiveBlocks = intOption(conf,
FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
// If CSE is enabled, do multipart uploads serially.
@ -835,12 +832,13 @@ public void initialize(URI name, Configuration originalConf)
protected S3AStore createS3AStore(final ClientManager clientManager,
final int rateLimitCapacity) {
return new S3AStoreBuilder()
.withAuditSpanSource(getAuditManager())
.withClientManager(clientManager)
.withDurationTrackerFactory(getDurationTrackerFactory())
.withStoreContextFactory(this)
.withAuditSpanSource(getAuditManager())
.withFsStatistics(getFsStatistics())
.withInstrumentation(getInstrumentation())
.withStatisticsContext(statisticsContext)
.withStoreContextFactory(this)
.withStorageStatistics(getStorageStatistics())
.withReadRateLimiter(unlimitedRate())
.withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity))
@ -1966,9 +1964,48 @@ private final class WriteOperationHelperCallbacksImpl
implements WriteOperationHelper.WriteOperationHelperCallbacks {
@Override
@Retries.OnceRaw
public CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request) {
return getS3Client().completeMultipartUpload(request);
return store.completeMultipartUpload(request);
}
@Override
@Retries.OnceRaw
public UploadPartResponse uploadPart(
final UploadPartRequest request,
final RequestBody body,
final DurationTrackerFactory durationTrackerFactory)
throws AwsServiceException, UncheckedIOException {
return store.uploadPart(request, body, durationTrackerFactory);
}
/**
* Perform post-write actions.
* <p>
* This operation MUST be called after any PUT/multipart PUT completes
* successfully.
* <p>
* The actions include calling
* {@link #deleteUnnecessaryFakeDirectories(Path)}
* if directory markers are not being retained.
* @param eTag eTag of the written object
* @param versionId S3 object versionId of the written object
* @param key key written to
* @param length total length of file written
* @param putOptions put object options
*/
@Override
@Retries.RetryExceptionsSwallowed
public void finishedWrite(
String key,
long length,
PutObjectOptions putOptions) {
S3AFileSystem.this.finishedWrite(
key,
length,
putOptions);
}
}
@ -2927,7 +2964,7 @@ public S3AStorageStatistics getStorageStatistics() {
/**
* Get the instrumentation's IOStatistics.
* @return statistics
* @return statistics or null if instrumentation has not yet been instantiated.
*/
@Override
public IOStatistics getIOStatistics() {
@ -2956,9 +2993,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() {
*/
protected DurationTrackerFactory nonNullDurationTrackerFactory(
DurationTrackerFactory factory) {
return factory != null
? factory
: getDurationTrackerFactory();
return store.nonNullDurationTrackerFactory(factory);
}
/**
@ -3273,18 +3308,7 @@ public PutObjectRequest.Builder newPutObjectRequestBuilder(String key,
@Retries.OnceRaw
public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
ProgressableProgressListener listener) throws IOException {
long len = getPutRequestLength(putObjectRequest);
LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key());
incrementPutStartStatistics(len);
FileUpload upload = store.getOrCreateTransferManager().uploadFile(
UploadFileRequest.builder()
.putObjectRequest(putObjectRequest)
.source(file)
.addTransferListener(listener)
.build());
return new UploadInfo(upload, len);
return store.putObject(putObjectRequest, file, listener);
}
/**
@ -3297,9 +3321,8 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
* <i>Important: this call will close any input stream in the request.</i>
* @param putObjectRequest the request
* @param putOptions put object options
* @param durationTrackerFactory factory for duration tracking
* @param uploadData data to be uploaded
* @param isFile represents if data to be uploaded is a file
* @param durationTrackerFactory factory for duration tracking
* @return the upload initiated
* @throws SdkException on problems
*/
@ -3307,27 +3330,27 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
@Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed")
PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest,
PutObjectOptions putOptions,
S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws SdkException {
long len = getPutRequestLength(putObjectRequest);
LOG.debug("PUT {} bytes to {}", len, putObjectRequest.key());
incrementPutStartStatistics(len);
final UploadContentProviders.BaseContentProvider provider =
uploadData.getContentProvider();
try {
PutObjectResponse response =
trackDurationOfSupplier(nonNullDurationTrackerFactory(durationTrackerFactory),
OBJECT_PUT_REQUESTS.getSymbol(),
() -> isFile
? getS3Client().putObject(putObjectRequest,
RequestBody.fromFile(uploadData.getFile()))
: getS3Client().putObject(putObjectRequest,
RequestBody.fromInputStream(uploadData.getUploadStream(),
putObjectRequest.contentLength())));
() -> getS3Client().putObject(putObjectRequest,
RequestBody.fromContentProvider(
provider,
provider.getSize(),
CONTENT_TYPE_OCTET_STREAM)));
incrementPutCompletedStatistics(true, len);
// apply any post-write actions.
finishedWrite(putObjectRequest.key(), len,
response.eTag(), response.versionId(),
putOptions);
finishedWrite(putObjectRequest.key(), len, putOptions);
return response;
} catch (SdkException e) {
incrementPutCompletedStatistics(false, len);
@ -3385,13 +3408,8 @@ UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body,
*
* @param bytes bytes in the request.
*/
public void incrementPutStartStatistics(long bytes) {
LOG.debug("PUT start {} bytes", bytes);
incrementWriteOperations();
incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
if (bytes > 0) {
incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
}
protected void incrementPutStartStatistics(long bytes) {
store.incrementPutStartStatistics(bytes);
}
/**
@ -3401,14 +3419,8 @@ public void incrementPutStartStatistics(long bytes) {
* @param success did the operation succeed?
* @param bytes bytes in the request.
*/
public void incrementPutCompletedStatistics(boolean success, long bytes) {
LOG.debug("PUT completed success={}; {} bytes", success, bytes);
if (bytes > 0) {
incrementStatistic(OBJECT_PUT_BYTES, bytes);
decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
}
incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
protected void incrementPutCompletedStatistics(boolean success, long bytes) {
store.incrementPutCompletedStatistics(success, bytes);
}
/**
@ -3418,12 +3430,8 @@ public void incrementPutCompletedStatistics(boolean success, long bytes) {
* @param key key to file that is being written (for logging)
* @param bytes bytes successfully uploaded.
*/
public void incrementPutProgressStatistics(String key, long bytes) {
PROGRESS.debug("PUT {}: {} bytes", key, bytes);
incrementWriteOperations();
if (bytes > 0) {
statistics.incrementBytesWritten(bytes);
}
protected void incrementPutProgressStatistics(String key, long bytes) {
store.incrementPutProgressStatistics(key, bytes);
}
/**
@ -4254,6 +4262,7 @@ public boolean deleteLocal(Path path, boolean recursive) throws IOException {
}
@Override
@Retries.RetryTranslated
public void copyLocalFileFromTo(File file, Path from, Path to) throws IOException {
// the duration of the put is measured, but the active span is the
// constructor-supplied one -this ensures all audit log events are grouped correctly
@ -4270,11 +4279,13 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio
}
@Override
@Retries.RetryTranslated
public FileStatus getFileStatus(Path f) throws IOException {
return S3AFileSystem.this.getFileStatus(f);
}
@Override
@Retries.RetryTranslated
public boolean createEmptyDir(Path path, StoreContext storeContext)
throws IOException {
return trackDuration(getDurationTrackerFactory(),
@ -4295,8 +4306,9 @@ public boolean createEmptyDir(Path path, StoreContext storeContext)
* @param putOptions put object options
* @return the upload result
* @throws IOException IO failure
* @throws CancellationException if the wait() was cancelled
*/
@Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed")
@Retries.OnceTranslated("For PUT; post-PUT actions are RetrySwallowed")
PutObjectResponse executePut(
final PutObjectRequest putObjectRequest,
final Progressable progress,
@ -4306,49 +4318,21 @@ PutObjectResponse executePut(
String key = putObjectRequest.key();
long len = getPutRequestLength(putObjectRequest);
ProgressableProgressListener listener =
new ProgressableProgressListener(this, putObjectRequest.key(), progress);
new ProgressableProgressListener(store, putObjectRequest.key(), progress);
UploadInfo info = putObject(putObjectRequest, file, listener);
PutObjectResponse result = waitForUploadCompletion(key, info).response();
PutObjectResponse result = store.waitForUploadCompletion(key, info).response();
listener.uploadCompleted(info.getFileUpload());
// post-write actions
finishedWrite(key, len,
result.eTag(), result.versionId(), putOptions);
finishedWrite(key, len, putOptions);
return result;
}
/**
* Wait for an upload to complete.
* If the upload (or its result collection) failed, this is where
* the failure is raised as an AWS exception.
* Calls {@link #incrementPutCompletedStatistics(boolean, long)}
* to update the statistics.
* @param key destination key
* @param uploadInfo upload to wait for
* @return the upload result
* @throws IOException IO failure
*/
@Retries.OnceRaw
CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
throws IOException {
FileUpload upload = uploadInfo.getFileUpload();
try {
CompletedFileUpload result = upload.completionFuture().join();
incrementPutCompletedStatistics(true, uploadInfo.getLength());
return result;
} catch (CompletionException e) {
LOG.info("Interrupted: aborting upload");
incrementPutCompletedStatistics(false, uploadInfo.getLength());
throw extractException("upload", key, e);
}
}
/**
* This override bypasses checking for existence.
*
* @param f the path to delete; this may be unqualified.
* @return true, always. * @param f the path to delete.
* @return true if deleteOnExit is successful, otherwise false.
* @return true, always.
* @throws IOException IO failure
*/
@Override
@ -4729,9 +4713,7 @@ CreateMultipartUploadResponse initiateMultipartUpload(
* {@link #deleteUnnecessaryFakeDirectories(Path)}
* if directory markers are not being retained.
* @param key key written to
* @param length total length of file written
* @param eTag eTag of the written object
* @param versionId S3 object versionId of the written object
* @param length total length of file written
* @param putOptions put object options
*/
@InterfaceAudience.Private
@ -4739,11 +4721,9 @@ CreateMultipartUploadResponse initiateMultipartUpload(
void finishedWrite(
String key,
long length,
String eTag,
String versionId,
PutObjectOptions putOptions) {
LOG.debug("Finished write to {}, len {}. etag {}, version {}",
key, length, eTag, versionId);
LOG.debug("Finished write to {}, len {}.",
key, length);
Preconditions.checkArgument(length >= 0, "content length is negative");
if (!putOptions.isKeepMarkers()) {
Path p = keyToQualifiedPath(key);
@ -4837,18 +4817,16 @@ private void createFakeDirectory(final String objectName,
@Retries.RetryTranslated
private void createEmptyObject(final String objectName, PutObjectOptions putOptions)
throws IOException {
final InputStream im = new InputStream() {
@Override
public int read() throws IOException {
return -1;
}
};
S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(im);
S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(
new byte[0], 0, 0, null);
invoker.retry("PUT 0-byte object ", objectName, true,
() -> putObjectDirect(getRequestFactory().newDirectoryMarkerRequest(objectName).build(),
putOptions, uploadData, false, getDurationTrackerFactory()));
() -> putObjectDirect(
getRequestFactory().newDirectoryMarkerRequest(objectName).build(),
putOptions,
uploadData,
getDurationTrackerFactory()));
incrementPutProgressStatistics(objectName, 0);
instrumentation.directoryCreated();
}

View File

@ -1505,6 +1505,7 @@ private OutputStreamStatistics(
INVOCATION_HFLUSH.getSymbol(),
INVOCATION_HSYNC.getSymbol())
.withGauges(
STREAM_WRITE_BLOCK_UPLOADS_ACTIVE.getSymbol(),
STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(),
STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol())
.withDurationTracking(

View File

@ -125,6 +125,11 @@ public class S3ARetryPolicy implements RetryPolicy {
*/
protected final RetryPolicy retryAwsClientExceptions;
/**
* Retry policy for all http 5xx errors not handled explicitly.
*/
protected final RetryPolicy http5xxRetryPolicy;
/**
* Instantiate.
* @param conf configuration to read.
@ -164,6 +169,13 @@ public S3ARetryPolicy(Configuration conf) {
// client connectivity: fixed retries without care for idempotency
connectivityFailure = baseExponentialRetry;
boolean retry5xxHttpErrors =
conf.getBoolean(RETRY_HTTP_5XX_ERRORS, DEFAULT_RETRY_HTTP_5XX_ERRORS);
http5xxRetryPolicy = retry5xxHttpErrors
? retryAwsClientExceptions
: fail;
Map<Class<? extends Exception>, RetryPolicy> policyMap =
createExceptionMap();
retryPolicy = retryByException(retryIdempotentCalls, policyMap);
@ -228,15 +240,13 @@ protected Map<Class<? extends Exception>, RetryPolicy> createExceptionMap() {
// throttled requests are can be retried, always
policyMap.put(AWSServiceThrottledException.class, throttlePolicy);
// Status 5xx error code is an immediate failure
// Status 5xx error code has historically been treated as an immediate failure
// this is sign of a server-side problem, and while
// rare in AWS S3, it does happen on third party stores.
// (out of disk space, etc).
// by the time we get here, the aws sdk will have
// already retried.
// there is specific handling for some 5XX codes (501, 503);
// this is for everything else
policyMap.put(AWSStatus500Exception.class, fail);
policyMap.put(AWSStatus500Exception.class, http5xxRetryPolicy);
// subclass of AWSServiceIOException whose cause is always S3Exception
policyMap.put(AWSS3IOException.class, retryIdempotentCalls);

View File

@ -18,16 +18,27 @@
package org.apache.hadoop.fs.s3a;
import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.time.Duration;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CancellationException;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -78,6 +89,55 @@ public interface S3AStore extends IOStatisticsSource, ClientManager {
ClientManager clientManager();
/**
* Increment read operations.
*/
void incrementReadOperations();
/**
* Increment the write operation counter.
* This is somewhat inaccurate, as it appears to be invoked more
* often than needed in progress callbacks.
*/
void incrementWriteOperations();
/**
* At the start of a put/multipart upload operation, update the
* relevant counters.
*
* @param bytes bytes in the request.
*/
void incrementPutStartStatistics(long bytes);
/**
* At the end of a put/multipart upload operation, update the
* relevant counters and gauges.
*
* @param success did the operation succeed?
* @param bytes bytes in the request.
*/
void incrementPutCompletedStatistics(boolean success, long bytes);
/**
* Callback for use in progress callbacks from put/multipart upload events.
* Increments those statistics which are expected to be updated during
* the ongoing upload operation.
* @param key key to file that is being written (for logging)
* @param bytes bytes successfully uploaded.
*/
void incrementPutProgressStatistics(String key, long bytes);
/**
* Given a possibly null duration tracker factory, return a non-null
* one for use in tracking durations -either that or the FS tracker
* itself.
*
* @param factory factory.
* @return a non-null factory.
*/
DurationTrackerFactory nonNullDurationTrackerFactory(
DurationTrackerFactory factory);
/**
* Perform a bulk object delete operation against S3.
* Increments the {@code OBJECT_DELETE_REQUESTS} and write
@ -133,4 +193,75 @@ Map.Entry<Duration, DeleteObjectsResponse> deleteObjects(DeleteObjectsRequest de
Map.Entry<Duration, Optional<DeleteObjectResponse>> deleteObject(
DeleteObjectRequest request) throws SdkException;
/**
* Upload part of a multi-partition file.
* Increments the write and put counters.
* <i>Important: this call does not close any input stream in the body.</i>
* <p>
* Retry Policy: none.
* @param durationTrackerFactory duration tracker factory for operation
* @param request the upload part request.
* @param body the request body.
* @return the result of the operation.
* @throws AwsServiceException on problems
* @throws UncheckedIOException failure to instantiate the s3 client
*/
@Retries.OnceRaw
UploadPartResponse uploadPart(
UploadPartRequest request,
RequestBody body,
DurationTrackerFactory durationTrackerFactory)
throws AwsServiceException, UncheckedIOException;
/**
* Start a transfer-manager managed async PUT of an object,
* incrementing the put requests and put bytes
* counters.
* <p>
* It does not update the other counters,
* as existing code does that as progress callbacks come in.
* Byte length is calculated from the file length, or, if there is no
* file, from the content length of the header.
* <p>
* Because the operation is async, any stream supplied in the request
* must reference data (files, buffers) which stay valid until the upload
* completes.
* Retry policy: N/A: the transfer manager is performing the upload.
* Auditing: must be inside an audit span.
* @param putObjectRequest the request
* @param file the file to be uploaded
* @param listener the progress listener for the request
* @return the upload initiated
* @throws IOException if transfer manager creation failed.
*/
@Retries.OnceRaw
UploadInfo putObject(
PutObjectRequest putObjectRequest,
File file,
ProgressableProgressListener listener) throws IOException;
/**
* Wait for an upload to complete.
* If the upload (or its result collection) failed, this is where
* the failure is raised as an AWS exception.
* Calls {@link S3AStore#incrementPutCompletedStatistics(boolean, long)}
* to update the statistics.
* @param key destination key
* @param uploadInfo upload to wait for
* @return the upload result
* @throws IOException IO failure
* @throws CancellationException if the wait() was cancelled
*/
@Retries.OnceTranslated
CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
throws IOException;
/**
* Complete a multipart upload.
* @param request request
* @return the response
*/
@Retries.OnceRaw
CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request);
}

View File

@ -24,6 +24,7 @@
import software.amazon.awssdk.core.exception.ApiCallTimeoutException;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.core.retry.RetryUtils;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.S3Exception;
import software.amazon.awssdk.services.s3.model.S3Object;
@ -298,7 +299,7 @@ public static IOException translateException(@Nullable String operation,
case SC_405_METHOD_NOT_ALLOWED:
case SC_415_UNSUPPORTED_MEDIA_TYPE:
case SC_501_NOT_IMPLEMENTED:
ioe = new AWSUnsupportedFeatureException(message, s3Exception);
ioe = new AWSUnsupportedFeatureException(message, ase);
break;
// precondition failure: the object is there, but the precondition
@ -1177,6 +1178,19 @@ public static S3AFileStatus[] iteratorToStatuses(
return statuses;
}
/**
* Get the length of the PUT, verifying that the length is known.
* @param putObjectRequest a request bound to a file or a stream.
* @return the request length
* @throws IllegalArgumentException if the length is negative
*/
public static long getPutRequestLength(PutObjectRequest putObjectRequest) {
long len = putObjectRequest.contentLength();
Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length");
return len;
}
/**
* An interface for use in lambda-expressions working with
* directory tree listings.

View File

@ -65,6 +65,38 @@ public enum Statistic {
"GET request.",
TYPE_DURATION),
/* Http error responses */
HTTP_RESPONSE_400(
StoreStatisticNames.HTTP_RESPONSE_400,
"400 response.",
TYPE_COUNTER),
HTTP_RESPONSE_429(
StoreStatisticNames.HTTP_RESPONSE_429,
"429 response.",
TYPE_COUNTER),
HTTP_RESPONSE_4XX(
StoreStatisticNames.HTTP_RESPONSE_4XX,
"4XX response.",
TYPE_COUNTER),
HTTP_RESPONSE_500(
StoreStatisticNames.HTTP_RESPONSE_500,
"500 response.",
TYPE_COUNTER),
HTTP_RESPONSE_503(
StoreStatisticNames.HTTP_RESPONSE_503,
"503 response.",
TYPE_COUNTER),
HTTP_RESPONSE_5XX(
StoreStatisticNames.HTTP_RESPONSE_5XX,
"5XX response.",
TYPE_COUNTER),
/* FileSystem Level statistics */
FILESYSTEM_INITIALIZATION(

View File

@ -21,9 +21,11 @@
import javax.annotation.Nullable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
@ -233,14 +235,12 @@ private void deactivateAuditSpan() {
* @param destKey destination key
* @param length size, if known. Use -1 for not known
* @param options options for the request
* @param isFile is data to be uploaded a file
* @return the request
*/
@Retries.OnceRaw
public PutObjectRequest createPutObjectRequest(String destKey,
long length,
final PutObjectOptions options,
boolean isFile) {
final PutObjectOptions options) {
activateAuditSpan();
@ -289,7 +289,7 @@ public String initiateMultiPartUpload(
/**
* Finalize a multipart PUT operation.
* This completes the upload, and, if that works, calls
* {@link S3AFileSystem#finishedWrite(String, long, String, String, org.apache.hadoop.fs.s3a.impl.PutObjectOptions)}
* {@link WriteOperationHelperCallbacks#finishedWrite(String, long, PutObjectOptions)}
* to update the filesystem.
* Retry policy: retrying, translated.
* @param destKey destination of the commit
@ -324,8 +324,7 @@ private CompleteMultipartUploadResponse finalizeMultipartUpload(
destKey, uploadId, partETags);
return writeOperationHelperCallbacks.completeMultipartUpload(requestBuilder.build());
});
owner.finishedWrite(destKey, length, uploadResult.eTag(),
uploadResult.versionId(),
writeOperationHelperCallbacks.finishedWrite(destKey, length,
putOptions);
return uploadResult;
}
@ -404,11 +403,12 @@ public void abortMultipartUpload(String destKey, String uploadId,
/**
* Abort a multipart commit operation.
* @param upload upload to abort.
* @throws FileNotFoundException if the upload is unknown
* @throws IOException on problems.
*/
@Retries.RetryTranslated
public void abortMultipartUpload(MultipartUpload upload)
throws IOException {
throws FileNotFoundException, IOException {
invoker.retry("Aborting multipart commit", upload.key(), true,
withinAuditSpan(getAuditSpan(),
() -> owner.abortMultipartUpload(upload)));
@ -508,20 +508,19 @@ public String toString() {
* file, from the content length of the header.
* @param putObjectRequest the request
* @param putOptions put object options
* @param durationTrackerFactory factory for duration tracking
* @param uploadData data to be uploaded
* @param isFile is data to be uploaded a file
*
* @param durationTrackerFactory factory for duration tracking
* @return the upload initiated
* @throws IOException on problems
*/
@Retries.RetryTranslated
public PutObjectResponse putObject(PutObjectRequest putObjectRequest,
PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
PutObjectOptions putOptions,
S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws IOException {
return retry("Writing Object", putObjectRequest.key(), true, withinAuditSpan(getAuditSpan(),
() -> owner.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile,
() -> owner.putObjectDirect(putObjectRequest, putOptions, uploadData,
durationTrackerFactory)));
}
@ -578,7 +577,6 @@ public CompleteMultipartUploadResponse commitUpload(
/**
* Upload part of a multi-partition file.
* @param request request
* @param durationTrackerFactory duration tracker factory for operation
* @param request the upload part request.
* @param body the request body.
@ -594,7 +592,9 @@ public UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body
request.key(),
true,
withinAuditSpan(getAuditSpan(),
() -> owner.uploadPart(request, body, durationTrackerFactory)));
() -> writeOperationHelperCallbacks.uploadPart(request,
body,
durationTrackerFactory)));
}
/**
@ -644,8 +644,44 @@ public interface WriteOperationHelperCallbacks {
* @param request Complete multi-part upload request
* @return completeMultipartUploadResult
*/
CompleteMultipartUploadResponse completeMultipartUpload(CompleteMultipartUploadRequest request);
@Retries.OnceRaw
CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request);
/**
* Upload part of a multi-partition file.
* Increments the write and put counters.
* <i>Important: this call does not close any input stream in the body.</i>
* <p>
* Retry Policy: none.
* @param durationTrackerFactory duration tracker factory for operation
* @param request the upload part request.
* @param body the request body.
* @return the result of the operation.
* @throws AwsServiceException on problems
* @throws UncheckedIOException failure to instantiate the s3 client
*/
@Retries.OnceRaw
UploadPartResponse uploadPart(
UploadPartRequest request,
RequestBody body,
DurationTrackerFactory durationTrackerFactory)
throws AwsServiceException, UncheckedIOException;
/**
* Perform post-write actions.
* <p>
* This operation MUST be called after any PUT/multipart PUT completes
* successfully.
* @param key key written to
* @param length total length of file written
* @param putOptions put object options
*/
@Retries.RetryExceptionsSwallowed
void finishedWrite(
String key,
long length,
PutObjectOptions putOptions);
}
}

View File

@ -74,13 +74,11 @@ <T> T retry(String action,
* @param destKey destination key
* @param length size, if known. Use -1 for not known
* @param options options for the request
* @param isFile is data to be uploaded a file
* @return the request
*/
PutObjectRequest createPutObjectRequest(String destKey,
long length,
@Nullable PutObjectOptions options,
boolean isFile);
@Nullable PutObjectOptions options);
/**
* Callback on a successful write.
@ -148,6 +146,7 @@ void abortMultipartUpload(String destKey, String uploadId,
/**
* Abort a multipart commit operation.
* @param upload upload to abort.
* @throws FileNotFoundException if the upload is unknown
* @throws IOException on problems.
*/
@Retries.RetryTranslated
@ -208,15 +207,15 @@ UploadPartRequest.Builder newUploadPartRequestBuilder(
* file, from the content length of the header.
* @param putObjectRequest the request
* @param putOptions put object options
* @param durationTrackerFactory factory for duration tracking
* @param uploadData data to be uploaded
* @param isFile is data to be uploaded a file
* @param durationTrackerFactory factory for duration tracking
* @return the upload initiated
* @throws IOException on problems
*/
@Retries.RetryTranslated
PutObjectResponse putObject(PutObjectRequest putObjectRequest,
PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
PutObjectOptions putOptions,
S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws IOException;

View File

@ -26,6 +26,8 @@
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.service.AbstractService;
import static java.util.Objects.requireNonNull;
/**
* 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.
@ -85,7 +87,7 @@ protected AbstractOperationAuditor(final String name) {
@Override
public void init(final OperationAuditorOptions opts) {
this.options = opts;
this.iostatistics = opts.getIoStatisticsStore();
this.iostatistics = requireNonNull(opts.getIoStatisticsStore());
init(opts.getConfiguration());
}

View File

@ -23,12 +23,14 @@
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import software.amazon.awssdk.awscore.AwsExecutionAttribute;
import software.amazon.awssdk.core.SdkRequest;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.http.SdkHttpRequest;
import software.amazon.awssdk.http.SdkHttpResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
@ -66,6 +68,7 @@
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;
import static org.apache.hadoop.fs.s3a.statistics.impl.StatisticsFromAwsSdkImpl.mapErrorStatusCodeToStatisticName;
/**
* The LoggingAuditor logs operations at DEBUG (in SDK Request) and
@ -444,6 +447,22 @@ public String toString() {
HttpReferrerAuditHeader getReferrer() {
return referrer;
}
/**
* Execution failure: extract an error code and if this maps to
* a statistic name, update that counter.
*/
@Override
public void onExecutionFailure(final Context.FailedExecution context,
final ExecutionAttributes executionAttributes) {
final Optional<SdkHttpResponse> response = context.httpResponse();
int sc = response.map(SdkHttpResponse::statusCode).orElse(0);
String stat = mapErrorStatusCodeToStatisticName(sc);
if (stat != null) {
LOG.debug("Incrementing error statistic {}", stat);
getIOStatistics().incrementCounter(stat);
}
}
}
/**
@ -522,4 +541,5 @@ public void beforeExecution(Context.BeforeExecution context,
super.beforeExecution(context, executionAttributes);
}
}
}

View File

@ -21,8 +21,6 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -62,6 +60,7 @@
import org.apache.hadoop.fs.s3a.impl.HeaderProcessing;
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.IOStatistics;
@ -81,6 +80,7 @@
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS;
import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator;
@ -88,11 +88,17 @@
* The implementation of the various actions a committer needs.
* This doesn't implement the protocol/binding to a specific execution engine,
* just the operations needed to to build one.
*
* <p>
* When invoking FS operations, it assumes that the underlying FS is
* handling retries and exception translation: it does not attempt to
* duplicate that work.
*
* <p>
* It does use {@link UploadContentProviders} to create a content provider
* for the request body which is capable of restarting a failed upload.
* This is not currently provided by the default AWS SDK implementation
* of {@code RequestBody#fromFile()}.
* <p>
* See HADOOP-19221 for details.
*/
public class CommitOperations extends AbstractStoreOperation
implements IOStatisticsSource {
@ -553,7 +559,6 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
commitData.setText(partition != null ? "partition: " + partition : "");
commitData.setLength(length);
long offset = 0;
long numParts = (length / uploadPartSize +
((length % uploadPartSize) > 0 ? 1 : 0));
// always write one part, even if it is just an empty one
@ -570,31 +575,19 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
numParts, length));
}
List<CompletedPart> parts = new ArrayList<>((int) numParts);
final int partCount = (int) numParts;
LOG.debug("File size is {}, number of parts to upload = {}",
length, numParts);
length, partCount);
// Open the file to upload.
try (InputStream fileStream = Files.newInputStream(localFile.toPath())) {
for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
progress.progress();
long size = Math.min(length - offset, uploadPartSize);
UploadPartRequest part = writeOperations.newUploadPartRequestBuilder(
destKey,
uploadId,
partNumber,
size).build();
// Read from the file input stream at current position.
RequestBody body = RequestBody.fromInputStream(fileStream, size);
UploadPartResponse response = writeOperations.uploadPart(part, body, statistics);
offset += uploadPartSize;
parts.add(CompletedPart.builder()
.partNumber(partNumber)
.eTag(response.eTag())
.build());
}
}
List<CompletedPart> parts = uploadFileData(
uploadId,
localFile,
destKey,
progress,
length,
partCount,
uploadPartSize);
commitData.bindCommitData(parts);
statistics.commitUploaded(length);
@ -617,6 +610,55 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
}
}
/**
* Upload file data using content provider API.
* This a rewrite of the previous code to address HADOOP-19221;
* our own {@link UploadContentProviders} file content provider
* is used to upload each part of the file.
* @param uploadId upload ID
* @param localFile locally staged file
* @param destKey destination path
* @param progress progress callback
* @param length file length
* @param numParts number of parts to upload
* @param uploadPartSize max size of a part
* @return the ordered list of parts
* @throws IOException IO failure
*/
private List<CompletedPart> uploadFileData(
final String uploadId,
final File localFile,
final String destKey,
final Progressable progress,
final long length,
final int numParts,
final long uploadPartSize) throws IOException {
List<CompletedPart> parts = new ArrayList<>(numParts);
long offset = 0;
for (int partNumber = 1; partNumber <= numParts; partNumber++) {
progress.progress();
int size = (int)Math.min(length - offset, uploadPartSize);
UploadPartRequest part = writeOperations.newUploadPartRequestBuilder(
destKey,
uploadId,
partNumber,
size).build();
// Create a file content provider starting at the current offset.
RequestBody body = RequestBody.fromContentProvider(
UploadContentProviders.fileContentProvider(localFile, offset, size),
size,
CONTENT_TYPE_OCTET_STREAM);
UploadPartResponse response = writeOperations.uploadPart(part, body, statistics);
offset += uploadPartSize;
parts.add(CompletedPart.builder()
.partNumber(partNumber)
.eTag(response.eTag())
.build());
}
return parts;
}
/**
* Add the filesystem statistics to the map; overwriting anything
* with the same name.

View File

@ -18,9 +18,7 @@
package org.apache.hadoop.fs.s3a.commit.magic;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -81,8 +79,8 @@ public boolean aboutToComplete(String uploadId,
PutObjectRequest originalDestPut = getWriter().createPutObjectRequest(
getOriginalDestKey(),
0,
new PutObjectOptions(true, null, headers), false);
upload(originalDestPut, new ByteArrayInputStream(EMPTY));
new PutObjectOptions(true, null, headers));
upload(originalDestPut, EMPTY);
// build the commit summary
SinglePendingCommit commitData = new SinglePendingCommit();
@ -105,8 +103,8 @@ public boolean aboutToComplete(String uploadId,
getPath(), getPendingPartKey(), commitData);
PutObjectRequest put = getWriter().createPutObjectRequest(
getPendingPartKey(),
bytes.length, null, false);
upload(put, new ByteArrayInputStream(bytes));
bytes.length, null);
upload(put, bytes);
return false;
}
@ -117,9 +115,9 @@ public boolean aboutToComplete(String uploadId,
* @throws IOException on problems
*/
@Retries.RetryTranslated
private void upload(PutObjectRequest request, InputStream inputStream) throws IOException {
private void upload(PutObjectRequest request, byte[] bytes) throws IOException {
trackDurationOfInvocation(getTrackerStatistics(), COMMITTER_MAGIC_MARKER_PUT.getSymbol(),
() -> getWriter().putObject(request, PutObjectOptions.keepingDirs(),
new S3ADataBlocks.BlockUploadData(inputStream), false, null));
new S3ADataBlocks.BlockUploadData(bytes, null), null));
}
}

View File

@ -20,6 +20,7 @@
import java.io.Closeable;
import java.io.IOException;
import java.io.UncheckedIOException;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.S3Client;
@ -39,10 +40,34 @@ public interface ClientManager extends Closeable {
S3TransferManager getOrCreateTransferManager()
throws IOException;
/**
* Get the S3Client, raising a failure to create as an IOException.
* @return the S3 client
* @throws IOException failure to create the client.
*/
S3Client getOrCreateS3Client() throws IOException;
/**
* Get the S3Client, raising a failure to create as an UncheckedIOException.
* @return the S3 client
* @throws UncheckedIOException failure to create the client.
*/
S3Client getOrCreateS3ClientUnchecked() throws UncheckedIOException;
/**
* Get the Async S3Client,raising a failure to create as an IOException.
* @return the Async S3 client
* @throws IOException failure to create the client.
*/
S3AsyncClient getOrCreateAsyncClient() throws IOException;
/**
* Get the AsyncS3Client, raising a failure to create as an UncheckedIOException.
* @return the S3 client
* @throws UncheckedIOException failure to create the client.
*/
S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException;
/**
* Close operation is required to not raise exceptions.
*/

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
@ -147,12 +148,34 @@ public synchronized S3Client getOrCreateS3Client() throws IOException {
return s3Client.eval();
}
/**
* Get the S3Client, raising a failure to create as an UncheckedIOException.
* @return the S3 client
* @throws UncheckedIOException failure to create the client.
*/
@Override
public synchronized S3Client getOrCreateS3ClientUnchecked() throws UncheckedIOException {
checkNotClosed();
return s3Client.get();
}
@Override
public synchronized S3AsyncClient getOrCreateAsyncClient() throws IOException {
checkNotClosed();
return s3AsyncClient.eval();
}
/**
* Get the AsyncS3Client, raising a failure to create as an UncheckedIOException.
* @return the S3 client
* @throws UncheckedIOException failure to create the client.
*/
@Override
public synchronized S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException {
checkNotClosed();
return s3Client.get();
}
@Override
public synchronized S3TransferManager getOrCreateTransferManager() throws IOException {
checkNotClosed();

View File

@ -294,4 +294,11 @@ private InternalConstants() {
*/
public static final String AUTH_SCHEME_AWS_SIGV_4 = "aws.auth#sigv4";
/**
* Progress logge name; fairly noisy.
*/
public static final String UPLOAD_PROGRESS_LOG_NAME =
"org.apache.hadoop.fs.s3a.S3AFileSystem.Progress";
}

View File

@ -20,10 +20,72 @@
/**
* Enum for progress listener events.
* Some are used in the {@code S3ABlockOutputStream}
* class to manage progress; others are to assist
* testing.
*/
public enum ProgressListenerEvent {
/**
* Stream has been closed.
*/
CLOSE_EVENT,
/** PUT operation completed successfully. */
PUT_COMPLETED_EVENT,
/** PUT operation was interrupted. */
PUT_INTERRUPTED_EVENT,
/** PUT operation was interrupted. */
PUT_FAILED_EVENT,
/** A PUT operation was started. */
PUT_STARTED_EVENT,
/** Bytes were transferred. */
REQUEST_BYTE_TRANSFER_EVENT,
/**
* A multipart upload was initiated.
*/
TRANSFER_MULTIPART_INITIATED_EVENT,
/**
* A multipart upload was aborted.
*/
TRANSFER_MULTIPART_ABORTED_EVENT,
/**
* A multipart upload was successfully.
*/
TRANSFER_MULTIPART_COMPLETED_EVENT,
/**
* An upload of a part of a multipart upload was started.
*/
TRANSFER_PART_STARTED_EVENT,
/**
* An upload of a part of a multipart upload was completed.
* This does not indicate the upload was successful.
*/
TRANSFER_PART_COMPLETED_EVENT,
TRANSFER_PART_FAILED_EVENT;
/**
* An upload of a part of a multipart upload was completed
* successfully.
*/
TRANSFER_PART_SUCCESS_EVENT,
/**
* An upload of a part of a multipart upload was abported.
*/
TRANSFER_PART_ABORTED_EVENT,
/**
* An upload of a part of a multipart upload failed.
*/
TRANSFER_PART_FAILED_EVENT,
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a.impl;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.S3AStore;
@ -50,6 +51,13 @@ public class S3AStoreBuilder {
private AuditSpanSource<AuditSpanS3A> auditSpanSource;
/**
* The original file system statistics: fairly minimal but broadly
* collected so it is important to pick up.
* This may be null.
*/
private FileSystem.Statistics fsStatistics;
public S3AStoreBuilder withStoreContextFactory(
final StoreContextFactory storeContextFactoryValue) {
this.storeContextFactory = storeContextFactoryValue;
@ -104,6 +112,11 @@ public S3AStoreBuilder withAuditSpanSource(
return this;
}
public S3AStoreBuilder withFsStatistics(final FileSystem.Statistics value) {
this.fsStatistics = value;
return this;
}
public S3AStore build() {
return new S3AStoreImpl(storeContextFactory,
clientManager,
@ -113,6 +126,7 @@ public S3AStore build() {
storageStatistics,
readRateLimiter,
writeRateLimiter,
auditSpanSource);
auditSpanSource,
fsStatistics);
}
}

View File

@ -18,34 +18,49 @@
package org.apache.hadoop.fs.s3a.impl;
import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletionException;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse;
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.S3Error;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import software.amazon.awssdk.transfer.s3.S3TransferManager;
import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
import software.amazon.awssdk.transfer.s3.model.FileUpload;
import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.ProgressableProgressListener;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.S3AStore;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.UploadInfo;
import org.apache.hadoop.fs.s3a.api.RequestFactory;
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
@ -57,11 +72,18 @@
import org.apache.hadoop.util.functional.Tuples;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3AUtils.extractException;
import static org.apache.hadoop.fs.s3a.S3AUtils.getPutRequestLength;
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS;
import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_BYTES;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_BYTES_PENDING;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS_COMPLETED;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RATE_LIMITED;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
@ -69,6 +91,7 @@
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
import static org.apache.hadoop.util.Preconditions.checkArgument;
/**
@ -80,6 +103,12 @@ public class S3AStoreImpl implements S3AStore {
private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class);
/**
* Progress logger; fairly noisy.
*/
private static final Logger PROGRESS =
LoggerFactory.getLogger(InternalConstants.UPLOAD_PROGRESS_LOG_NAME);
/** Factory to create store contexts. */
private final StoreContextFactory storeContextFactory;
@ -119,6 +148,13 @@ public class S3AStoreImpl implements S3AStore {
/** Audit span source. */
private final AuditSpanSource<AuditSpanS3A> auditSpanSource;
/**
* The original file system statistics: fairly minimal but broadly
* collected so it is important to pick up.
* This may be null.
*/
private final FileSystem.Statistics fsStatistics;
/** Constructor to create S3A store. */
S3AStoreImpl(StoreContextFactory storeContextFactory,
ClientManager clientManager,
@ -128,7 +164,8 @@ public class S3AStoreImpl implements S3AStore {
S3AStorageStatistics storageStatistics,
RateLimiting readRateLimiter,
RateLimiting writeRateLimiter,
AuditSpanSource<AuditSpanS3A> auditSpanSource) {
AuditSpanSource<AuditSpanS3A> auditSpanSource,
@Nullable FileSystem.Statistics fsStatistics) {
this.storeContextFactory = requireNonNull(storeContextFactory);
this.clientManager = requireNonNull(clientManager);
this.durationTrackerFactory = requireNonNull(durationTrackerFactory);
@ -139,6 +176,7 @@ public class S3AStoreImpl implements S3AStore {
this.writeRateLimiter = requireNonNull(writeRateLimiter);
this.auditSpanSource = requireNonNull(auditSpanSource);
this.storeContext = requireNonNull(storeContextFactory.createStoreContext());
this.fsStatistics = fsStatistics;
this.invoker = storeContext.getInvoker();
this.bucket = storeContext.getBucket();
this.requestFactory = storeContext.getRequestFactory();
@ -178,10 +216,10 @@ public StoreContext getStoreContext() {
/**
* Get the S3 client.
* @return the S3 client.
* @throws IOException on any failure to create the client.
* @throws UncheckedIOException on any failure to create the client.
*/
private S3Client getS3Client() throws IOException {
return clientManager.getOrCreateS3Client();
private S3Client getS3Client() throws UncheckedIOException {
return clientManager.getOrCreateS3ClientUnchecked();
}
@Override
@ -199,6 +237,16 @@ public S3AsyncClient getOrCreateAsyncClient() throws IOException {
return clientManager.getOrCreateAsyncClient();
}
@Override
public S3Client getOrCreateS3ClientUnchecked() throws UncheckedIOException {
return clientManager.getOrCreateS3ClientUnchecked();
}
@Override
public S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException {
return clientManager.getOrCreateAsyncS3ClientUnchecked();
}
@Override
public DurationTrackerFactory getDurationTrackerFactory() {
return durationTrackerFactory;
@ -305,6 +353,105 @@ public IOStatistics getIOStatistics() {
return instrumentation.getIOStatistics();
}
/**
* Increment read operations.
*/
@Override
public void incrementReadOperations() {
if (fsStatistics != null) {
fsStatistics.incrementReadOps(1);
}
}
/**
* Increment the write operation counter.
* This is somewhat inaccurate, as it appears to be invoked more
* often than needed in progress callbacks.
*/
@Override
public void incrementWriteOperations() {
if (fsStatistics != null) {
fsStatistics.incrementWriteOps(1);
}
}
/**
* Increment the bytes written statistic.
* @param bytes number of bytes written.
*/
private void incrementBytesWritten(final long bytes) {
if (fsStatistics != null) {
fsStatistics.incrementBytesWritten(bytes);
}
}
/**
* At the start of a put/multipart upload operation, update the
* relevant counters.
*
* @param bytes bytes in the request.
*/
@Override
public void incrementPutStartStatistics(long bytes) {
LOG.debug("PUT start {} bytes", bytes);
incrementWriteOperations();
incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
if (bytes > 0) {
incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
}
}
/**
* At the end of a put/multipart upload operation, update the
* relevant counters and gauges.
*
* @param success did the operation succeed?
* @param bytes bytes in the request.
*/
@Override
public void incrementPutCompletedStatistics(boolean success, long bytes) {
LOG.debug("PUT completed success={}; {} bytes", success, bytes);
if (bytes > 0) {
incrementStatistic(OBJECT_PUT_BYTES, bytes);
decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
}
incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
}
/**
* Callback for use in progress callbacks from put/multipart upload events.
* Increments those statistics which are expected to be updated during
* the ongoing upload operation.
* @param key key to file that is being written (for logging)
* @param bytes bytes successfully uploaded.
*/
@Override
public void incrementPutProgressStatistics(String key, long bytes) {
PROGRESS.debug("PUT {}: {} bytes", key, bytes);
incrementWriteOperations();
if (bytes > 0) {
incrementBytesWritten(bytes);
}
}
/**
* Given a possibly null duration tracker factory, return a non-null
* one for use in tracking durations -either that or the FS tracker
* itself.
*
* @param factory factory.
* @return a non-null factory.
*/
@Override
public DurationTrackerFactory nonNullDurationTrackerFactory(
DurationTrackerFactory factory) {
return factory != null
? factory
: getDurationTrackerFactory();
}
/**
* Start an operation; this informs the audit service of the event
* and then sets it as the active span.
@ -388,7 +535,6 @@ public Map.Entry<Duration, DeleteObjectsResponse> deleteObjects(
return Tuples.pair(d.asDuration(), response);
} catch (IOException e) {
// this is part of the retry signature, nothing else.
// convert to unchecked.
throw new UncheckedIOException(e);
}
@ -430,10 +576,125 @@ public Map.Entry<Duration, Optional<DeleteObjectResponse>> deleteObject(
d.close();
return Tuples.pair(d.asDuration(), Optional.empty());
} catch (IOException e) {
// this is part of the retry signature, nothing else.
// convert to unchecked.
throw new UncheckedIOException(e);
}
}
/**
* Upload part of a multi-partition file.
* Increments the write and put counters.
* <i>Important: this call does not close any input stream in the body.</i>
* <p>
* Retry Policy: none.
* @param trackerFactory duration tracker factory for operation
* @param request the upload part request.
* @param body the request body.
* @return the result of the operation.
* @throws AwsServiceException on problems
* @throws UncheckedIOException failure to instantiate the s3 client
*/
@Override
@Retries.OnceRaw
public UploadPartResponse uploadPart(
final UploadPartRequest request,
final RequestBody body,
@Nullable final DurationTrackerFactory trackerFactory)
throws AwsServiceException, UncheckedIOException {
long len = request.contentLength();
incrementPutStartStatistics(len);
try {
UploadPartResponse uploadPartResponse = trackDurationOfSupplier(
nonNullDurationTrackerFactory(trackerFactory),
MULTIPART_UPLOAD_PART_PUT.getSymbol(), () ->
getS3Client().uploadPart(request, body));
incrementPutCompletedStatistics(true, len);
return uploadPartResponse;
} catch (AwsServiceException e) {
incrementPutCompletedStatistics(false, len);
throw e;
}
}
/**
* Start a transfer-manager managed async PUT of an object,
* incrementing the put requests and put bytes
* counters.
* <p>
* It does not update the other counters,
* as existing code does that as progress callbacks come in.
* Byte length is calculated from the file length, or, if there is no
* file, from the content length of the header.
* <p>
* Because the operation is async, any stream supplied in the request
* must reference data (files, buffers) which stay valid until the upload
* completes.
* Retry policy: N/A: the transfer manager is performing the upload.
* Auditing: must be inside an audit span.
* @param putObjectRequest the request
* @param file the file to be uploaded
* @param listener the progress listener for the request
* @return the upload initiated
* @throws IOException if transfer manager creation failed.
*/
@Override
@Retries.OnceRaw
public UploadInfo putObject(
PutObjectRequest putObjectRequest,
File file,
ProgressableProgressListener listener) throws IOException {
long len = getPutRequestLength(putObjectRequest);
LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key());
incrementPutStartStatistics(len);
FileUpload upload = getOrCreateTransferManager().uploadFile(
UploadFileRequest.builder()
.putObjectRequest(putObjectRequest)
.source(file)
.addTransferListener(listener)
.build());
return new UploadInfo(upload, len);
}
/**
* Wait for an upload to complete.
* If the upload (or its result collection) failed, this is where
* the failure is raised as an AWS exception.
* Calls {@link S3AStore#incrementPutCompletedStatistics(boolean, long)}
* to update the statistics.
* @param key destination key
* @param uploadInfo upload to wait for
* @return the upload result
* @throws IOException IO failure
* @throws CancellationException if the wait() was cancelled
*/
@Override
@Retries.OnceTranslated
public CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
throws IOException {
FileUpload upload = uploadInfo.getFileUpload();
try {
CompletedFileUpload result = upload.completionFuture().join();
incrementPutCompletedStatistics(true, uploadInfo.getLength());
return result;
} catch (CompletionException e) {
LOG.info("Interrupted: aborting upload");
incrementPutCompletedStatistics(false, uploadInfo.getLength());
throw extractException("upload", key, e);
}
}
/**
* Complete a multipart upload.
* @param request request
* @return the response
*/
@Override
@Retries.OnceRaw
public CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request) {
return getS3Client().completeMultipartUpload(request);
}
}

View File

@ -0,0 +1,549 @@
/*
* 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.BufferedInputStream;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.File;
import java.io.FileInputStream;
import java.io.InputStream;
import java.io.UncheckedIOException;
import java.nio.ByteBuffer;
import java.util.function.Supplier;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import software.amazon.awssdk.http.ContentStreamProvider;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.store.ByteBufferInputStream;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.util.Preconditions.checkArgument;
import static org.apache.hadoop.util.Preconditions.checkState;
import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions;
/**
* Implementations of {@code software.amazon.awssdk.http.ContentStreamProvider}.
* <p>
* These are required to ensure that retry of multipart uploads are reliable,
* while also avoiding memory copy/consumption overhead.
* <p>
* For these reasons the providers built in to the AWS SDK are not used.
* <p>
* See HADOOP-19221 for details.
*/
public final class UploadContentProviders {
public static final Logger LOG = LoggerFactory.getLogger(UploadContentProviders.class);
private UploadContentProviders() {
}
/**
* Create a content provider from a file.
* @param file file to read.
* @param offset offset in file.
* @param size of data.
* @return the provider
* @throws IllegalArgumentException if the offset is negative.
*/
public static BaseContentProvider<BufferedInputStream> fileContentProvider(
File file,
long offset,
final int size) {
return new FileWithOffsetContentProvider(file, offset, size);
}
/**
* Create a content provider from a file.
* @param file file to read.
* @param offset offset in file.
* @param size of data.
* @param isOpen optional predicate to check if the stream is open.
* @return the provider
* @throws IllegalArgumentException if the offset is negative.
*/
public static BaseContentProvider<BufferedInputStream> fileContentProvider(
File file,
long offset,
final int size,
final Supplier<Boolean> isOpen) {
return new FileWithOffsetContentProvider(file, offset, size, isOpen);
}
/**
* Create a content provider from a byte buffer.
* The buffer is not copied and MUST NOT be modified while
* the upload is taking place.
* @param byteBuffer buffer to read.
* @param size size of the data.
* @return the provider
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null
*/
public static BaseContentProvider<ByteBufferInputStream> byteBufferContentProvider(
final ByteBuffer byteBuffer,
final int size) {
return new ByteBufferContentProvider(byteBuffer, size);
}
/**
* Create a content provider from a byte buffer.
* The buffer is not copied and MUST NOT be modified while
* the upload is taking place.
* @param byteBuffer buffer to read.
* @param size size of the data.
* @param isOpen optional predicate to check if the stream is open.
* @return the provider
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null
*/
public static BaseContentProvider<ByteBufferInputStream> byteBufferContentProvider(
final ByteBuffer byteBuffer,
final int size,
final @Nullable Supplier<Boolean> isOpen) {
return new ByteBufferContentProvider(byteBuffer, size, isOpen);
}
/**
* Create a content provider for all or part of a byte array.
* The buffer is not copied and MUST NOT be modified while
* the upload is taking place.
* @param bytes buffer to read.
* @param offset offset in buffer.
* @param size size of the data.
* @return the provider
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null.
*/
public static BaseContentProvider<ByteArrayInputStream> byteArrayContentProvider(
final byte[] bytes, final int offset, final int size) {
return new ByteArrayContentProvider(bytes, offset, size);
}
/**
* Create a content provider for all or part of a byte array.
* The buffer is not copied and MUST NOT be modified while
* the upload is taking place.
* @param bytes buffer to read.
* @param offset offset in buffer.
* @param size size of the data.
* @param isOpen optional predicate to check if the stream is open.
* @return the provider
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null.
*/
public static BaseContentProvider<ByteArrayInputStream> byteArrayContentProvider(
final byte[] bytes,
final int offset,
final int size,
final @Nullable Supplier<Boolean> isOpen) {
return new ByteArrayContentProvider(bytes, offset, size, isOpen);
}
/**
* Create a content provider for all of a byte array.
* @param bytes buffer to read.
* @return the provider
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null.
*/
public static BaseContentProvider<ByteArrayInputStream> byteArrayContentProvider(
final byte[] bytes) {
return byteArrayContentProvider(bytes, 0, bytes.length);
}
/**
* Create a content provider for all of a byte array.
* @param bytes buffer to read.
* @param isOpen optional predicate to check if the stream is open.
* @return the provider
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null.
*/
public static BaseContentProvider<ByteArrayInputStream> byteArrayContentProvider(
final byte[] bytes,
final @Nullable Supplier<Boolean> isOpen) {
return byteArrayContentProvider(bytes, 0, bytes.length, isOpen);
}
/**
* Base class for content providers; tracks the number of times a stream
* has been opened.
* @param <T> type of stream created.
*/
@VisibleForTesting
public static abstract class BaseContentProvider<T extends InputStream>
implements ContentStreamProvider, Closeable {
/**
* Size of the data.
*/
private final int size;
/**
* Probe to check if the stream is open.
* Invoked in {@link #checkOpen()}, which is itself
* invoked in {@link #newStream()}.
*/
private final Supplier<Boolean> isOpen;
/**
* How many times has a stream been created?
*/
private int streamCreationCount;
/**
* Current stream. Null if not opened yet.
* When {@link #newStream()} is called, this is set to the new value,
* Note: when the input stream itself is closed, this reference is not updated.
* Therefore this field not being null does not imply that the stream is open.
*/
private T currentStream;
/**
* Constructor.
* @param size size of the data. Must be non-negative.
*/
protected BaseContentProvider(int size) {
this(size, null);
}
/**
* Constructor.
* @param size size of the data. Must be non-negative.
* @param isOpen optional predicate to check if the stream is open.
*/
protected BaseContentProvider(int size, @Nullable Supplier<Boolean> isOpen) {
checkArgument(size >= 0, "size is negative: %s", size);
this.size = size;
this.isOpen = isOpen;
}
/**
* Check if the stream is open.
* If the stream is not open, raise an exception
* @throws IllegalStateException if the stream is not open.
*/
private void checkOpen() {
checkState(isOpen == null || isOpen.get(), "Stream is closed: %s", this);
}
/**
* Close the current stream.
*/
@Override
public void close() {
cleanupWithLogger(LOG, getCurrentStream());
setCurrentStream(null);
}
/**
* Create a new stream.
* <p>
* Calls {@link #close()} to ensure that any existing stream is closed,
* then {@link #checkOpen()} to verify that the data source is still open.
* Logs if this is a subsequent event as it implies a failure of the first attempt.
* @return the new stream
*/
@Override
public final InputStream newStream() {
close();
checkOpen();
streamCreationCount++;
if (streamCreationCount > 1) {
LOG.info("Stream created more than once: {}", this);
}
return setCurrentStream(createNewStream());
}
/**
* Override point for subclasses to create their new streams.
* @return a stream
*/
protected abstract T createNewStream();
/**
* How many times has a stream been created?
* @return stream creation count
*/
public int getStreamCreationCount() {
return streamCreationCount;
}
/**
* Size as set by constructor parameter.
* @return size of the data
*/
public int getSize() {
return size;
}
/**
* Current stream.
* When {@link #newStream()} is called, this is set to the new value,
* after closing the previous one.
* <p>
* Why? The AWS SDK implementations do this, so there
* is an implication that it is needed to avoid keeping streams
* open on retries.
* @return the current stream, or null if none is open.
*/
protected T getCurrentStream() {
return currentStream;
}
/**
* Set the current stream.
* @param stream the new stream
* @return the current stream.
*/
protected T setCurrentStream(T stream) {
this.currentStream = stream;
return stream;
}
@Override
public String toString() {
return "BaseContentProvider{" +
"size=" + size +
", streamCreationCount=" + streamCreationCount +
", currentStream=" + currentStream +
'}';
}
}
/**
* Content provider for a file with an offset.
*/
private static final class FileWithOffsetContentProvider
extends BaseContentProvider<BufferedInputStream> {
/**
* File to read.
*/
private final File file;
/**
* Offset in file.
*/
private final long offset;
/**
* Constructor.
* @param file file to read.
* @param offset offset in file.
* @param size of data.
* @param isOpen optional predicate to check if the stream is open.
* @throws IllegalArgumentException if the offset is negative.
*/
private FileWithOffsetContentProvider(
final File file,
final long offset,
final int size,
@Nullable final Supplier<Boolean> isOpen) {
super(size, isOpen);
this.file = requireNonNull(file);
checkArgument(offset >= 0, "Offset is negative: %s", offset);
this.offset = offset;
}
/**
* Constructor.
* @param file file to read.
* @param offset offset in file.
* @param size of data.
* @throws IllegalArgumentException if the offset is negative.
*/
private FileWithOffsetContentProvider(final File file,
final long offset,
final int size) {
this(file, offset, size, null);
}
/**
* Create a new stream.
* @return a stream at the start of the offset in the file
* @throws UncheckedIOException on IO failure.
*/
@Override
protected BufferedInputStream createNewStream() throws UncheckedIOException {
// create the stream, seek to the offset.
final FileInputStream fis = uncheckIOExceptions(() -> {
final FileInputStream f = new FileInputStream(file);
f.getChannel().position(offset);
return f;
});
return setCurrentStream(new BufferedInputStream(fis));
}
@Override
public String toString() {
return "FileWithOffsetContentProvider{" +
"file=" + file +
", offset=" + offset +
"} " + super.toString();
}
}
/**
* Create a content provider for a byte buffer.
* Uses {@link ByteBufferInputStream} to read the data.
*/
private static final class ByteBufferContentProvider
extends BaseContentProvider<ByteBufferInputStream> {
/**
* The buffer which will be read; on or off heap.
*/
private final ByteBuffer blockBuffer;
/**
* The position in the buffer at the time the provider was created.
*/
private final int initialPosition;
/**
* Constructor.
* @param blockBuffer buffer to read.
* @param size size of the data.
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null
*/
private ByteBufferContentProvider(final ByteBuffer blockBuffer, int size) {
this(blockBuffer, size, null);
}
/**
* Constructor.
* @param blockBuffer buffer to read.
* @param size size of the data.
* @param isOpen optional predicate to check if the stream is open.
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null
*/
private ByteBufferContentProvider(
final ByteBuffer blockBuffer,
int size,
@Nullable final Supplier<Boolean> isOpen) {
super(size, isOpen);
this.blockBuffer = blockBuffer;
this.initialPosition = blockBuffer.position();
}
@Override
protected ByteBufferInputStream createNewStream() {
// set the buffer up from reading from the beginning
blockBuffer.limit(initialPosition);
blockBuffer.position(0);
return new ByteBufferInputStream(getSize(), blockBuffer);
}
@Override
public String toString() {
return "ByteBufferContentProvider{" +
"blockBuffer=" + blockBuffer +
", initialPosition=" + initialPosition +
"} " + super.toString();
}
}
/**
* Simple byte array content provider.
* <p>
* The array is not copied; if it is changed during the write the outcome
* of the upload is undefined.
*/
private static final class ByteArrayContentProvider
extends BaseContentProvider<ByteArrayInputStream> {
/**
* The buffer where data is stored.
*/
private final byte[] bytes;
/**
* Offset in the buffer.
*/
private final int offset;
/**
* Constructor.
* @param bytes buffer to read.
* @param offset offset in buffer.
* @param size length of the data.
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null
*/
private ByteArrayContentProvider(
final byte[] bytes,
final int offset,
final int size) {
this(bytes, offset, size, null);
}
/**
* Constructor.
* @param bytes buffer to read.
* @param offset offset in buffer.
* @param size length of the data.
* @param isOpen optional predicate to check if the stream is open.
* @throws IllegalArgumentException if the arguments are invalid.
* @throws NullPointerException if the buffer is null
*/
private ByteArrayContentProvider(
final byte[] bytes,
final int offset,
final int size,
final Supplier<Boolean> isOpen) {
super(size, isOpen);
this.bytes = bytes;
this.offset = offset;
checkArgument(offset >= 0, "Offset is negative: %s", offset);
final int length = bytes.length;
checkArgument((offset + size) <= length,
"Data to read [%d-%d] is past end of array %s",
offset,
offset + size, length);
}
@Override
protected ByteArrayInputStream createNewStream() {
return new ByteArrayInputStream(bytes, offset, getSize());
}
@Override
public String toString() {
return "ByteArrayContentProvider{" +
"buffer with length=" + bytes.length +
", offset=" + offset +
"} " + super.toString();
}
}
}

View File

@ -42,7 +42,8 @@ public interface BlockOutputStreamStatistics extends Closeable,
void blockUploadStarted(Duration timeInQueue, long blockSize);
/**
* A block upload has completed. Duration excludes time in the queue.
* A block upload has completed, successfully or not.
* Duration excludes time in the queue.
* @param timeSinceUploadStarted time in since the transfer began.
* @param blockSize block size
*/

View File

@ -27,6 +27,16 @@
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_429_TOO_MANY_REQUESTS_GCS;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_500_INTERNAL_SERVER_ERROR;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_503_SERVICE_UNAVAILABLE;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_400;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_4XX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_500;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_503;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_5XX;
/**
* Hook up AWS SDK Statistics to the S3 counters.
@ -85,4 +95,37 @@ public void noteRequestSigningTime(final Duration duration) {
public void noteResponseProcessingTime(final Duration duration) {
}
/**
* Map error status codes to statistic names, excluding 404.
* 429 (google throttle events) are mapped to the 503 statistic.
* @param sc status code.
* @return a statistic name or null.
*/
public static String mapErrorStatusCodeToStatisticName(int sc) {
String stat = null;
switch (sc) {
case SC_400_BAD_REQUEST:
stat = HTTP_RESPONSE_400;
break;
case SC_404_NOT_FOUND:
/* do not map; not measured */
break;
case SC_500_INTERNAL_SERVER_ERROR:
stat = HTTP_RESPONSE_500;
break;
case SC_503_SERVICE_UNAVAILABLE:
case SC_429_TOO_MANY_REQUESTS_GCS:
stat = HTTP_RESPONSE_503;
break;
default:
if (sc > 500) {
stat = HTTP_RESPONSE_5XX;
} else if (sc > 400) {
stat = HTTP_RESPONSE_4XX;
}
}
return stat;
}
}

View File

@ -213,7 +213,26 @@ as they keep trying to reconnect to ports which are never going to be available.
<name>fs.s3a.bucket.nonexistent-bucket-example.connection.establish.timeout</name>
<value>500</value>
</property>
<property>
<name>fs.s3a.bucket.nonexistent-bucket-example.retry.http.5xx.errors</name>
<value>false</value>
</property>
```
Setting the option `fs.s3a.retry.http.5xx.errors` to `false` stops the S3A client from treating
500 and other HTTP 5xx status codes other than 501 and 503 as errors to retry on.
With AWS S3 they are eventually recovered from.
On a third-party store they may be cause by other problems, such as:
* General service misconfiguration
* Running out of disk storage
* Storage Permissions
Disabling the S3A client's retrying of these errors ensures that failures happen faster;
the AWS SDK itself still makes a limited attempt to retry.
## Cloudstore's Storediag
There's an external utility, [cloudstore](https://github.com/steveloughran/cloudstore) whose [storediag](https://github.com/steveloughran/cloudstore#command-storediag) exists to debug the connection settings to hadoop cloud storage.

View File

@ -29,7 +29,7 @@ Common problems working with S3 are:
7. [Other Errors](#other)
8. [SDK Upgrade Warnings](#upgrade_warnings)
This document also includes some [best pactises](#best) to aid troubleshooting.
This document also includes some [best practises](#best) to aid troubleshooting.
Troubleshooting IAM Assumed Roles is covered in its
@ -236,8 +236,61 @@ read requests are allowed, but operations which write to the bucket are denied.
Check the system clock.
### <a name="bad_request"></a> "Bad Request" exception when working with data stores in an AWS region other than us-eaast
### `Class does not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`
A credential provider listed in `fs.s3a.aws.credentials.provider` does not implement
the interface `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
```
InstantiationIOException: `s3a://stevel-gcs/': Class org.apache.hadoop.fs.s3a.S3ARetryPolicy does not implement
software.amazon.awssdk.auth.credentials.AwsCredentialsProvider (configuration key fs.s3a.aws.credentials.provider)
at org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf(InstantiationIOException.java:128)
at org.apache.hadoop.fs.s3a.S3AUtils.getInstanceFromReflection(S3AUtils.java:604)
at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSV2CredentialProvider(CredentialProviderListFactory.java:299)
at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList(CredentialProviderListFactory.java:245)
at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList(CredentialProviderListFactory.java:144)
at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:971)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:624)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3601)
at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:171)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3702)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3653)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:555)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366)
```
There's two main causes
1. A class listed there is not an implementation of the interface.
Fix: review the settings and correct as appropriate.
1. A class listed there does implement the interface, but it has been loaded in a different
classloader, so the JVM does not consider it to be an implementation.
Fix: learn the entire JVM classloader model and see if you can then debug it.
Tip: having both the AWS Shaded SDK and individual AWS SDK modules on your classpath
may be a cause of this.
If you see this and you are trying to use the S3A connector with Spark, then the cause can
be that the isolated classloader used to load Hive classes is interfering with the S3A
connector's dynamic loading of `software.amazon.awssdk` classes. To fix this, declare that
the classes in the aws SDK are loaded from the same classloader which instantiated
the S3A FileSystem instance:
```
spark.sql.hive.metastore.sharedPrefixes software.amazon.awssdk.
```
## <a name="400_bad_request"></a> 400 Bad Request errors
S3 stores return HTTP status code 400 "Bad Request" when the client make a request which
the store considers invalid.
This is most commonly caused by signing errors: secrets, region, even confusion between public and private
S3 stores.
### <a name="bad_request"></a> "Bad Request" exception when working with data stores in an AWS region other than us-east
```
@ -286,50 +339,59 @@ S3 region as `ca-central-1`.
</property>
```
### `Classdoes not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`
A credential provider listed in `fs.s3a.aws.credentials.provider` does not implement
the interface `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
### <a name="request_timeout"></a> 400 + RequestTimeout "Your socket connection to the server was not read from or written to within the timeout period"
```
InstantiationIOException: `s3a://stevel-gcs/': Class org.apache.hadoop.fs.s3a.S3ARetryPolicy does not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider (configuration key fs.s3a.aws.credentials.provider)
at org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf(InstantiationIOException.java:128)
at org.apache.hadoop.fs.s3a.S3AUtils.getInstanceFromReflection(S3AUtils.java:604)
at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSV2CredentialProvider(CredentialProviderListFactory.java:299)
at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList(CredentialProviderListFactory.java:245)
at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList(CredentialProviderListFactory.java:144)
at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:971)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:624)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3601)
at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:171)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3702)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3653)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:555)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366)
org.apache.hadoop.fs.s3a.AWSBadRequestException: upload part #1 upload ID 1122334455:
software.amazon.awssdk.services.s3.model.S3Exception:
Your socket connection to the server was not read from or written to within the timeout period.
Idle connections will be closed.
(Service: S3, Status Code: 400, Request ID: 1122334455, Extended Request ID: ...):
RequestTimeout:
Your socket connection to the server was not read from or written to within the timeout period.
Idle connections will be closed. (Service: S3, Status Code: 400, Request ID: 1122334455, Extended Request ID: ...
```
There's two main causes
This is an obscure failure which was encountered as part of
[HADOOP-19221](https://issues.apache.org/jira/browse/HADOOP-19221) : an upload of part of a file could not
be succesfully retried after a failure was reported on the first attempt.
1. A class listed there is not an implementation of the interface.
Fix: review the settings and correct as appropriate.
1. A class listed there does implement the interface, but it has been loaded in a different
classloader, so the JVM does not consider it to be an implementation.
Fix: learn the entire JVM classloader model and see if you can then debug it.
Tip: having both the AWS Shaded SDK and individual AWS SDK modules on your classpath
may be a cause of this.
1. It was only encountered during uploading files via the Staging Committers
2. And is a regression in the V2 SDK.
3. This should have been addressed in the S3A connector.
If you see this and you are trying to use the S3A connector with Spark, then the cause can
be that the isolated classloader used to load Hive classes is interfering with the S3A
connector's dynamic loading of `software.amazon.awssdk` classes. To fix this, declare that
the classes in the aws SDK are loaded from the same classloader which instantiated
the S3A FileSystem instance:
* If it is encountered on a hadoop release with HADOOP-19221, then this is a regression -please report it.
* If it is encountered on a release without the fix, please upgrade.
It may be that the problem arises in the AWS SDK's "TransferManager", which is used for a
higher performance upload of data from the local fileystem. If this is the case. disable this feature:
```
spark.sql.hive.metastore.sharedPrefixes software.amazon.awssdk.
<property>
<name>fs.s3a.optimized.copy.from.local.enabled</name>
<value>false</value>
</property>
```
## <a name="access_denied"></a> "The security token included in the request is invalid"
### Status Code 400 "One or more of the specified parts could not be found"
```
org.apache.hadoop.fs.s3a.AWSBadRequestException: Completing multipart upload on job-00-fork-0003/test/testTwoPartUpload:
software.amazon.awssdk.services.s3.model.S3Exception: One or more of the specified parts could not be found.
The part may not have been uploaded, or the specified entity tag may not match the part's entity tag.
(Service: S3, Status Code: 400, Request ID: EKNW2V7P34T7YK9E,
Extended Request ID: j64Dfdmfd2ZnjErbX1c05YmidLGx/5pJF9Io4B0w8Cx3aDTSFn1pW007BuzyxPeAbph/ZqXHjbU=):InvalidPart:
```
Happens if a multipart upload is being completed, but one of the parts is missing.
* An upload took so long that the part was deleted by the store
* A magic committer job's list of in-progress uploads somehow got corrupted
* Bug in the S3A codebase (rare, but not impossible...)
## <a name="access_denied"></a> Access Denied
HTTP error codes 401 and 403 are mapped to `AccessDeniedException` in the S3A connector.
### "The security token included in the request is invalid"
You are trying to use session/temporary credentials and the session token
supplied is considered invalid.
@ -501,7 +563,53 @@ endpoint and region like the following:
<value>${sts.region}</value>
</property>
```
## <a name="500_internal_error"></a> HTTP 500 status code "We encountered an internal error"
```
We encountered an internal error. Please try again.
(Service: S3, Status Code: 500, Request ID: <id>, Extended Request ID: <extended-id>)
```
The [status code 500](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/500) indicates
the S3 store has reported an internal problem.
When raised by Amazon S3, this is a rare sign of a problem within the S3 system
or another part of the cloud infrastructure on which it depends.
Retrying _should_ make it go away.
The 500 error is considered retryable by the AWS SDK, which will have already
tried it `fs.s3a.attempts.maximum` times before reaching the S3A client -which
will also retry.
The S3A client will attempt to retry on a 500 (or other 5xx error other than 501/503)
if the option `fs.s3a.retry.http.5xx.errors` is set to `true`.
This is the default.
```xml
<property>
<name>fs.s3a.retry.http.5xx.errors</name>
<value>true</value>
</property>
```
If encountered against a third party store (the lack of an extended request ID always implies this),
then it may be a permanent server-side failure.
* All HTTP status codes other than 503 (service unavailable) and 501 (unsupported) are
treated as 500 exceptions.
* The S3A Filesystem IOStatistics counts the number of 500 errors received.
## <a name="503 Throttling"></a> HTTP 503 status code "slow down" or 429 "Too Many Requests"
The [status code 503](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/503)
is returned by AWS S3 when the IO rate limit of the bucket is reached.
Google's cloud storage returns the response [429 Too Many Requests](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/429)
for the same situation.
The AWS S3 documentation [covers this and suggests mitigation strategies](https://repost.aws/knowledge-center/http-5xx-errors-s3).
Note that it can also be caused by throttling in the KMS bencryption subsystem if
SSE-KMS or DSSE-KMS is used to encrypt data.
Consult [performance - throttling](./performance.html#throttling) for details on throttling.
## <a name="connectivity"></a> Connectivity Problems

View File

@ -23,6 +23,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.io.IOUtils;
@ -127,7 +128,7 @@ private void verifyUpload(String name, int fileSize) throws IOException {
* @return the factory
*/
protected S3ADataBlocks.BlockFactory createFactory(S3AFileSystem fileSystem) {
return new S3ADataBlocks.ArrayBlockFactory(fileSystem);
return new S3ADataBlocks.ArrayBlockFactory(fileSystem.createStoreContext());
}
private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory)
@ -139,9 +140,9 @@ private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory)
S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats);
block.write(dataset, 0, dataset.length);
S3ADataBlocks.BlockUploadData uploadData = block.startUpload();
InputStream stream = uploadData.getUploadStream();
final UploadContentProviders.BaseContentProvider cp = uploadData.getContentProvider();
InputStream stream = cp.newStream();
assertNotNull(stream);
assertTrue("Mark not supported in " + stream, stream.markSupported());
assertEquals(0, stream.read());
stream.mark(BLOCK_SIZE);
// read a lot

View File

@ -27,7 +27,7 @@ protected String getBlockOutputBufferName() {
}
protected S3ADataBlocks.BlockFactory createFactory(S3AFileSystem fileSystem) {
return new S3ADataBlocks.ByteBufferBlockFactory(fileSystem);
return new S3ADataBlocks.ByteBufferBlockFactory(fileSystem.createStoreContext());
}
}

View File

@ -36,7 +36,7 @@ protected String getBlockOutputBufferName() {
* @return null
*/
protected S3ADataBlocks.BlockFactory createFactory(S3AFileSystem fileSystem) {
Assume.assumeTrue("mark/reset nopt supoprted", false);
Assume.assumeTrue("mark/reset not supported", false);
return null;
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.fs.s3a;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.net.URI;
import java.nio.charset.StandardCharsets;
@ -107,9 +106,11 @@ public void testPutObjectDirect() throws Throwable {
factory.newPutObjectRequestBuilder(path.toUri().getPath(), null, -1, false);
putObjectRequestBuilder.contentLength(-1L);
LambdaTestUtils.intercept(IllegalStateException.class,
() -> fs.putObjectDirect(putObjectRequestBuilder.build(), PutObjectOptions.keepingDirs(),
new S3ADataBlocks.BlockUploadData(new ByteArrayInputStream("PUT".getBytes())),
false, null));
() -> fs.putObjectDirect(
putObjectRequestBuilder.build(),
PutObjectOptions.keepingDirs(),
new S3ADataBlocks.BlockUploadData("PUT".getBytes(), null),
null));
assertPathDoesNotExist("put object was created", path);
}
}

View File

@ -50,6 +50,7 @@
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.fs.s3a.test.MinimalWriteOperationHelperCallbacks;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
import org.apache.hadoop.fs.store.audit.AuditSpan;
import org.apache.hadoop.util.Progressable;
@ -184,7 +185,7 @@ public void initialize(URI name, Configuration originalConf)
new EmptyS3AStatisticsContext(),
noopAuditor(conf),
AuditTestSupport.NOOP_SPAN,
new MinimalWriteOperationHelperCallbacks());
new MinimalWriteOperationHelperCallbacks(this::getS3Client));
}
@Override
@ -196,6 +197,11 @@ public WriteOperationHelper getWriteOperationHelper() {
return writeHelper;
}
@Override
public WriteOperationHelper createWriteOperationHelper(final AuditSpan auditSpan) {
return writeHelper;
}
@Override
public Configuration getConf() {
return conf;
@ -230,8 +236,6 @@ public boolean exists(Path f) throws IOException {
@Override
void finishedWrite(String key,
long length,
String eTag,
String versionId,
final PutObjectOptions putOptions) {
}

View File

@ -18,41 +18,90 @@
package org.apache.hadoop.fs.s3a;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.junit.Assert;
import org.junit.Before;
import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.Collection;
import java.util.Optional;
import org.assertj.core.api.Assertions;
import org.assertj.core.data.Index;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.store.ByteBufferInputStream;
import org.apache.hadoop.test.HadoopTestBase;
import static java.util.Optional.empty;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_ARRAY;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_DISK;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BYTEBUFFER;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Unit tests for {@link S3ADataBlocks}.
* Parameterized on the buffer type.
*/
public class TestDataBlocks extends Assert {
@RunWith(Parameterized.class)
public class TestDataBlocks extends HadoopTestBase {
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{FAST_UPLOAD_BUFFER_DISK},
{FAST_UPLOAD_BUFFER_ARRAY},
{FAST_UPLOAD_BYTEBUFFER}
});
}
@Rule
public Timeout testTimeout = new Timeout(30 * 1000);
public final TemporaryFolder tempDir = new TemporaryFolder();
@Before
public void nameThread() {
Thread.currentThread().setName("JUnit");
/**
* Buffer type.
*/
private final String bufferType;
public TestDataBlocks(final String bufferType) {
this.bufferType = bufferType;
}
/**
* Test the {@link S3ADataBlocks.ByteBufferBlockFactory}.
* That code implements an input stream over a ByteBuffer, and has to
* return the buffer to the pool after the read complete.
*
* This test verifies the basic contract of the process.
* Create a block factory.
* @return the factory
*/
private S3ADataBlocks.BlockFactory createFactory() {
switch (bufferType) {
// this one passed in a file allocation function
case FAST_UPLOAD_BUFFER_DISK:
return new S3ADataBlocks.DiskBlockFactory((i, l) ->
tempDir.newFile("file" + i));
case FAST_UPLOAD_BUFFER_ARRAY:
return new S3ADataBlocks.ArrayBlockFactory(null);
case FAST_UPLOAD_BYTEBUFFER:
return new S3ADataBlocks.ByteBufferBlockFactory(null);
default:
throw new IllegalArgumentException("Unknown buffer type: " + bufferType);
}
}
/**
* Test the content providers from the block factory and the streams
* they produce.
* There are extra assertions on the {@link ByteBufferInputStream}.
*/
@Test
public void testByteBufferIO() throws Throwable {
try (S3ADataBlocks.ByteBufferBlockFactory factory =
new S3ADataBlocks.ByteBufferBlockFactory(null)) {
public void testBlockFactoryIO() throws Throwable {
try (S3ADataBlocks.BlockFactory factory = createFactory()) {
int limit = 128;
S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock block
S3ADataBlocks.DataBlock block
= factory.create(1, limit, null);
assertOutstandingBuffers(factory, 1);
maybeAssertOutstandingBuffers(factory, 1);
byte[] buffer = ContractTestUtils.toAsciiByteArray("test data");
int bufferLen = buffer.length;
@ -66,32 +115,46 @@ public void testByteBufferIO() throws Throwable {
// now start the write
S3ADataBlocks.BlockUploadData blockUploadData = block.startUpload();
S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock.ByteBufferInputStream
stream =
(S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock.ByteBufferInputStream)
blockUploadData.getUploadStream();
assertTrue("Mark not supported in " + stream, stream.markSupported());
assertTrue("!hasRemaining() in " + stream, stream.hasRemaining());
int expected = bufferLen;
assertEquals("wrong available() in " + stream,
expected, stream.available());
final UploadContentProviders.BaseContentProvider<?> cp =
blockUploadData.getContentProvider();
assertEquals('t', stream.read());
stream.mark(limit);
assertStreamCreationCount(cp, 0);
InputStream stream = cp.newStream();
assertStreamCreationCount(cp, 1);
Assertions.assertThat(stream.markSupported())
.describedAs("markSupported() of %s", stream)
.isTrue();
Optional<ByteBufferInputStream> bbStream =
stream instanceof ByteBufferInputStream
? Optional.of((ByteBufferInputStream) stream)
: empty();
bbStream.ifPresent(bb -> {
Assertions.assertThat(bb.hasRemaining())
.describedAs("hasRemaining() in %s", bb)
.isTrue();
});
int expected = bufferLen;
assertAvailableValue(stream, expected);
assertReadEquals(stream, 't');
stream.mark(Integer.MAX_VALUE);
expected--;
assertEquals("wrong available() in " + stream,
expected, stream.available());
assertAvailableValue(stream, expected);
// read into a byte array with an offset
int offset = 5;
byte[] in = new byte[limit];
assertEquals(2, stream.read(in, offset, 2));
assertEquals('e', in[offset]);
assertEquals('s', in[offset + 1]);
assertByteAtIndex(in, offset++, 'e');
assertByteAtIndex(in, offset++, 's');
expected -= 2;
assertEquals("wrong available() in " + stream,
expected, stream.available());
assertAvailableValue(stream, expected);
// read to end
byte[] remainder = new byte[limit];
@ -101,37 +164,107 @@ public void testByteBufferIO() throws Throwable {
remainder[index++] = (byte) c;
}
assertEquals(expected, index);
assertEquals('a', remainder[--index]);
assertByteAtIndex(remainder, --index, 'a');
assertEquals("wrong available() in " + stream,
0, stream.available());
assertTrue("hasRemaining() in " + stream, !stream.hasRemaining());
// no more data left
assertAvailableValue(stream, 0);
bbStream.ifPresent(bb -> {
Assertions.assertThat(bb.hasRemaining())
.describedAs("hasRemaining() in %s", bb)
.isFalse();
});
// at the end of the stream, a read fails
assertReadEquals(stream, -1);
// go the mark point
stream.reset();
assertEquals('e', stream.read());
assertAvailableValue(stream, bufferLen - 1);
assertReadEquals(stream, 'e');
// when the stream is closed, the data should be returned
stream.close();
assertOutstandingBuffers(factory, 1);
// now ask the content provider for another content stream.
final InputStream stream2 = cp.newStream();
assertStreamCreationCount(cp, 2);
// this must close the old stream
bbStream.ifPresent(bb -> {
Assertions.assertThat(bb.isOpen())
.describedAs("stream %s is open", bb)
.isFalse();
});
// do a read(byte[]) of everything
byte[] readBuffer = new byte[bufferLen];
Assertions.assertThat(stream2.read(readBuffer))
.describedAs("number of bytes read from stream %s", stream2)
.isEqualTo(bufferLen);
Assertions.assertThat(readBuffer)
.describedAs("data read into buffer")
.isEqualTo(buffer);
// when the block is closed, the buffer must be returned
// to the pool.
block.close();
assertOutstandingBuffers(factory, 0);
maybeAssertOutstandingBuffers(factory, 0);
stream.close();
assertOutstandingBuffers(factory, 0);
maybeAssertOutstandingBuffers(factory, 0);
// now the block is closed, the content provider must fail to
// create a new stream
intercept(IllegalStateException.class, cp::newStream);
}
}
private static void assertByteAtIndex(final byte[] bytes,
final int index, final char expected) {
Assertions.assertThat(bytes)
.contains(expected, Index.atIndex(index));
}
private static void assertReadEquals(final InputStream stream,
final int ch)
throws IOException {
Assertions.assertThat(stream.read())
.describedAs("read() in %s", stream)
.isEqualTo(ch);
}
private static void assertAvailableValue(final InputStream stream,
final int expected) throws IOException {
Assertions.assertThat(stream.available())
.describedAs("wrong available() in %s", stream)
.isEqualTo(expected);
}
private static void assertStreamCreationCount(
final UploadContentProviders.BaseContentProvider<?> cp,
final int count) {
Assertions.assertThat(cp.getStreamCreationCount())
.describedAs("stream creation count of %s", cp)
.isEqualTo(count);
}
/**
* Assert the number of buffers active for a block factory.
* Assert the number of buffers active for a block factory,
* if the factory is a ByteBufferBlockFactory.
* <p>
* If it is of any other type, no checks are made.
* @param factory factory
* @param expectedCount expected count.
*/
private static void assertOutstandingBuffers(
S3ADataBlocks.ByteBufferBlockFactory factory,
private static void maybeAssertOutstandingBuffers(
S3ADataBlocks.BlockFactory factory,
int expectedCount) {
assertEquals("outstanding buffers in " + factory,
expectedCount, factory.getOutstandingBufferCount());
if (factory instanceof S3ADataBlocks.ByteBufferBlockFactory) {
S3ADataBlocks.ByteBufferBlockFactory bufferFactory =
(S3ADataBlocks.ByteBufferBlockFactory) factory;
Assertions.assertThat(bufferFactory.getOutstandingBufferCount())
.describedAs("outstanding buffers in %s", factory)
.isEqualTo(expectedCount);
}
}
}

View File

@ -26,36 +26,42 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;
import org.assertj.core.api.Assertions;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.services.s3.model.S3Exception;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.net.ConnectTimeoutException;
import org.apache.hadoop.test.HadoopTestBase;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.Invoker.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.verifyExceptionClass;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_500_INTERNAL_SERVER_ERROR;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_501_NOT_IMPLEMENTED;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_503_SERVICE_UNAVAILABLE;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_504_GATEWAY_TIMEOUT;
import static org.apache.hadoop.test.LambdaTestUtils.*;
/**
* Test the {@link Invoker} code and the associated {@link S3ARetryPolicy}.
*
* <p>
* Some of the tests look at how Connection Timeout Exceptions are processed.
* Because of how the AWS libraries shade the classes, there have been some
* regressions here during development. These tests are intended to verify that
* the current match process based on classname works.
* <p>
* 500 errors may or may not be considered retriable; these tests validate
* both configurations with different retry policies for each.
*/
@SuppressWarnings("ThrowableNotThrown")
public class TestInvoker extends Assert {
public class TestInvoker extends HadoopTestBase {
/** Configuration to use for short retry intervals. */
private static final Configuration FAST_RETRY_CONF;
@ -85,6 +91,15 @@ public class TestInvoker extends Assert {
*/
public static final int SAFE_RETRY_COUNT = 5;
public static final String INTERNAL_ERROR_PLEASE_TRY_AGAIN =
"We encountered an internal error. Please try again";
/**
* Retry configuration derived from {@link #FAST_RETRY_CONF} with 500 errors
* never retried.
*/
public static final Configuration RETRY_EXCEPT_500_ERRORS;
static {
FAST_RETRY_CONF = new Configuration();
String interval = "10ms";
@ -92,18 +107,48 @@ public class TestInvoker extends Assert {
FAST_RETRY_CONF.set(RETRY_THROTTLE_INTERVAL, interval);
FAST_RETRY_CONF.setInt(RETRY_LIMIT, ACTIVE_RETRY_LIMIT);
FAST_RETRY_CONF.setInt(RETRY_THROTTLE_LIMIT, ACTIVE_RETRY_LIMIT);
FAST_RETRY_CONF.setBoolean(RETRY_HTTP_5XX_ERRORS, DEFAULT_RETRY_HTTP_5XX_ERRORS);
RETRY_EXCEPT_500_ERRORS = new Configuration(FAST_RETRY_CONF);
RETRY_EXCEPT_500_ERRORS.setBoolean(RETRY_HTTP_5XX_ERRORS, false);
}
/**
* Retry policy with 500 error retry the default.
*/
private static final S3ARetryPolicy RETRY_POLICY =
new S3ARetryPolicy(FAST_RETRY_CONF);
/**
* Retry policyd with 500 errors never retried.
*/
private static final S3ARetryPolicy RETRY_POLICY_NO_500_ERRORS =
new S3ARetryPolicy(RETRY_EXCEPT_500_ERRORS);
/**
* Count of retries performed when invoking an operation which
* failed.
*/
private int retryCount;
private Invoker invoker = new Invoker(RETRY_POLICY,
(text, e, retries, idempotent) -> retryCount++);
/**
* Retry handler which increments {@link #retryCount}.
*/
private final Retried retryHandler = (text, e, retries, idempotent) -> retryCount++;
private final Invoker invoker = new Invoker(RETRY_POLICY, retryHandler);
/**
* AWS SDK exception wrapping a ConnectTimeoutException.
*/
private static final SdkException CLIENT_TIMEOUT_EXCEPTION =
SdkException.builder()
.cause(new Local.ConnectTimeoutException("timeout"))
.build();
/**
* AWS SDK 400 Bad Request exception.
*/
private static final AwsServiceException BAD_REQUEST = serviceException(
SC_400_BAD_REQUEST,
"bad request");
@ -147,28 +192,145 @@ private static <E extends Throwable> E verifyTranslated(Class<E> clazz,
translateException("test", "/", exception));
}
/**
* jReset the retry count.
*/
private void resetCounters() {
retryCount = 0;
}
@Test
public void test503isThrottled() throws Exception {
verifyTranslated(503, AWSServiceThrottledException.class);
verifyTranslated(SC_503_SERVICE_UNAVAILABLE, AWSServiceThrottledException.class);
}
@Test
public void testS3500isStatus500Exception() throws Exception {
verifyTranslated(500, AWSStatus500Exception.class);
verifyTranslated(SC_500_INTERNAL_SERVER_ERROR, AWSStatus500Exception.class);
}
/**
* 500 error handling with the default options: the responses
* trigger retry.
*/
@Test
public void test500isStatus500Exception() throws Exception {
AwsServiceException ex = AwsServiceException.builder()
.message("")
.statusCode(500)
public void test500ResponseHandling() throws Exception {
// create a 500 SDK Exception;
AwsServiceException ex = awsException(SC_500_INTERNAL_SERVER_ERROR,
INTERNAL_ERROR_PLEASE_TRY_AGAIN);
// translate this to a Hadoop IOE.
AWSStatus500Exception ex500 =
verifyTranslated(AWSStatus500Exception.class, ex);
// the status code is preserved
Assertions.assertThat(ex500.statusCode())
.describedAs("status code of %s", ex)
.isEqualTo(SC_500_INTERNAL_SERVER_ERROR);
// the default retry policies reject this and fail
assertRetryAction("Expected retry on 500 error",
RETRY_POLICY, RetryPolicy.RetryAction.RETRY,
ex, 0, true);
Assertions.assertThat(invoker.getRetryPolicy()
.shouldRetry(ex500, 1, 0, false).action)
.describedAs("should retry %s", ex500)
.isEqualTo(RetryPolicy.RetryAction.RETRY.action);
}
/**
* Validate behavior on 500 errors when retry is disabled.
*/
@Test
public void test500ResponseHandlingRetryDisabled() throws Exception {
// create a 500 SDK Exception;
AwsServiceException ex = awsException(SC_500_INTERNAL_SERVER_ERROR,
INTERNAL_ERROR_PLEASE_TRY_AGAIN);
// translate this to a Hadoop IOE.
AWSStatus500Exception ex500 =
verifyTranslated(AWSStatus500Exception.class, ex);
// the no 500 retry policies reject this and fail
final Invoker failingInvoker = new Invoker(RETRY_POLICY_NO_500_ERRORS, retryHandler);
assertRetryAction("Expected failure first throttle",
RETRY_POLICY_NO_500_ERRORS, RetryPolicy.RetryAction.FAIL,
ex, 0, true);
Assertions.assertThat(failingInvoker.getRetryPolicy()
.shouldRetry(ex500, 1, 0, false).action)
.describedAs("should retry %s", ex500)
.isEqualTo(RetryPolicy.RetryAction.FAIL.action);
}
/**
* A 501 error is never retried.
*/
@Test
public void test501UnsupportedFeatureNoRetry() throws Throwable {
AwsServiceException ex = awsException(501,
"501 We encountered an internal error. Please try again");
final AWSUnsupportedFeatureException ex501 =
intercept(AWSUnsupportedFeatureException.class, "501", () ->
invoker.retry("ex", null, true, () -> {
throw ex;
}));
Assertions.assertThat(ex501.statusCode())
.describedAs("status code of %s", ex)
.isEqualTo(501);
Assertions.assertThat(retryCount)
.describedAs("retry count")
.isEqualTo(0);
}
/**
* Construct an S3Exception.
* @param statusCode status code
* @param message message
* @return the exception
*/
private static AwsServiceException awsException(final int statusCode, final String message) {
return S3Exception.builder()
.statusCode(statusCode)
.message(message)
.requestId("reqID")
.extendedRequestId("extreqID")
.build();
verifyTranslated(AWSStatus500Exception.class,
ex);
}
/**
* Assert expected retry actions on 5xx responses when 5xx errors are disabled.
*/
@Test
public void test5xxRetriesDisabled() throws Throwable {
final S3ARetryPolicy policy = RETRY_POLICY_NO_500_ERRORS;
assertRetryAction("500", policy, RetryPolicy.RetryAction.FAIL,
awsException(SC_500_INTERNAL_SERVER_ERROR, INTERNAL_ERROR_PLEASE_TRY_AGAIN), 1, true);
assertRetryAction("501", policy, RetryPolicy.RetryAction.FAIL,
awsException(SC_501_NOT_IMPLEMENTED, INTERNAL_ERROR_PLEASE_TRY_AGAIN), 1, true);
assertRetryAction("510", policy, RetryPolicy.RetryAction.FAIL,
awsException(510, INTERNAL_ERROR_PLEASE_TRY_AGAIN), 1, true);
assertRetryAction("gateway", policy, RetryPolicy.RetryAction.RETRY,
awsException(SC_504_GATEWAY_TIMEOUT, "gateway"), 1, true);
}
/**
* Various 5xx exceptions when 5xx errors are enabled.
*/
@Test
public void test5xxRetriesEnabled() throws Throwable {
final Configuration conf = new Configuration(FAST_RETRY_CONF);
conf.setBoolean(RETRY_HTTP_5XX_ERRORS, true);
final S3ARetryPolicy policy = new S3ARetryPolicy(conf);
assertRetryAction("500", policy, RetryPolicy.RetryAction.RETRY,
awsException(SC_500_INTERNAL_SERVER_ERROR, INTERNAL_ERROR_PLEASE_TRY_AGAIN), 1, true);
assertRetryAction("501", policy, RetryPolicy.RetryAction.FAIL,
awsException(SC_501_NOT_IMPLEMENTED, INTERNAL_ERROR_PLEASE_TRY_AGAIN), 1, true);
assertRetryAction("510", policy, RetryPolicy.RetryAction.RETRY,
awsException(510, INTERNAL_ERROR_PLEASE_TRY_AGAIN), 1, true);
assertRetryAction("gateway", policy, RetryPolicy.RetryAction.RETRY,
awsException(SC_504_GATEWAY_TIMEOUT, "gateway"), 1, true);
}
@Test
@ -261,23 +423,17 @@ public void testExtractSocketTimeoutExceptionFromCompletionException() throws Th
* @throws AssertionError if the returned action was not that expected.
*/
private void assertRetryAction(String text,
S3ARetryPolicy policy,
RetryPolicy policy,
RetryPolicy.RetryAction expected,
Exception ex,
int retries,
boolean idempotent) throws Exception {
RetryPolicy.RetryAction outcome = policy.shouldRetry(ex, retries, 0,
idempotent);
if (!expected.action.equals(outcome.action)) {
throw new AssertionError(
String.format(
"%s Expected action %s from shouldRetry(%s, %s, %s), but got"
+ " %s",
text,
expected, ex.toString(), retries, idempotent,
outcome.action),
ex);
}
Assertions.assertThat(outcome.action)
.describedAs("%s Expected action %s from shouldRetry(%s, %s, %s)",
text, expected, ex.toString(), retries, idempotent)
.isEqualTo(expected.action);
}
@Test

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.fs.s3a;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ClosedIOException;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.fs.s3a.audit.AuditTestSupport;
import org.apache.hadoop.fs.s3a.commit.PutTracker;
@ -30,7 +31,6 @@
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.concurrent.ExecutorService;
import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditor;
@ -86,7 +86,7 @@ public void setUp() throws Exception {
@Test
public void testFlushNoOpWhenStreamClosed() throws Exception {
doThrow(new IOException()).when(stream).checkOpen();
doThrow(new StreamClosedException()).when(stream).checkOpen();
stream.flush();
}
@ -103,7 +103,7 @@ public void testWriteOperationHelperPartLimits() throws Throwable {
new EmptyS3AStatisticsContext(),
noopAuditor(conf),
AuditTestSupport.NOOP_SPAN,
new MinimalWriteOperationHelperCallbacks());
new MinimalWriteOperationHelperCallbacks(null)); // raises NPE if S3 client used
// first one works
String key = "destKey";
woh.newUploadPartRequestBuilder(key,
@ -114,7 +114,12 @@ public void testWriteOperationHelperPartLimits() throws Throwable {
"uploadId", 50000, 1024));
}
static class StreamClosedException extends IOException {}
static class StreamClosedException extends ClosedIOException {
StreamClosedException() {
super("path", "message");
}
}
@Test
public void testStreamClosedAfterAbort() throws Exception {
@ -122,7 +127,7 @@ public void testStreamClosedAfterAbort() throws Exception {
// This verification replaces testing various operations after calling
// abort: after calling abort, stream is closed like calling close().
intercept(IOException.class, () -> stream.checkOpen());
intercept(ClosedIOException.class, () -> stream.checkOpen());
// check that calling write() will call checkOpen() and throws exception
doThrow(new StreamClosedException()).when(stream).checkOpen();

View File

@ -36,6 +36,11 @@
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.NOOP_AUDIT_SERVICE;
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_ENABLED;
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_400;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_4XX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_500;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_503;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_5XX;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
/**
@ -105,7 +110,12 @@ public static IOStatisticsStore createIOStatisticsStoreForAuditing() {
AUDIT_ACCESS_CHECK_FAILURE.getSymbol(),
AUDIT_FAILURE.getSymbol(),
AUDIT_REQUEST_EXECUTION.getSymbol(),
AUDIT_SPAN_CREATION.getSymbol())
AUDIT_SPAN_CREATION.getSymbol(),
HTTP_RESPONSE_400,
HTTP_RESPONSE_4XX,
HTTP_RESPONSE_500,
HTTP_RESPONSE_503,
HTTP_RESPONSE_5XX)
.build();
}

View File

@ -18,10 +18,14 @@
package org.apache.hadoop.fs.s3a.audit;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.core.interceptor.InterceptorContext;
import software.amazon.awssdk.core.internal.interceptor.DefaultFailedExecutionContext;
import software.amazon.awssdk.http.SdkHttpResponse;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest;
import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
import software.amazon.awssdk.services.s3.model.UploadPartCopyRequest;
import software.amazon.awssdk.transfer.s3.progress.TransferListener;
import org.junit.Before;
@ -34,6 +38,8 @@
import org.apache.hadoop.fs.store.audit.AuditSpan;
import static org.apache.hadoop.fs.s3a.Statistic.HTTP_RESPONSE_400;
import static org.apache.hadoop.fs.s3a.Statistic.HTTP_RESPONSE_500;
import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig;
import static org.assertj.core.api.Assertions.assertThat;
@ -219,4 +225,39 @@ public void testSpanIdsAreDifferent() throws Throwable {
assertThat(s1.getSpanId())
.doesNotMatch(s2.getSpanId());
}
/**
* Verify that the auditor processes 400 exceptions.
*/
@Test
public void testErrorCode400Extraction() throws Throwable {
span().onExecutionFailure(createFailureContext(400),
ExecutionAttributes.builder().build());
verifyCounter(HTTP_RESPONSE_400, 1);
}
/**
* Verify that the auditor processes 500 exceptions.
*/
@Test
public void testErrorCode500Extraction() throws Throwable {
span().onExecutionFailure(createFailureContext(500),
ExecutionAttributes.builder().build());
verifyCounter(HTTP_RESPONSE_500, 1);
}
private static DefaultFailedExecutionContext createFailureContext(final int statusCode) {
final DefaultFailedExecutionContext failedExecutionContext =
DefaultFailedExecutionContext.builder()
.exception(SdkClientException.builder().message(Integer.toString(statusCode)).build())
.interceptorContext(
InterceptorContext.builder()
.request(HeadBucketRequest.builder().bucket("bucket").build())
.httpResponse(SdkHttpResponse.builder()
.statusCode(statusCode)
.build())
.build()
).build();
return failedExecutionContext;
}
}

View File

@ -0,0 +1,259 @@
/*
* 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.commit;
import java.io.File;
import java.util.Arrays;
import java.util.Collection;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import org.assertj.core.api.Assertions;
import org.assertj.core.api.Assumptions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import software.amazon.awssdk.core.interceptor.Context;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
import org.apache.hadoop.fs.s3a.commit.impl.CommitContext;
import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations;
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
import org.apache.hadoop.fs.s3a.test.SdkFaultInjector;
import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MULTIPART_SIZE;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_OPERATIONS_PURGE_UPLOADS;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_ARRAY;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_DISK;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BYTEBUFFER;
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES;
import static org.apache.hadoop.fs.s3a.Constants.RETRY_HTTP_5XX_ERRORS;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.BASE;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX;
import static org.apache.hadoop.fs.s3a.test.SdkFaultInjector.setRequestFailureConditions;
/**
* Test upload recovery by injecting failures into the response chain.
* The tests are parameterized on upload buffering.
* <p>
* The test case {@link #testCommitOperations()} is independent of this option;
* the test parameterization only runs this once.
* A bit inelegant but as the fault injection code is shared and the problem "adjacent"
* this isolates all forms of upload recovery into the same test class without
* wasting time with duplicate uploads.
* <p>
* Fault injection is implemented in {@link SdkFaultInjector}.
*/
@RunWith(Parameterized.class)
public class ITestUploadRecovery extends AbstractS3ACostTest {
private static final Logger LOG =
LoggerFactory.getLogger(ITestUploadRecovery.class);
/**
* Parameterization.
*/
@Parameterized.Parameters(name = "{0}-commit-{1}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{FAST_UPLOAD_BUFFER_ARRAY, true},
{FAST_UPLOAD_BUFFER_DISK, false},
{FAST_UPLOAD_BYTEBUFFER, false},
});
}
private static final String JOB_ID = UUID.randomUUID().toString();
/**
* Upload size for the committer test.
*/
public static final int COMMIT_FILE_UPLOAD_SIZE = 1024 * 2;
/**
* should the commit test be included?
*/
private final boolean includeCommitTest;
/**
* Buffer type for this test run.
*/
private final String buffer;
/**
* Parameterized test suite.
* @param buffer buffer type
* @param includeCommitTest should the commit upload test be included?
*/
public ITestUploadRecovery(final String buffer, final boolean includeCommitTest) {
this.includeCommitTest = includeCommitTest;
this.buffer = buffer;
}
@Override
public Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
removeBaseAndBucketOverrides(conf,
AUDIT_EXECUTION_INTERCEPTORS,
DIRECTORY_OPERATIONS_PURGE_UPLOADS,
FAST_UPLOAD_BUFFER,
FS_S3A_CREATE_PERFORMANCE,
MAX_ERROR_RETRIES,
RETRY_HTTP_5XX_ERRORS);
// select buffer location
conf.set(FAST_UPLOAD_BUFFER, buffer);
// save overhead on file creation
conf.setBoolean(FS_S3A_CREATE_PERFORMANCE, true);
// guarantees teardown will abort pending uploads.
conf.setBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, true);
// fail fast on 500 errors
conf.setBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, false);
// use the fault injector
SdkFaultInjector.addFaultInjection(conf);
return conf;
}
/**
* Setup MUST set up the evaluator before the FS is created.
*/
@Override
public void setup() throws Exception {
SdkFaultInjector.resetEvaluator();
super.setup();
}
@Override
public void teardown() throws Exception {
// safety check in case the evaluation is failing any
// request needed in cleanup.
SdkFaultInjector.resetEvaluator();
super.teardown();
}
/**
* Verify that failures of simple PUT requests can be recovered from.
*/
@Test
public void testPutRecovery() throws Throwable {
describe("test put recovery");
final S3AFileSystem fs = getFileSystem();
final Path path = methodPath();
final int attempts = 2;
final Function<Context.ModifyHttpResponse, Boolean> evaluator =
SdkFaultInjector::isPutRequest;
setRequestFailureConditions(attempts, evaluator);
final FSDataOutputStream out = fs.create(path);
out.writeUTF("utfstring");
out.close();
}
/**
* Validate recovery of multipart uploads within a magic write sequence.
*/
@Test
public void testMagicWriteRecovery() throws Throwable {
describe("test magic write recovery with multipart uploads");
final S3AFileSystem fs = getFileSystem();
Assumptions.assumeThat(fs.isMultipartUploadEnabled())
.describedAs("Multipart upload is disabled")
.isTrue();
final Path path = new Path(methodPath(),
MAGIC_PATH_PREFIX + buffer + "/" + BASE + "/file.txt");
SdkFaultInjector.setEvaluator(SdkFaultInjector::isPartUpload);
final FSDataOutputStream out = fs.create(path);
// set the failure count again
SdkFaultInjector.setRequestFailureCount(2);
out.writeUTF("utfstring");
out.close();
}
/**
* Test the commit operations iff {@link #includeCommitTest} is true.
*/
@Test
public void testCommitOperations() throws Throwable {
Assumptions.assumeThat(includeCommitTest)
.describedAs("commit test excluded")
.isTrue();
describe("test staging upload");
final S3AFileSystem fs = getFileSystem();
// write a file to the local fS, to simulate a staged upload
final byte[] dataset = ContractTestUtils.dataset(COMMIT_FILE_UPLOAD_SIZE, '0', 36);
File tempFile = File.createTempFile("commit", ".txt");
FileUtils.writeByteArrayToFile(tempFile, dataset);
CommitOperations actions = new CommitOperations(fs);
Path dest = methodPath();
setRequestFailureConditions(2, SdkFaultInjector::isPartUpload);
// upload from the local FS to the S3 store.
// making sure that progress callbacks occur
AtomicLong progress = new AtomicLong(0);
SinglePendingCommit commit =
actions.uploadFileToPendingCommit(tempFile,
dest,
null,
DEFAULT_MULTIPART_SIZE,
progress::incrementAndGet);
// at this point the upload must have succeeded, despite the failures.
// commit will fail twice on the complete call.
setRequestFailureConditions(2,
SdkFaultInjector::isCompleteMultipartUploadRequest);
try (CommitContext commitContext
= actions.createCommitContextForTesting(dest, JOB_ID, 0)) {
commitContext.commitOrFail(commit);
}
// make sure the saved data is as expected
verifyFileContents(fs, dest, dataset);
// and that we got some progress callbacks during upload
Assertions.assertThat(progress.get())
.describedAs("progress count")
.isGreaterThan(0);
}
}

View File

@ -202,6 +202,15 @@ private Configuration newConfig() {
return new Configuration(false);
}
@Test
public void testMockFSclientWiredUp() throws Throwable {
final S3Client client = mockFS.getS3AInternals().getAmazonS3Client("test");
Assertions.assertThat(client)
.describedAs("S3Client from FS")
.isNotNull()
.isSameAs(mockClient);
}
@Test
public void testUUIDPropagation() throws Exception {
Configuration config = newConfig();

View File

@ -301,7 +301,7 @@ protected int directoriesInPath(Path path) {
/**
* Reset all the metrics being tracked.
*/
private void resetStatistics() {
protected void resetStatistics() {
costValidator.resetMetricDiffs();
}

View File

@ -24,8 +24,6 @@
import java.time.Duration;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.IntFunction;
import org.assertj.core.api.Assertions;
@ -47,14 +45,11 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.impl.ProgressListener;
import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.io.ElasticByteBufferPool;
import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.Progressable;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
@ -223,7 +218,7 @@ public void test_010_CreateHugeFile() throws IOException {
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
BlockOutputStreamStatistics streamStatistics;
long blocksPer10MB = blocksPerMB * 10;
ProgressCallback progress = new ProgressCallback(timer);
CountingProgressListener progress = new CountingProgressListener(timer);
try (FSDataOutputStream out = fs.create(fileToCreate,
true,
uploadBlockSize,
@ -388,84 +383,6 @@ protected int getPartitionSize() {
return partitionSize;
}
/**
* Progress callback.
*/
private final class ProgressCallback implements Progressable, ProgressListener {
private AtomicLong bytesTransferred = new AtomicLong(0);
private AtomicLong uploadEvents = new AtomicLong(0);
private AtomicInteger failures = new AtomicInteger(0);
private final ContractTestUtils.NanoTimer timer;
private ProgressCallback(NanoTimer timer) {
this.timer = timer;
}
@Override
public void progress() {
}
@Override
public void progressChanged(ProgressListenerEvent eventType, long transferredBytes) {
switch (eventType) {
case TRANSFER_PART_FAILED_EVENT:
// failure
failures.incrementAndGet();
LOG.warn("Transfer failure");
break;
case TRANSFER_PART_COMPLETED_EVENT:
// completion
bytesTransferred.addAndGet(transferredBytes);
long elapsedTime = timer.elapsedTime();
double elapsedTimeS = elapsedTime / 1.0e9;
long written = bytesTransferred.get();
long writtenMB = written / _1MB;
LOG.info(String.format(
"Event %s; total uploaded=%d MB in %.1fs;" +
" effective upload bandwidth = %.2f MB/s",
eventType,
writtenMB, elapsedTimeS, writtenMB / elapsedTimeS));
break;
case REQUEST_BYTE_TRANSFER_EVENT:
uploadEvents.incrementAndGet();
break;
default:
// nothing
break;
}
}
public String toString() {
String sb = "ProgressCallback{"
+ "bytesTransferred=" + bytesTransferred.get() +
", uploadEvents=" + uploadEvents.get() +
", failures=" + failures.get() +
'}';
return sb;
}
/**
* Get the number of bytes transferred.
* @return byte count
*/
private long getBytesTransferred() {
return bytesTransferred.get();
}
/**
* Get the number of event callbacks.
* @return count of byte transferred events.
*/
private long getUploadEvents() {
return uploadEvents.get();
}
private void verifyNoFailures(String operation) {
assertEquals("Failures in " + operation + ": " + this, 0, failures.get());
}
}
/**
* Assume that the huge file exists; skip the test if it does not.
* @throws IOException IO failure

View File

@ -0,0 +1,192 @@
/*
* 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.scale;
import java.util.EnumMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.assertj.core.api.AbstractLongAssert;
import org.assertj.core.api.Assertions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.impl.ProgressListener;
import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.util.Progressable;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.PUT_FAILED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.PUT_STARTED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.REQUEST_BYTE_TRANSFER_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.TRANSFER_PART_FAILED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.TRANSFER_PART_STARTED_EVENT;
/**
* Progress listener for AWS upload tracking.
* Declared as {@link Progressable} to be passed down through the hadoop FS create()
* operations, it also implements {@link ProgressListener} to get direct
* information from the AWS SDK
*/
public class CountingProgressListener implements Progressable, ProgressListener {
private static final Logger LOG = LoggerFactory.getLogger(AbstractSTestS3AHugeFiles.class);
private final ContractTestUtils.NanoTimer timer;
private final Map<ProgressListenerEvent, AtomicLong> eventCounts;
private final AtomicLong bytesTransferred = new AtomicLong(0);
/**
* Create a progress listener.
* @param timer timer to use
*/
public CountingProgressListener(final ContractTestUtils.NanoTimer timer) {
this.timer = timer;
this.eventCounts = new EnumMap<>(ProgressListenerEvent.class);
for (ProgressListenerEvent e : ProgressListenerEvent.values()) {
this.eventCounts.put(e, new AtomicLong(0));
}
}
/**
* Create a progress listener with a nano timer.
*/
public CountingProgressListener() {
this(new ContractTestUtils.NanoTimer());
}
@Override
public void progress() {
}
@Override
public void progressChanged(ProgressListenerEvent eventType, long transferredBytes) {
eventCounts.get(eventType).incrementAndGet();
switch (eventType) {
// part Upload has started
case TRANSFER_PART_STARTED_EVENT:
case PUT_STARTED_EVENT:
LOG.info("Transfer started");
break;
// an upload part completed
case TRANSFER_PART_COMPLETED_EVENT:
case PUT_COMPLETED_EVENT:
// completion
bytesTransferred.addAndGet(transferredBytes);
long elapsedTime = timer.elapsedTime();
double elapsedTimeS = elapsedTime / 1.0e9;
long written = bytesTransferred.get();
long writtenMB = written / S3AScaleTestBase._1MB;
LOG.info(String.format(
"Event %s; total uploaded=%d MB in %.1fs;" + " effective upload bandwidth = %.2f MB/s",
eventType, writtenMB, elapsedTimeS, writtenMB / elapsedTimeS));
break;
// and a transfer failed
case PUT_FAILED_EVENT:
case TRANSFER_PART_FAILED_EVENT:
LOG.warn("Transfer failure");
break;
default:
// nothing
break;
}
}
public String toString() {
String sb =
"ProgressCallback{" + "bytesTransferred=" + bytesTransferred.get() + '}';
return sb;
}
/**
* Get the count of a specific event.
* @param key event
* @return count
*/
public long get(ProgressListenerEvent key) {
return eventCounts.get(key).get();
}
/**
* Get the number of bytes transferred.
* @return byte count
*/
public long getBytesTransferred() {
return bytesTransferred.get();
}
/**
* Get the number of event callbacks.
* @return count of byte transferred events.
*/
public long getUploadEvents() {
return get(REQUEST_BYTE_TRANSFER_EVENT);
}
/**
* Get count of started events.
* @return count of started events.
*/
public long getStartedEvents() {
return get(PUT_STARTED_EVENT) + get(TRANSFER_PART_STARTED_EVENT);
}
/**
* Get count of started events.
* @return count of started events.
*/
public long getFailures() {
return get(PUT_FAILED_EVENT) + get(TRANSFER_PART_FAILED_EVENT);
}
/**
* Verify that no failures took place.
* @param operation operation being verified
*/
public void verifyNoFailures(String operation) {
Assertions.assertThat(getFailures())
.describedAs("Failures in %s: %s", operation, this)
.isEqualTo(0);
}
/**
* Assert that the event count is as expected.
* @param event event to look up
* @return ongoing assertion
*/
public AbstractLongAssert<?> assertEventCount(ProgressListenerEvent event) {
return Assertions.assertThat(get(event)).describedAs("Event %s count", event);
}
/**
* Assert that the event count is as expected.
* @param event event to look up
* @param count expected value.
*/
public void assertEventCount(ProgressListenerEvent event, long count) {
assertEventCount(event).isEqualTo(count);
}
}

View File

@ -0,0 +1,493 @@
/*
* 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.scale;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Abortable;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.fs.s3a.test.SdkFaultInjector;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import org.apache.hadoop.util.functional.InvocationRaisingIOE;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_OPERATIONS_PURGE_UPLOADS;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_ACTIVE_BLOCKS;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_ARRAY;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_DISK;
import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BYTEBUFFER;
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES;
import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE;
import static org.apache.hadoop.fs.s3a.Constants.RETRY_HTTP_5XX_ERRORS;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyInt;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_ABORTED;
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.PUT_INTERRUPTED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.PUT_STARTED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.TRANSFER_MULTIPART_ABORTED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.TRANSFER_MULTIPART_INITIATED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.TRANSFER_PART_FAILED_EVENT;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.TRANSFER_PART_STARTED_EVENT;
import static org.apache.hadoop.fs.s3a.test.SdkFaultInjector.setRequestFailureConditions;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
* Testing interrupting file writes to s3 in
* {@link FSDataOutputStream#close()}.
* <p>
* This is a bit tricky as we want to verify for all the block types that we
* can interrupt active and pending uploads and not end up with failures
* in the close() method.
* Ideally cleanup should take place, especially of files.
* <p>
* Marked as a scale test even though it tries to aggressively abort streams being written
* and should, if working, complete fast.
*/
@RunWith(Parameterized.class)
public class ITestS3ABlockOutputStreamInterruption extends S3AScaleTestBase {
public static final int MAX_RETRIES_IN_SDK = 2;
/**
* Parameterized on (buffer type, active blocks).
* @return parameters
*/
@Parameterized.Parameters(name = "{0}-{1}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{FAST_UPLOAD_BUFFER_DISK, 2},
{FAST_UPLOAD_BUFFER_ARRAY, 1},
{FAST_UPLOAD_BYTEBUFFER, 2}
});
}
public static final int MPU_SIZE = 5 * _1MB;
/**
* Buffer type.
*/
private final String bufferType;
/**
* How many blocks can a stream have uploading?
*/
private final int activeBlocks;
/**
* Constructor.
* @param bufferType buffer type
* @param activeBlocks number of active blocks which can be uploaded
*/
public ITestS3ABlockOutputStreamInterruption(final String bufferType,
int activeBlocks) {
this.bufferType = requireNonNull(bufferType);
this.activeBlocks = activeBlocks;
}
/**
* Get the test timeout in seconds.
* @return the test timeout as set in system properties or the default.
*/
protected int getTestTimeoutSeconds() {
return getTestPropertyInt(new Configuration(),
KEY_TEST_TIMEOUT,
SCALE_TEST_TIMEOUT_SECONDS);
}
@Override
protected Configuration createScaleConfiguration() {
Configuration conf = super.createScaleConfiguration();
removeBaseAndBucketOverrides(conf,
AUDIT_EXECUTION_INTERCEPTORS,
DIRECTORY_OPERATIONS_PURGE_UPLOADS,
FAST_UPLOAD_BUFFER,
MAX_ERROR_RETRIES,
MULTIPART_SIZE,
RETRY_HTTP_5XX_ERRORS);
conf.set(FAST_UPLOAD_BUFFER, bufferType);
conf.setLong(MULTIPART_SIZE, MPU_SIZE);
// limiting block size allows for stricter ordering of block uploads:
// only 1 should be active at a time, so when a write is cancelled
// it should be the only one to be aborted.
conf.setLong(FAST_UPLOAD_ACTIVE_BLOCKS, activeBlocks);
// guarantees teardown will abort pending uploads.
conf.setBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, true);
// don't retry much
conf.setInt(MAX_ERROR_RETRIES, MAX_RETRIES_IN_SDK);
// use the fault injector
SdkFaultInjector.addFaultInjection(conf);
return conf;
}
/**
* Setup MUST set up the evaluator before the FS is created.
*/
@Override
public void setup() throws Exception {
SdkFaultInjector.resetEvaluator();
super.setup();
}
@Override
public void teardown() throws Exception {
// safety check in case the evaluation is failing any
// request needed in cleanup.
SdkFaultInjector.resetEvaluator();
super.teardown();
}
@Test
public void testInterruptMultipart() throws Throwable {
describe("Interrupt a thread performing close() on a multipart upload");
interruptMultipartUpload(methodPath(), 6 * _1MB);
}
/**
* Initiate the upload of a file of a given length, then interrupt the
* operation in close(); assert the expected outcome including verifying
* that it was a multipart upload which was interrupted.
* @param path path to write
* @param len file length
*/
private void interruptMultipartUpload(final Path path, int len) throws Exception {
// dataset is bigger than one block
final byte[] dataset = dataset(len, 'a', 'z' - 'a');
InterruptingProgressListener listener = new InterruptingProgressListener(
Thread.currentThread(),
TRANSFER_PART_STARTED_EVENT);
final FSDataOutputStream out = createFile(path, listener);
// write it twice to force a multipart upload
out.write(dataset);
out.write(dataset);
expectCloseInterrupted(out);
LOG.info("Write aborted; total bytes written = {}", listener.getBytesTransferred());
final IOStatistics streamStats = out.getIOStatistics();
LOG.info("stream statistics {}", ioStatisticsToPrettyString(streamStats));
listener.assertTriggered();
listener.assertEventCount(TRANSFER_MULTIPART_INITIATED_EVENT, 1);
listener.assertEventCount(TRANSFER_MULTIPART_ABORTED_EVENT, 1);
// examine the statistics
verifyStatisticCounterValue(streamStats,
StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED, 1);
// expect at least one byte to be transferred
assertBytesTransferred(listener, 1, len * 2);
}
/**
* Invoke Abortable.abort() during the upload,
* then go on to simulate an NPE in the part upload and verify
* that this does not get escalated.
*/
@Test
public void testAbortDuringUpload() throws Throwable {
describe("Abort during multipart upload");
int len = 6 * _1MB;
final byte[] dataset = dataset(len, 'a', 'z' - 'a');
// the listener aborts a target
AtomicReference<Abortable> target = new AtomicReference<>();
Semaphore semaphore = new Semaphore(1);
semaphore.acquire();
InterruptingProgressListener listener = new InterruptingProgressListener(
TRANSFER_PART_STARTED_EVENT,
() -> {
final NullPointerException ex =
new NullPointerException("simulated failure after abort");
LOG.info("aborting target", ex);
// abort the stream
target.get().abort();
// wake up any thread
semaphore.release();
throw ex;
});
final FSDataOutputStream out = createFile(methodPath(), listener);
// the target can only be set once we have the stream reference
target.set(out);
// queue the write which, once the block upload begins, will trigger the abort
out.write(dataset);
// block until the abort is triggered
semaphore.acquire();
// rely on the stream having closed at this point so that the
// failed multipart event doesn't cause any problem
out.close();
// abort the stream again, expect it to be already closed
final Abortable.AbortableResult result = target.get().abort();
Assertions.assertThat(result.alreadyClosed())
.describedAs("already closed flag in %s", result)
.isTrue();
listener.assertEventCount(TRANSFER_MULTIPART_ABORTED_EVENT, 1);
// the raised NPE should have been noted but does not escalate to any form of failure.
// note that race conditions in the code means that it is too brittle for a strict
// assert here
listener.assertEventCount(TRANSFER_PART_FAILED_EVENT)
.isBetween(0L, 1L);
}
/**
* Test that a part upload failure is propagated to
* the close() call.
*/
@Test
public void testPartUploadFailure() throws Throwable {
describe("Trigger a failure during a multipart upload");
int len = 6 * _1MB;
final byte[] dataset = dataset(len, 'a', 'z' - 'a');
final String text = "Simulated failure";
// uses a semaphore to control the timing of the NPE and close() call.
Semaphore semaphore = new Semaphore(1);
semaphore.acquire();
InterruptingProgressListener listener = new InterruptingProgressListener(
TRANSFER_PART_STARTED_EVENT,
() -> {
// wake up any thread
semaphore.release();
throw new NullPointerException(text);
});
final FSDataOutputStream out = createFile(methodPath(), listener);
out.write(dataset);
semaphore.acquire();
// quick extra sleep to ensure the NPE is raised
Thread.sleep(1000);
// this will pass up the exception from the part upload
intercept(IOException.class, text, out::close);
listener.assertEventCount(TRANSFER_MULTIPART_ABORTED_EVENT, 1);
listener.assertEventCount(TRANSFER_PART_FAILED_EVENT, 1);
}
/**
* Assert that bytes were transferred between (inclusively) the min and max values.
* @param listener listener
* @param min minimum
* @param max maximum
*/
private static void assertBytesTransferred(
final InterruptingProgressListener listener,
final long min,
final long max) {
Assertions.assertThat(listener.getBytesTransferred())
.describedAs("bytes transferred")
.isBetween(min, max);
}
/**
* Write a small dataset and interrupt the close() operation.
*/
@Test
public void testInterruptMagicWrite() throws Throwable {
describe("Interrupt a thread performing close() on a magic upload");
// write a smaller file to a magic path and assert multipart outcome
Path path = new Path(methodPath(), MAGIC_PATH_PREFIX + "1/__base/file");
interruptMultipartUpload(path, _1MB);
}
/**
* Write a small dataset and interrupt the close() operation.
*/
@Test
public void testInterruptWhenAbortingAnUpload() throws Throwable {
describe("Interrupt a thread performing close() on a magic upload");
// fail more than the SDK will retry
setRequestFailureConditions(MAX_RETRIES_IN_SDK * 2, SdkFaultInjector::isMultipartAbort);
// write a smaller file to a magic path and assert multipart outcome
Path path = new Path(methodPath(), MAGIC_PATH_PREFIX + "1/__base/file");
interruptMultipartUpload(path, _1MB);
// an abort is double counted; the outer one also includes time to cancel
// all pending aborts so is important to measure.
verifyStatisticCounterValue(getFileSystem().getIOStatistics(),
OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol() + SUFFIX_FAILURES,
2);
}
/**
* Interrupt a thread performing close() on a simple PUT.
* This is less complex than the multipart upload case
* because the progress callback should be on the current thread.
* <p>
* We do expect exception translation to map the interruption to
* a {@code InterruptedIOException} and the count of interrupted events
* to increase.
*/
@Test
public void testInterruptSimplePut() throws Throwable {
describe("Interrupt simple object PUT");
// dataset is less than one block
final int len = _1MB;
final byte[] dataset = dataset(len, 'a', 'z' - 'a');
Path path = methodPath();
InterruptingProgressListener listener = new InterruptingProgressListener(
Thread.currentThread(),
PUT_STARTED_EVENT);
final FSDataOutputStream out = createFile(path, listener);
out.write(dataset);
expectCloseInterrupted(out);
LOG.info("Write aborted; total bytes written = {}", listener.getBytesTransferred());
final IOStatistics streamStats = out.getIOStatistics();
LOG.info("stream statistics {}", ioStatisticsToPrettyString(streamStats));
listener.assertTriggered();
listener.assertEventCount(PUT_INTERRUPTED_EVENT, 1);
assertBytesTransferred(listener, 0, len);
}
/**
* Expect that a close operation is interrupted the first time it
* is invoked.
* the second time it is invoked, it should succeed.
* @param out output stream
*/
private static void expectCloseInterrupted(final FSDataOutputStream out)
throws Exception {
// first call will be interrupted
intercept(InterruptedIOException.class, out::close);
// second call must be safe
out.close();
}
/**
* Create a file with a progress listener.
* @param path path to file
* @param listener listener
* @return the output stream
* @throws IOException IO failure
*/
private FSDataOutputStream createFile(final Path path,
final InterruptingProgressListener listener) throws IOException {
final FSDataOutputStreamBuilder builder = getFileSystem().createFile(path);
builder
.overwrite(true)
.progress(listener)
.must(FS_S3A_CREATE_PERFORMANCE, true);
return builder.build();
}
/**
* Progress listener which interrupts the thread at any chosen callback.
* or any other action
*/
private static final class InterruptingProgressListener
extends CountingProgressListener {
/** Event to trigger action. */
private final ProgressListenerEvent trigger;
/** Flag set when triggered. */
private final AtomicBoolean triggered = new AtomicBoolean(false);
/**
* Action to take on trigger.
*/
private final InvocationRaisingIOE action;
/**
* Create.
* @param thread thread to interrupt
* @param trigger event to trigger on
*/
private InterruptingProgressListener(
final Thread thread,
final ProgressListenerEvent trigger) {
this(trigger, thread::interrupt);
}
/**
* Create for any arbitrary action.
* @param trigger event to trigger on
* @param action action to take
*/
private InterruptingProgressListener(
final ProgressListenerEvent trigger,
final InvocationRaisingIOE action) {
this.trigger = trigger;
this.action = action;
}
@Override
public void progressChanged(final ProgressListenerEvent eventType,
final long transferredBytes) {
super.progressChanged(eventType, transferredBytes);
if (trigger == eventType && !triggered.getAndSet(true)) {
LOG.info("triggering action");
try {
action.apply();
} catch (IOException e) {
LOG.warn("action failed", e);
}
}
}
/**
* Assert that the trigger took place.
*/
private void assertTriggered() {
assertTrue("Not triggered", triggered.get());
}
}
}

View File

@ -21,7 +21,6 @@
import java.io.IOException;
import java.net.URI;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -29,21 +28,24 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
import org.apache.hadoop.fs.s3a.Constants;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.assertj.core.api.Assertions;
import org.junit.After;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
/**
* Tests concurrent operations on a single S3AFileSystem instance.
@ -61,9 +63,21 @@ protected int getTestTimeoutSeconds() {
return 16 * 60;
}
@Override
protected Configuration createScaleConfiguration() {
final Configuration conf = super.createScaleConfiguration();
removeBaseAndBucketOverrides(conf, MULTIPART_SIZE);
conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE);
return conf;
}
@Override
public void setup() throws Exception {
super.setup();
final S3AFileSystem fs = getFileSystem();
final Configuration conf = fs.getConf();
assume("multipart upload/copy disabled",
conf.getBoolean(MULTIPART_UPLOADS_ENABLED, true));
auxFs = getNormalFileSystem();
// this is set to the method path, even in test setup.
@ -124,20 +138,16 @@ public Thread newThread(Runnable r) {
try {
((ThreadPoolExecutor)executor).prestartAllCoreThreads();
Future<Boolean>[] futures = new Future[concurrentRenames];
for (int i = 0; i < concurrentRenames; i++) {
final int index = i;
futures[i] = executor.submit(new Callable<Boolean>() {
@Override
public Boolean call() throws Exception {
NanoTimer timer = new NanoTimer();
boolean result = fs.rename(source[index], target[index]);
timer.end("parallel rename %d", index);
LOG.info("Rename {} ran from {} to {}", index,
timer.getStartTime(), timer.getEndTime());
return result;
}
IntStream.range(0, concurrentRenames).forEachOrdered(i -> {
futures[i] = executor.submit(() -> {
NanoTimer timer = new NanoTimer();
boolean result = fs.rename(source[i], target[i]);
timer.end("parallel rename %d", i);
LOG.info("Rename {} ran from {} to {}", i,
timer.getStartTime(), timer.getEndTime());
return result;
});
}
});
LOG.info("Waiting for tasks to complete...");
LOG.info("Deadlock may have occurred if nothing else is logged" +
" or the test times out");
@ -159,17 +169,16 @@ public Boolean call() throws Exception {
* that now can't enter the resource pool to get completed.
*/
@Test
@SuppressWarnings("unchecked")
public void testParallelRename() throws InterruptedException,
ExecutionException, IOException {
Configuration conf = getConfiguration();
// clone the fs with all its per-bucket settings
Configuration conf = new Configuration(getFileSystem().getConf());
// shrink the thread pool
conf.setInt(MAX_THREADS, 2);
conf.setInt(MAX_TOTAL_TASKS, 1);
conf.set(MIN_MULTIPART_THRESHOLD, "10K");
conf.set(MULTIPART_SIZE, "5K");
try (S3AFileSystem tinyThreadPoolFs = new S3AFileSystem()) {
tinyThreadPoolFs.initialize(auxFs.getUri(), conf);
@ -178,35 +187,42 @@ public void testParallelRename() throws InterruptedException,
}
}
/**
* Verify that after a parallel rename batch there are multiple
* transfer threads active -and that after the timeout duration
* that thread count has dropped to zero.
*/
@Test
public void testThreadPoolCoolDown() throws InterruptedException,
ExecutionException, IOException {
int hotThreads = 0;
int coldThreads = 0;
parallelRenames(concurrentRenames, auxFs,
"testThreadPoolCoolDown-source", "testThreadPoolCoolDown-target");
for (Thread t : Thread.getAllStackTraces().keySet()) {
if (t.getName().startsWith("s3a-transfer")) {
hotThreads++;
}
}
int hotThreads = (int) Thread.getAllStackTraces()
.keySet()
.stream()
.filter(t -> t.getName().startsWith("s3a-transfer"))
.count();
int timeoutMs = Constants.DEFAULT_KEEPALIVE_TIME * 1000;
Assertions.assertThat(hotThreads)
.describedAs("Failed to find threads in active FS - test is flawed")
.isNotEqualTo(0);
long timeoutMs = DEFAULT_KEEPALIVE_TIME_DURATION.toMillis();
Thread.sleep((int)(1.1 * timeoutMs));
for (Thread t : Thread.getAllStackTraces().keySet()) {
if (t.getName().startsWith("s3a-transfer")) {
coldThreads++;
}
}
assertNotEquals("Failed to find threads in active FS - test is flawed",
hotThreads, 0);
assertTrue("s3a-transfer threads went from " + hotThreads + " to " +
coldThreads + ", should have gone to 0", 0 == coldThreads);
int coldThreads = (int) Thread.getAllStackTraces()
.keySet()
.stream()
.filter(t -> t.getName().startsWith("s3a-transfer"))
.count();
Assertions.assertThat(coldThreads)
.describedAs(("s3a-transfer threads went from %s to %s;"
+ " should have gone to 0"),
hotThreads, coldThreads)
.isEqualTo(0);
}
}

View File

@ -44,7 +44,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
@ -262,7 +261,7 @@ public void testMultiPagesListingPerformanceAndCorrectness()
futures.add(submit(executorService,
() -> writeOperationHelper.putObject(putObjectRequestBuilder.build(),
PutObjectOptions.keepingDirs(),
new S3ADataBlocks.BlockUploadData(new FailingInputStream()), false, null)));
new S3ADataBlocks.BlockUploadData(new byte[0], null), null)));
}
LOG.info("Waiting for PUTs to complete");
waitForCompletion(futures);
@ -363,16 +362,6 @@ public void testMultiPagesListingPerformanceAndCorrectness()
}
}
/**
* Input stream which always returns -1.
*/
private static final class FailingInputStream extends InputStream {
@Override
public int read() throws IOException {
return -1;
}
}
/**
* Sleep briefly.
* @param eachFileProcessingTime time to sleep.

View File

@ -0,0 +1,83 @@
/*
* 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;
import java.util.Arrays;
import java.util.Collection;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.apache.hadoop.fs.s3a.statistics.impl.StatisticsFromAwsSdkImpl;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_429_TOO_MANY_REQUESTS_GCS;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_500_INTERNAL_SERVER_ERROR;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_503_SERVICE_UNAVAILABLE;
import static org.apache.hadoop.fs.s3a.statistics.impl.StatisticsFromAwsSdkImpl.mapErrorStatusCodeToStatisticName;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_400;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_4XX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_500;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_503;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_5XX;
/**
* Test mapping logic of {@link StatisticsFromAwsSdkImpl}.
*/
@RunWith(Parameterized.class)
public class TestErrorCodeMapping extends AbstractHadoopTestBase {
/**
* Parameterization.
*/
@Parameterized.Parameters(name = "http {0} to {1}")
public static Collection<Object[]> params() {
return Arrays.asList(new Object[][]{
{200, null},
{302, null},
{SC_400_BAD_REQUEST, HTTP_RESPONSE_400},
{SC_404_NOT_FOUND, null},
{416, HTTP_RESPONSE_4XX},
{SC_429_TOO_MANY_REQUESTS_GCS, HTTP_RESPONSE_503},
{SC_500_INTERNAL_SERVER_ERROR, HTTP_RESPONSE_500},
{SC_503_SERVICE_UNAVAILABLE, HTTP_RESPONSE_503},
{510, HTTP_RESPONSE_5XX},
});
}
private final int code;
private final String name;
public TestErrorCodeMapping(final int code, final String name) {
this.code = code;
this.name = name;
}
@Test
public void testMapping() throws Throwable {
Assertions.assertThat(mapErrorStatusCodeToStatisticName(code))
.describedAs("Mapping of status code %d", code)
.isEqualTo(name);
}
}

View File

@ -18,22 +18,63 @@
package org.apache.hadoop.fs.s3a.test;
import java.io.UncheckedIOException;
import java.util.function.Supplier;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
/**
* Stub implementation of writeOperationHelper callbacks.
* Minimal implementation of writeOperationHelper callbacks.
* Callbacks which need to talk to S3 use the s3 client resolved
* on demand from {@link #s3clientSupplier}.
* if this returns null, the operations raise NPEs.
*/
public class MinimalWriteOperationHelperCallbacks
implements WriteOperationHelper.WriteOperationHelperCallbacks {
/**
* Supplier of the s3 client.
*/
private final Supplier<S3Client> s3clientSupplier;
/**
* Constructor.
* @param s3clientSupplier supplier of the S3 client.
*/
public MinimalWriteOperationHelperCallbacks(
final Supplier<S3Client> s3clientSupplier) {
this.s3clientSupplier = s3clientSupplier;
}
@Override
public CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request) {
return null;
return s3clientSupplier.get().completeMultipartUpload(request);
}
@Override
public UploadPartResponse uploadPart(final UploadPartRequest request,
final RequestBody body,
final DurationTrackerFactory durationTrackerFactory)
throws AwsServiceException, UncheckedIOException {
return s3clientSupplier.get().uploadPart(request, body);
}
@Override
public void finishedWrite(final String key,
final long length,
final PutObjectOptions putOptions) {
}
}

View File

@ -0,0 +1,218 @@
/*
* 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.test;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import software.amazon.awssdk.core.SdkRequest;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
import software.amazon.awssdk.http.SdkHttpMethod;
import software.amazon.awssdk.http.SdkHttpResponse;
import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import org.apache.hadoop.conf.Configuration;
import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.enableLoggingAuditor;
import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions;
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_500_INTERNAL_SERVER_ERROR;
/**
* This runs inside the AWS execution pipeline so can insert faults and so
* trigger recovery in the SDK.
* It is wired up through the auditor mechanism.
* <p>
* This uses the evaluator function {@link #evaluator} to determine if
* the request type is that for which failures are targeted;
* When there is a match then the failure count
* is decremented and, if the count is still positive, an error is raised with the
* error code defined in {@link #FAILURE_STATUS_CODE}.
* This happens <i>after</i> the request has already succeeded against the S3 store:
* whatever was requested has actually already happened.
*/
public final class SdkFaultInjector implements ExecutionInterceptor {
private static final Logger LOG =
LoggerFactory.getLogger(SdkFaultInjector.class);
private static final AtomicInteger FAILURE_STATUS_CODE =
new AtomicInteger(SC_500_INTERNAL_SERVER_ERROR);
/**
* Always allow requests.
*/
public static final Function<Context.ModifyHttpResponse, Boolean>
ALWAYS_ALLOW = (c) -> false;
/**
* How many requests with the matching evaluator to fail on.
*/
public static final AtomicInteger REQUEST_FAILURE_COUNT = new AtomicInteger(1);
/**
* Evaluator for responses.
*/
private static Function<Context.ModifyHttpResponse, Boolean> evaluator = ALWAYS_ALLOW;
/**
* Update the value of {@link #FAILURE_STATUS_CODE}.
* @param value new value
*/
public static void setFailureStatusCode(int value) {
FAILURE_STATUS_CODE.set(value);
}
/**
* Set the evaluator function used to determine whether or not to raise
* an exception.
* @param value new evaluator.
*/
public static void setEvaluator(Function<Context.ModifyHttpResponse, Boolean> value) {
evaluator = value;
}
/**
* Reset the evaluator to enable everything.
*/
public static void resetEvaluator() {
setEvaluator(ALWAYS_ALLOW);
}
/**
* Set the failure count.
* @param count failure count
*/
public static void setRequestFailureCount(int count) {
LOG.debug("Failure count set to {}", count);
REQUEST_FAILURE_COUNT.set(count);
}
/**
* Set up the request failure conditions.
* @param attempts how many times to fail before succeeding
* @param condition condition to trigger the failure
*/
public static void setRequestFailureConditions(final int attempts,
final Function<Context.ModifyHttpResponse, Boolean> condition) {
setRequestFailureCount(attempts);
setEvaluator(condition);
}
/**
* Is the response being processed from a PUT request?
* @param context request context.
* @return true if the request is of the right type.
*/
public static boolean isPutRequest(final Context.ModifyHttpResponse context) {
return context.httpRequest().method().equals(SdkHttpMethod.PUT);
}
/**
* Is the response being processed from any POST request?
* @param context request context.
* @return true if the request is of the right type.
*/
public static boolean isPostRequest(final Context.ModifyHttpResponse context) {
return context.httpRequest().method().equals(SdkHttpMethod.POST);
}
/**
* Is the request a commit completion request?
* @param context response
* @return true if the predicate matches
*/
public static boolean isCompleteMultipartUploadRequest(
final Context.ModifyHttpResponse context) {
return context.request() instanceof CompleteMultipartUploadRequest;
}
/**
* Is the request a part upload?
* @param context response
* @return true if the predicate matches
*/
public static boolean isPartUpload(final Context.ModifyHttpResponse context) {
return context.request() instanceof UploadPartRequest;
}
/**
* Is the request a multipart upload abort?
* @param context response
* @return true if the predicate matches
*/
public static boolean isMultipartAbort(final Context.ModifyHttpResponse context) {
return context.request() instanceof AbortMultipartUploadRequest;
}
/**
* Review response from S3 and optionall modify its status code.
* @return the original response or a copy with a different status code.
*/
@Override
public SdkHttpResponse modifyHttpResponse(final Context.ModifyHttpResponse context,
final ExecutionAttributes executionAttributes) {
SdkRequest request = context.request();
SdkHttpResponse httpResponse = context.httpResponse();
if (evaluator.apply(context) && shouldFail()) {
// fail the request
final int code = FAILURE_STATUS_CODE.get();
LOG.info("Fault Injector returning {} error code for request {}",
code, request);
return httpResponse.copy(b -> {
b.statusCode(code);
});
} else {
// pass unchanged
return httpResponse;
}
}
/**
* Should the request fail based on the failure count?
* @return true if the request count means a request must fail
*/
private static boolean shouldFail() {
return REQUEST_FAILURE_COUNT.decrementAndGet() > 0;
}
/**
* Add fault injection.
* This wires up auditing as needed.
* @param conf configuration to patch
* @return patched configuration
*/
public static Configuration addFaultInjection(Configuration conf) {
resetAuditOptions(conf);
enableLoggingAuditor(conf);
// use the fault injector
conf.set(AUDIT_EXECUTION_INTERCEPTORS, SdkFaultInjector.class.getName());
return conf;
}
}