From ea6e0f7cd58d0129897dfc7870aee188be80a904 Mon Sep 17 00:00:00 2001
From: Steve Loughran
Date: Fri, 13 Sep 2024 20:02:14 +0100
Subject: [PATCH] HADOOP-19221. S3A: Unable to recover from failure of
multipart block upload attempt (#6938)
This is a major change which handles 400 error responses when uploading
large files from memory heap/buffer (or staging committer) and the remote S3
store returns a 500 response from a upload of a block in a multipart upload.
The SDK's own streaming code seems unable to fully replay the upload;
at attempts to but then blocks and the S3 store returns a 400 response
"Your socket connection to the server was not read from or written to
within the timeout period. Idle connections will be closed.
(Service: S3, Status Code: 400...)"
There is an option to control whether or not the S3A client itself
attempts to retry on a 50x error other than 503 throttling events
(which are independently processed as before)
Option: fs.s3a.retry.http.5xx.errors
Default: true
500 errors are very rare from standard AWS S3, which has a five nines
SLA. It may be more common against S3 Express which has lower
guarantees.
Third party stores have unknown guarantees, and the exception may
indicate a bad server configuration. Consider setting
fs.s3a.retry.http.5xx.errors to false when working with
such stores.
Signification Code changes:
There is now a custom set of implementations of
software.amazon.awssdk.http.ContentStreamProvidercontent in
the class org.apache.hadoop.fs.s3a.impl.UploadContentProviders.
These:
* Restart on failures
* Do not copy buffers/byte buffers into new private byte arrays,
so avoid exacerbating memory problems..
There new IOStatistics for specific http error codes -these are collected
even when all recovery is performed within the SDK.
S3ABlockOutputStream has major changes, including handling of
Thread.interrupt() on the main thread, which now triggers and briefly
awaits cancellation of any ongoing uploads.
If the writing thread is interrupted in close(), it is mapped to
an InterruptedIOException. Applications like Hive and Spark must
catch these after cancelling a worker thread.
Contributed by Steve Loughran
---
.../fs/statistics/StoreStatisticNames.java | 41 ++
.../fs/store/ByteBufferInputStream.java | 199 +++++++
.../apache/hadoop/fs/store/DataBlocks.java | 155 +----
.../hadoop/util/functional/FutureIO.java | 91 ++-
.../hadoop/fs/s3a/AWSStatus500Exception.java | 21 +-
.../org/apache/hadoop/fs/s3a/Constants.java | 16 +
.../fs/s3a/ProgressableProgressListener.java | 16 +-
.../hadoop/fs/s3a/S3ABlockOutputStream.java | 486 +++++++++++-----
.../apache/hadoop/fs/s3a/S3ADataBlocks.java | 373 +++++-------
.../apache/hadoop/fs/s3a/S3AFileSystem.java | 194 +++----
.../hadoop/fs/s3a/S3AInstrumentation.java | 1 +
.../apache/hadoop/fs/s3a/S3ARetryPolicy.java | 18 +-
.../org/apache/hadoop/fs/s3a/S3AStore.java | 131 +++++
.../org/apache/hadoop/fs/s3a/S3AUtils.java | 16 +-
.../org/apache/hadoop/fs/s3a/Statistic.java | 32 +
.../hadoop/fs/s3a/WriteOperationHelper.java | 66 ++-
.../apache/hadoop/fs/s3a/WriteOperations.java | 11 +-
.../audit/impl/AbstractOperationAuditor.java | 4 +-
.../fs/s3a/audit/impl/LoggingAuditor.java | 20 +
.../fs/s3a/commit/impl/CommitOperations.java | 96 ++-
.../commit/magic/S3MagicCommitTracker.java | 14 +-
.../hadoop/fs/s3a/impl/ClientManager.java | 25 +
.../hadoop/fs/s3a/impl/ClientManagerImpl.java | 23 +
.../hadoop/fs/s3a/impl/InternalConstants.java | 7 +
.../fs/s3a/impl/ProgressListenerEvent.java | 64 +-
.../hadoop/fs/s3a/impl/S3AStoreBuilder.java | 16 +-
.../hadoop/fs/s3a/impl/S3AStoreImpl.java | 273 ++++++++-
.../fs/s3a/impl/UploadContentProviders.java | 549 ++++++++++++++++++
.../BlockOutputStreamStatistics.java | 3 +-
.../impl/StatisticsFromAwsSdkImpl.java | 43 ++
.../tools/hadoop-aws/third_party_stores.md | 19 +
.../tools/hadoop-aws/troubleshooting_s3a.md | 180 ++++--
.../fs/s3a/ITestS3ABlockOutputArray.java | 7 +-
.../fs/s3a/ITestS3ABlockOutputByteBuffer.java | 2 +-
.../fs/s3a/ITestS3ABlockOutputDisk.java | 2 +-
.../hadoop/fs/s3a/ITestS3AMiscOperations.java | 9 +-
.../hadoop/fs/s3a/MockS3AFileSystem.java | 10 +-
.../apache/hadoop/fs/s3a/TestDataBlocks.java | 233 ++++++--
.../org/apache/hadoop/fs/s3a/TestInvoker.java | 208 ++++++-
.../fs/s3a/TestS3ABlockOutputStream.java | 15 +-
.../hadoop/fs/s3a/audit/AuditTestSupport.java | 12 +-
.../fs/s3a/audit/TestLoggingAuditor.java | 41 ++
.../fs/s3a/commit/ITestUploadRecovery.java | 259 +++++++++
.../commit/staging/TestStagingCommitter.java | 9 +
.../s3a/performance/AbstractS3ACostTest.java | 2 +-
.../s3a/scale/AbstractSTestS3AHugeFiles.java | 85 +--
.../s3a/scale/CountingProgressListener.java | 192 ++++++
...ITestS3ABlockOutputStreamInterruption.java | 493 ++++++++++++++++
.../fs/s3a/scale/ITestS3AConcurrentOps.java | 94 +--
.../scale/ITestS3ADirectoryPerformance.java | 13 +-
.../s3a/statistics/TestErrorCodeMapping.java | 83 +++
.../MinimalWriteOperationHelperCallbacks.java | 45 +-
.../hadoop/fs/s3a/test/SdkFaultInjector.java | 218 +++++++
53 files changed, 4250 insertions(+), 985 deletions(-)
create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java
create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestUploadRecovery.java
create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/CountingProgressListener.java
create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ABlockOutputStreamInterruption.java
create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/TestErrorCodeMapping.java
create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/SdkFaultInjector.java
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 6bf4b736518e1..e58ac02e33731 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
@@ -1129,6 +1129,22 @@ private Constants() {
*/
public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms";
+
+ /**
+ * Should S3A connector retry on all 5xx errors which don't have
+ * explicit support: {@value}?
+ *
+ * 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
+ *
+ *
Support single/multipart uploads
+ *
Multiple buffering options
+ *
Magic files are uploaded but not completed
+ *
Implements {@link Abortable} API
+ *
Doesn't implement {@link Syncable}; whether to ignore or reject calls is configurable
a
+ *
When multipart uploads are triggered, will queue blocks for asynchronous uploads
+ *
Provides progress information to any supplied {@link Progressable} callback,
+ * during async uploads and in the {@link #close()} operation.
+ *
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.
+ *
*
- * 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".
+ *
+ *
For "normal" files, data is buffered until either of:
+ * the limit of {@link #blockSize} is reached or the stream is closed.
+ *
+ *
If if there are any problems call mukund
+ *
+ *
+ * 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 3c70004714459..366da8392b8e8 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;
@@ -734,7 +731,7 @@ public void initialize(URI name, Configuration originalConf)
}
blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
DEFAULT_FAST_UPLOAD_BUFFER);
- blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
+ blockFactory = S3ADataBlocks.createFactory(createStoreContext(), blockOutputBuffer);
blockOutputActiveBlocks = intOption(conf,
FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
// If CSE is enabled, do multipart uploads serially.
@@ -835,12 +832,13 @@ public void initialize(URI name, Configuration originalConf)
protected S3AStore createS3AStore(final ClientManager clientManager,
final int rateLimitCapacity) {
return new S3AStoreBuilder()
+ .withAuditSpanSource(getAuditManager())
.withClientManager(clientManager)
.withDurationTrackerFactory(getDurationTrackerFactory())
- .withStoreContextFactory(this)
- .withAuditSpanSource(getAuditManager())
+ .withFsStatistics(getFsStatistics())
.withInstrumentation(getInstrumentation())
.withStatisticsContext(statisticsContext)
+ .withStoreContextFactory(this)
.withStorageStatistics(getStorageStatistics())
.withReadRateLimiter(unlimitedRate())
.withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity))
@@ -1966,9 +1964,48 @@ private final class WriteOperationHelperCallbacksImpl
implements WriteOperationHelper.WriteOperationHelperCallbacks {
@Override
+ @Retries.OnceRaw
public CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request) {
- return getS3Client().completeMultipartUpload(request);
+ return store.completeMultipartUpload(request);
+ }
+
+ @Override
+ @Retries.OnceRaw
+ public UploadPartResponse uploadPart(
+ final UploadPartRequest request,
+ final RequestBody body,
+ final DurationTrackerFactory durationTrackerFactory)
+ throws AwsServiceException, UncheckedIOException {
+ return store.uploadPart(request, body, durationTrackerFactory);
+ }
+
+ /**
+ * Perform post-write actions.
+ *
+ * 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);
+
}
}
@@ -2927,7 +2964,7 @@ public S3AStorageStatistics getStorageStatistics() {
/**
* Get the instrumentation's IOStatistics.
- * @return statistics
+ * @return statistics or null if instrumentation has not yet been instantiated.
*/
@Override
public IOStatistics getIOStatistics() {
@@ -2956,9 +2993,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() {
*/
protected DurationTrackerFactory nonNullDurationTrackerFactory(
DurationTrackerFactory factory) {
- return factory != null
- ? factory
- : getDurationTrackerFactory();
+ return store.nonNullDurationTrackerFactory(factory);
}
/**
@@ -3273,18 +3308,7 @@ public PutObjectRequest.Builder newPutObjectRequestBuilder(String key,
@Retries.OnceRaw
public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
ProgressableProgressListener listener) throws IOException {
- long len = getPutRequestLength(putObjectRequest);
- LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key());
- incrementPutStartStatistics(len);
-
- FileUpload upload = store.getOrCreateTransferManager().uploadFile(
- UploadFileRequest.builder()
- .putObjectRequest(putObjectRequest)
- .source(file)
- .addTransferListener(listener)
- .build());
-
- return new UploadInfo(upload, len);
+ return store.putObject(putObjectRequest, file, listener);
}
/**
@@ -3297,9 +3321,8 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
* 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
*/
@@ -3307,27 +3330,27 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
@Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed")
PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest,
PutObjectOptions putOptions,
- S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
+ S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws SdkException {
+
long len = getPutRequestLength(putObjectRequest);
LOG.debug("PUT {} bytes to {}", len, putObjectRequest.key());
incrementPutStartStatistics(len);
+ final UploadContentProviders.BaseContentProvider provider =
+ uploadData.getContentProvider();
try {
PutObjectResponse response =
trackDurationOfSupplier(nonNullDurationTrackerFactory(durationTrackerFactory),
OBJECT_PUT_REQUESTS.getSymbol(),
- () -> isFile
- ? getS3Client().putObject(putObjectRequest,
- RequestBody.fromFile(uploadData.getFile()))
- : getS3Client().putObject(putObjectRequest,
- RequestBody.fromInputStream(uploadData.getUploadStream(),
- putObjectRequest.contentLength())));
+ () -> getS3Client().putObject(putObjectRequest,
+ RequestBody.fromContentProvider(
+ provider,
+ provider.getSize(),
+ CONTENT_TYPE_OCTET_STREAM)));
incrementPutCompletedStatistics(true, len);
// apply any post-write actions.
- finishedWrite(putObjectRequest.key(), len,
- response.eTag(), response.versionId(),
- putOptions);
+ finishedWrite(putObjectRequest.key(), len, putOptions);
return response;
} catch (SdkException e) {
incrementPutCompletedStatistics(false, len);
@@ -3385,13 +3408,8 @@ UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body,
*
* @param bytes bytes in the request.
*/
- public void incrementPutStartStatistics(long bytes) {
- LOG.debug("PUT start {} bytes", bytes);
- incrementWriteOperations();
- incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
- if (bytes > 0) {
- incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
- }
+ protected void incrementPutStartStatistics(long bytes) {
+ store.incrementPutStartStatistics(bytes);
}
/**
@@ -3401,14 +3419,8 @@ public void incrementPutStartStatistics(long bytes) {
* @param success did the operation succeed?
* @param bytes bytes in the request.
*/
- public void incrementPutCompletedStatistics(boolean success, long bytes) {
- LOG.debug("PUT completed success={}; {} bytes", success, bytes);
- if (bytes > 0) {
- incrementStatistic(OBJECT_PUT_BYTES, bytes);
- decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
- }
- incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
- decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
+ protected void incrementPutCompletedStatistics(boolean success, long bytes) {
+ store.incrementPutCompletedStatistics(success, bytes);
}
/**
@@ -3418,12 +3430,8 @@ public void incrementPutCompletedStatistics(boolean success, long bytes) {
* @param key key to file that is being written (for logging)
* @param bytes bytes successfully uploaded.
*/
- public void incrementPutProgressStatistics(String key, long bytes) {
- PROGRESS.debug("PUT {}: {} bytes", key, bytes);
- incrementWriteOperations();
- if (bytes > 0) {
- statistics.incrementBytesWritten(bytes);
- }
+ protected void incrementPutProgressStatistics(String key, long bytes) {
+ store.incrementPutProgressStatistics(key, bytes);
}
/**
@@ -4254,6 +4262,7 @@ public boolean deleteLocal(Path path, boolean recursive) throws IOException {
}
@Override
+ @Retries.RetryTranslated
public void copyLocalFileFromTo(File file, Path from, Path to) throws IOException {
// the duration of the put is measured, but the active span is the
// constructor-supplied one -this ensures all audit log events are grouped correctly
@@ -4270,11 +4279,13 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio
}
@Override
+ @Retries.RetryTranslated
public FileStatus getFileStatus(Path f) throws IOException {
return S3AFileSystem.this.getFileStatus(f);
}
@Override
+ @Retries.RetryTranslated
public boolean createEmptyDir(Path path, StoreContext storeContext)
throws IOException {
return trackDuration(getDurationTrackerFactory(),
@@ -4295,8 +4306,9 @@ public boolean createEmptyDir(Path path, StoreContext storeContext)
* @param putOptions put object options
* @return the upload result
* @throws IOException IO failure
+ * @throws CancellationException if the wait() was cancelled
*/
- @Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed")
+ @Retries.OnceTranslated("For PUT; post-PUT actions are RetrySwallowed")
PutObjectResponse executePut(
final PutObjectRequest putObjectRequest,
final Progressable progress,
@@ -4306,49 +4318,21 @@ PutObjectResponse executePut(
String key = putObjectRequest.key();
long len = getPutRequestLength(putObjectRequest);
ProgressableProgressListener listener =
- new ProgressableProgressListener(this, putObjectRequest.key(), progress);
+ new ProgressableProgressListener(store, putObjectRequest.key(), progress);
UploadInfo info = putObject(putObjectRequest, file, listener);
- PutObjectResponse result = waitForUploadCompletion(key, info).response();
+ PutObjectResponse result = store.waitForUploadCompletion(key, info).response();
listener.uploadCompleted(info.getFileUpload());
// post-write actions
- finishedWrite(key, len,
- result.eTag(), result.versionId(), putOptions);
+ finishedWrite(key, len, putOptions);
return result;
}
- /**
- * Wait for an upload to complete.
- * If the upload (or its result collection) failed, this is where
- * the failure is raised as an AWS exception.
- * Calls {@link #incrementPutCompletedStatistics(boolean, long)}
- * to update the statistics.
- * @param key destination key
- * @param uploadInfo upload to wait for
- * @return the upload result
- * @throws IOException IO failure
- */
- @Retries.OnceRaw
- CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
- throws IOException {
- FileUpload upload = uploadInfo.getFileUpload();
- try {
- CompletedFileUpload result = upload.completionFuture().join();
- incrementPutCompletedStatistics(true, uploadInfo.getLength());
- return result;
- } catch (CompletionException e) {
- LOG.info("Interrupted: aborting upload");
- incrementPutCompletedStatistics(false, uploadInfo.getLength());
- throw extractException("upload", key, e);
- }
- }
-
/**
* This override bypasses checking for existence.
*
* @param f the path to delete; this may be unqualified.
- * @return true, always. * @param f the path to delete.
- * @return true if deleteOnExit is successful, otherwise false.
+ * @return true, always.
* @throws IOException IO failure
*/
@Override
@@ -4729,9 +4713,7 @@ CreateMultipartUploadResponse initiateMultipartUpload(
* {@link #deleteUnnecessaryFakeDirectories(Path)}
* if directory markers are not being retained.
* @param key key written to
- * @param length total length of file written
- * @param eTag eTag of the written object
- * @param versionId S3 object versionId of the written object
+ * @param length total length of file written
* @param putOptions put object options
*/
@InterfaceAudience.Private
@@ -4739,11 +4721,9 @@ CreateMultipartUploadResponse initiateMultipartUpload(
void finishedWrite(
String key,
long length,
- String eTag,
- String versionId,
PutObjectOptions putOptions) {
- LOG.debug("Finished write to {}, len {}. etag {}, version {}",
- key, length, eTag, versionId);
+ LOG.debug("Finished write to {}, len {}.",
+ key, length);
Preconditions.checkArgument(length >= 0, "content length is negative");
if (!putOptions.isKeepMarkers()) {
Path p = keyToQualifiedPath(key);
@@ -4837,18 +4817,16 @@ private void createFakeDirectory(final String objectName,
@Retries.RetryTranslated
private void createEmptyObject(final String objectName, PutObjectOptions putOptions)
throws IOException {
- final InputStream im = new InputStream() {
- @Override
- public int read() throws IOException {
- return -1;
- }
- };
- S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(im);
+ S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(
+ new byte[0], 0, 0, null);
invoker.retry("PUT 0-byte object ", objectName, true,
- () -> putObjectDirect(getRequestFactory().newDirectoryMarkerRequest(objectName).build(),
- putOptions, uploadData, false, getDurationTrackerFactory()));
+ () -> putObjectDirect(
+ getRequestFactory().newDirectoryMarkerRequest(objectName).build(),
+ putOptions,
+ uploadData,
+ getDurationTrackerFactory()));
incrementPutProgressStatistics(objectName, 0);
instrumentation.directoryCreated();
}
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 685b95dfc3d77..69390a8cc724e 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;
@@ -298,7 +299,7 @@ public static IOException translateException(@Nullable String operation,
case SC_405_METHOD_NOT_ALLOWED:
case SC_415_UNSUPPORTED_MEDIA_TYPE:
case SC_501_NOT_IMPLEMENTED:
- ioe = new AWSUnsupportedFeatureException(message, s3Exception);
+ ioe = new AWSUnsupportedFeatureException(message, ase);
break;
// precondition failure: the object is there, but the precondition
@@ -1177,6 +1178,19 @@ public static S3AFileStatus[] iteratorToStatuses(
return statuses;
}
+ /**
+ * Get the length of the PUT, verifying that the length is known.
+ * @param putObjectRequest a request bound to a file or a stream.
+ * @return the request length
+ * @throws IllegalArgumentException if the length is negative
+ */
+ public static long getPutRequestLength(PutObjectRequest putObjectRequest) {
+ long len = putObjectRequest.contentLength();
+
+ Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length");
+ return len;
+ }
+
/**
* An interface for use in lambda-expressions working with
* directory tree listings.
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 7d23c10d8b5c8..5bb64ddc28920 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
@@ -294,4 +294,11 @@ private InternalConstants() {
*/
public static final String AUTH_SCHEME_AWS_SIGV_4 = "aws.auth#sigv4";
+
+ /**
+ * Progress logge name; fairly noisy.
+ */
+ public static final String UPLOAD_PROGRESS_LOG_NAME =
+ "org.apache.hadoop.fs.s3a.S3AFileSystem.Progress";
+
}
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.timeout500
+
+
+ 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