diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index 44f794aa77478..e3deda775286a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -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}. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java new file mode 100644 index 0000000000000..08d15a5e2eb9a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java @@ -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(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java index 0ae9ee6378b57..e8b6684f12015 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java @@ -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() { } } } + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java index 0a0d023d931d0..fca521a5b8689 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java @@ -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. - *

+ *

+ * 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. *

+ * 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 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 awaitFuture(final Future 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 awaitFuture(final Future future) * extracted and rethrown. *

* @param future future to evaluate - * @param timeout timeout to wait + * @param timeout timeout to wait. * @param unit time unit. * @param 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 awaitFuture(final Future future) public static T awaitFuture(final Future 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 awaitFuture(final Future future, * @param collection collection of futures to be evaluated * @param 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 List awaitAllFutures(final Collection> collection) - throws InterruptedIOException, IOException, RuntimeException { + throws InterruptedIOException, IOException, CancellationException, RuntimeException { List results = new ArrayList<>(); for (Future future : collection) { results.add(awaitFuture(future)); @@ -148,19 +175,19 @@ public static List awaitAllFutures(final Collection> 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. - *

* @param collection collection of futures to be evaluated * @param duration timeout duration * @param 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 List awaitAllFutures(final Collection> collection, final Duration duration) - throws InterruptedIOException, IOException, RuntimeException, + throws InterruptedIOException, IOException, CancellationException, RuntimeException, TimeoutException { List results = new ArrayList<>(); for (Future future : collection) { @@ -169,6 +196,48 @@ public static List awaitAllFutures(final Collection> collection return results; } + /** + * Cancels a collection of futures and awaits the specified duration for their completion. + *

+ * 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 type of the result. + * @return all futures which completed successfully. + */ + public static List cancelAllFuturesAndAwaitCompletion( + final Collection> collection, + final boolean interruptIfRunning, + final Duration duration) { + + for (Future future : collection) { + future.cancel(interruptIfRunning); + } + // timeout is relative to the start of the operation + long timeout = duration.toMillis(); + List results = new ArrayList<>(); + for (Future 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. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java index f7c72f8530959..fa942efc20f9a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java @@ -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 + *

+ * 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. + *

+ * 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}. + *

+ * 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; - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 7e614bc11d6ee..cf7bc3ddcf248 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1119,6 +1119,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}? + *

+ * 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? */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java index 7ee6c55c191b7..25b5d774cdf7f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java @@ -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; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 5fe39ac6ea336..741a78a0537f2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -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. + *

+ * Key Features + *

    + *
  1. Support single/multipart uploads
  2. + *
  3. Multiple buffering options
  4. + *
  5. Magic files are uploaded but not completed
  6. + *
  7. Implements {@link Abortable} API
  8. + *
  9. Doesn't implement {@link Syncable}; whether to ignore or reject calls is configurable
  10. a + *
  11. When multipart uploads are triggered, will queue blocks for asynchronous uploads
  12. + *
  13. Provides progress information to any supplied {@link Progressable} callback, + * during async uploads and in the {@link #close()} operation.
  14. + *
  15. 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. + *
  16. * - * 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. - * + *
+ * This class is best described as "complicated". + *
    + *
  1. For "normal" files, data is buffered until either of: + * the limit of {@link #blockSize} is reached or the stream is closed. + *
  2. + *
  3. If if there are any problems call mukund
  4. + *
+ *

+ * The upload will not be completed until {@link #close()}, and + * then only if {@link PutTracker#outputImmediatelyVisible()} is true. + *

+ * 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. + *

+ * 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. + *

* Syncable is declared as supported so the calls can be - * explicitly rejected. + * explicitly rejected if the filesystem is configured to do so. + *

*/ @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. + *

+ * 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. - * + *

* 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 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. + *

+ * 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 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> partETagsFutures; + + /** + * List of completed uploads, in order of blocks written. + */ + private final List> partETagsFutures = + Collections.synchronizedList(new ArrayList<>()); + + /** blocks which need to be closed when aborting a stream. */ + private final Map 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 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 partETagFuture) { partETagsFutures.add(partETagFuture); + blocksToClose.put(currentPartNumber, block); + } + + /** + * Complete an upload. + *

+ * 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 waitForAllPartUploads() throws IOException { + private List 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 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 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 partETags) throws IOException { maybeRethrowUploadFailure(); @@ -994,23 +1197,35 @@ private void complete(List 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. + *

* 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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 1c6facfd54f8c..dff7493e08b36 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -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. + *

+ * The data is accessed via the content provider; other constructors + * create the appropriate content provider for the data. + *

+ * 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; /** - * File constructor; input stream will be null. - * @param file file to upload + * 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 BlockUploadData(File file) { - Preconditions.checkArgument(file.exists(), "No file: " + file); - this.file = file; - this.uploadStream = null; + public UploadContentProviders.BaseContentProvider getContentProvider() { + return contentProvider; } /** - * Stream constructor, file field will be null. - * @param uploadStream stream to upload + * 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(InputStream uploadStream) { - Preconditions.checkNotNull(uploadStream, "rawUploadStream"); - this.uploadStream = uploadStream; - this.file = null; + public BlockUploadData(File file, final Supplier 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); } /** - * Predicate: does this instance contain a file reference. - * @return true if there is a file. + * 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. */ - boolean hasFile() { - return file != null; + public BlockUploadData(byte[] bytes, int offset, int size, + final Supplier isOpen) { + this.contentProvider = byteArrayContentProvider(bytes, offset, size, isOpen); } /** - * Get the file, if there is one. - * @return the file for uploading, or null. + * 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. */ - File getFile() { - return file; + public BlockUploadData(byte[] bytes, final Supplier isOpen) { + this.contentProvider = byteArrayContentProvider(bytes, isOpen); } /** - * Get the raw upload stream, if the object was - * created with one. - * @return the upload stream or null. + * Size as declared by the content provider. + * @return size of the data */ - 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 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 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(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index ea2800b3ab8f5..fb936f1848d1d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -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; @@ -729,7 +726,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. @@ -829,12 +826,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)) @@ -1960,9 +1958,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. + *

+ * This operation MUST be called after any PUT/multipart PUT completes + * successfully. + *

+ * 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); + } } @@ -2921,7 +2958,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() { @@ -2950,9 +2987,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() { */ protected DurationTrackerFactory nonNullDurationTrackerFactory( DurationTrackerFactory factory) { - return factory != null - ? factory - : getDurationTrackerFactory(); + return store.nonNullDurationTrackerFactory(factory); } /** @@ -3267,18 +3302,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); } /** @@ -3291,9 +3315,8 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file, * Important: this call will close any input stream in the request. * @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 */ @@ -3301,27 +3324,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); @@ -3379,13 +3402,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); } /** @@ -3395,14 +3413,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); } /** @@ -3412,12 +3424,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); } /** @@ -4248,6 +4256,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 @@ -4264,11 +4273,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(), @@ -4289,8 +4300,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, @@ -4300,49 +4312,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 @@ -4723,9 +4707,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 @@ -4733,11 +4715,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); @@ -4831,18 +4811,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(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 9d34457ab9443..e3bef9f470727 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -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( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java index faf105c8e2c86..aa3d604cc4f83 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java @@ -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, RetryPolicy> policyMap = createExceptionMap(); retryPolicy = retryByException(retryIdempotentCalls, policyMap); @@ -228,15 +240,13 @@ protected Map, 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); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java index a11ed19670587..aed4442716963 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java @@ -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 deleteObjects(DeleteObjectsRequest de Map.Entry> deleteObject( DeleteObjectRequest request) throws SdkException; + /** + * Upload part of a multi-partition file. + * Increments the write and put counters. + * Important: this call does not close any input stream in the body. + *

+ * 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. + *

+ * 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. + *

+ * 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); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 2335f09c51037..1b858f5c1e8a7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -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; @@ -297,7 +298,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 @@ -1176,6 +1177,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. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 3bee1008ce42b..0bcdb29330d56 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -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( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 3bbe000bf5b6e..b7387fc12e140 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -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. + * Important: this call does not close any input stream in the body. + *

+ * 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. + *

+ * 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); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 5ad9c9f9b6482..68709c40f45ca 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -74,13 +74,11 @@ 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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java index 97ee92a20b1e3..c5ce1a2c9e4b8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java @@ -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()); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java index 3a2d9d7f823ee..5f93454795a94 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -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 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); } } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java index d1943fa47773f..f33d94ce84fef 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java @@ -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. - * + *

* When invoking FS operations, it assumes that the underlying FS is * handling retries and exception translation: it does not attempt to * duplicate that work. - * + *

+ * 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()}. + *

+ * 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 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 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 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 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. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java index 0ab3cee5201e7..1f6c9123bae62 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java @@ -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)); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java index 84770861cc489..7fadac8623d50 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java @@ -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. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java index ff6748e66d1d2..4b2fc1c599b26 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java @@ -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(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 1d12a41008b6b..ddbcad6dc047f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -292,4 +292,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"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java index f3f9fb61e434d..391e11d956212 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java @@ -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, + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java index ca629f16be992..a7565fe046e3e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java @@ -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 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); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index d0871e7af2388..385023598c559 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -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 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 auditSpanSource) { + AuditSpanSource 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 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> 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. + * Important: this call does not close any input stream in the body. + *

+ * 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. + *

+ * 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. + *

+ * 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); + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java new file mode 100644 index 0000000000000..5676e67cde2fa --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java @@ -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}. + *

+ * These are required to ensure that retry of multipart uploads are reliable, + * while also avoiding memory copy/consumption overhead. + *

+ * For these reasons the providers built in to the AWS SDK are not used. + *

+ * 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 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 fileContentProvider( + File file, + long offset, + final int size, + final Supplier 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 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 byteBufferContentProvider( + final ByteBuffer byteBuffer, + final int size, + final @Nullable Supplier 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 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 byteArrayContentProvider( + final byte[] bytes, + final int offset, + final int size, + final @Nullable Supplier 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 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 byteArrayContentProvider( + final byte[] bytes, + final @Nullable Supplier isOpen) { + return byteArrayContentProvider(bytes, 0, bytes.length, isOpen); + } + + /** + * Base class for content providers; tracks the number of times a stream + * has been opened. + * @param type of stream created. + */ + @VisibleForTesting + public static abstract class BaseContentProvider + 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 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 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. + *

+ * 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. + *

+ * 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 { + + /** + * 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 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 { + + /** + * 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 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. + *

+ * 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 { + + /** + * 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 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(); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java index 554b628d003a4..6bf2354a83ede 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java @@ -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 */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java index 48b0b2bf454d3..15c2ab9b2548f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java @@ -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; + } } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md index de3ea79f4fb1d..1018ec9e7d6c2 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md @@ -213,7 +213,26 @@ as they keep trying to reconnect to ports which are never going to be available. fs.s3a.bucket.nonexistent-bucket-example.connection.establish.timeout 500 + + + fs.s3a.bucket.nonexistent-bucket-example.retry.http.5xx.errors + false + ``` + +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. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index e53e4a002265a..4856b0f576026 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -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. -### "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. +``` + + +## 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. + +### "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`. ``` -### `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`. +### 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. + + fs.s3a.optimized.copy.from.local.enabled + false + ``` -## "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...) + +## 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: ${sts.region} ``` +## HTTP 500 status code "We encountered an internal error" + +``` +We encountered an internal error. Please try again. +(Service: S3, Status Code: 500, Request ID: , Extended Request 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 + + fs.s3a.retry.http.5xx.errors + true + +``` + +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. + +## 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. ## Connectivity Problems diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java index 53fa0d83b55a7..b0e15adacd886 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java @@ -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 diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java index 02f3de094fb4b..fcee7839a5ab2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputByteBuffer.java @@ -27,7 +27,7 @@ protected String getBlockOutputBufferName() { } protected S3ADataBlocks.BlockFactory createFactory(S3AFileSystem fileSystem) { - return new S3ADataBlocks.ByteBufferBlockFactory(fileSystem); + return new S3ADataBlocks.ByteBufferBlockFactory(fileSystem.createStoreContext()); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java index abe8656be5fdd..ad8c76d2a61a4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java @@ -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; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index a375044add43a..5e127050fe65b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -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); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index fb797038601ab..3c53fd6081663 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -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) { } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java index 700ef5ced3d8a..4a53028860baf 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java @@ -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 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(); + + /** + * Buffer type. + */ + private final String bufferType; - @Before - public void nameThread() { - Thread.currentThread().setName("JUnit"); + 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()); + final UploadContentProviders.BaseContentProvider cp = + blockUploadData.getContentProvider(); + + assertStreamCreationCount(cp, 0); + InputStream stream = cp.newStream(); + + assertStreamCreationCount(cp, 1); + Assertions.assertThat(stream.markSupported()) + .describedAs("markSupported() of %s", stream) + .isTrue(); + + Optional 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; - assertEquals("wrong available() in " + stream, - expected, stream.available()); + assertAvailableValue(stream, expected); - assertEquals('t', stream.read()); - stream.mark(limit); + 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'); + + // no more data left + assertAvailableValue(stream, 0); - assertEquals("wrong available() in " + stream, - 0, stream.available()); - assertTrue("hasRemaining() in " + stream, !stream.hasRemaining()); + 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. + *

+ * 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); + } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java index f6e8f0c376d9b..1a58e870de609 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java @@ -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}. - * + *

* 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. + *

+ * 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 verifyTranslated(Class 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 diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index 4f329afe7ad51..5caa7c8785534 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -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(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java index 1520e588e544e..e453fd3caad57 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java @@ -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(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java index 0059e5b6c5392..632a243a4e1dc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java @@ -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; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestUploadRecovery.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestUploadRecovery.java new file mode 100644 index 0000000000000..1abece4bfeee2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestUploadRecovery.java @@ -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. + *

+ * 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. + *

+ * 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 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 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); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java index fae7a6232d1c9..28bd8b878e5b3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java @@ -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(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 19feb386333a8..f1f4f1d147d83 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -301,7 +301,7 @@ protected int directoriesInPath(Path path) { /** * Reset all the metrics being tracked. */ - private void resetStatistics() { + protected void resetStatistics() { costValidator.resetMetricDiffs(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index c2c941798e77a..70cab0d75544e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -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 diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/CountingProgressListener.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/CountingProgressListener.java new file mode 100644 index 0000000000000..b00138d1fbcd3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/CountingProgressListener.java @@ -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 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); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ABlockOutputStreamInterruption.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ABlockOutputStreamInterruption.java new file mode 100644 index 0000000000000..d5df8c42d5a21 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ABlockOutputStreamInterruption.java @@ -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()}. + *

+ * 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. + *

+ * 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 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 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. + *

+ * 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()); + } + } + + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java index c7e65d70fd340..5b63b20dc67d4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java @@ -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[] futures = new Future[concurrentRenames]; - for (int i = 0; i < concurrentRenames; i++) { - final int index = i; - futures[i] = executor.submit(new Callable() { - @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 timeoutMs = Constants.DEFAULT_KEEPALIVE_TIME * 1000; - Thread.sleep((int)(1.1 * timeoutMs)); + int hotThreads = (int) Thread.getAllStackTraces() + .keySet() + .stream() + .filter(t -> t.getName().startsWith("s3a-transfer")) + .count(); - for (Thread t : Thread.getAllStackTraces().keySet()) { - if (t.getName().startsWith("s3a-transfer")) { - coldThreads++; - } - } + Assertions.assertThat(hotThreads) + .describedAs("Failed to find threads in active FS - test is flawed") + .isNotEqualTo(0); - 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); + long timeoutMs = DEFAULT_KEEPALIVE_TIME_DURATION.toMillis(); + Thread.sleep((int)(1.1 * timeoutMs)); + 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); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index 173099bb2ca71..b09bd91357e7d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -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. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/TestErrorCodeMapping.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/TestErrorCodeMapping.java new file mode 100644 index 0000000000000..ed3a11fa579d0 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/TestErrorCodeMapping.java @@ -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 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); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java index 6838129bb30d1..5c50ebff5c6b0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java @@ -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 s3clientSupplier; + + /** + * Constructor. + * @param s3clientSupplier supplier of the S3 client. + */ + public MinimalWriteOperationHelperCallbacks( + final Supplier 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) { + + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/SdkFaultInjector.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/SdkFaultInjector.java new file mode 100644 index 0000000000000..3af31b3f89efd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/SdkFaultInjector.java @@ -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. + *

+ * 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 after 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 + 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 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 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 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; + } +}