diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
index 44f794aa77478..e3deda775286a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
@@ -384,6 +384,47 @@ public final class StoreStatisticNames {
public static final String ACTION_HTTP_PATCH_REQUEST
= "action_http_patch_request";
+ /**
+ * HTTP error response: {@value}.
+ */
+ public static final String HTTP_RESPONSE_400
+ = "http_response_400";
+
+ /**
+ * HTTP error response: {@value}.
+ * Returned by some stores for throttling events.
+ */
+ public static final String HTTP_RESPONSE_429
+ = "http_response_429";
+
+ /**
+ * Other 4XX HTTP response: {@value}.
+ * (404 responses are excluded as they are rarely 'errors'
+ * and will be reported differently if they are.
+ */
+ public static final String HTTP_RESPONSE_4XX
+ = "http_response_4XX";
+
+ /**
+ * HTTP error response: {@value}.
+ * Sign of server-side problems, possibly transient
+ */
+ public static final String HTTP_RESPONSE_500
+ = "http_response_500";
+
+ /**
+ * HTTP error response: {@value}.
+ * AWS Throttle.
+ */
+ public static final String HTTP_RESPONSE_503
+ = "http_response_503";
+
+ /**
+ * Other 5XX HTTP response: {@value}.
+ */
+ public static final String HTTP_RESPONSE_5XX
+ = "http_response_5XX";
+
/**
* An HTTP POST request was made: {@value}.
*/
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
new file mode 100644
index 0000000000000..08d15a5e2eb9a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/ByteBufferInputStream.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.store;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.util.Preconditions;
+
+/**
+ * Provide an input stream from a byte buffer; supporting
+ * {@link #mark(int)}.
+ */
+public final class ByteBufferInputStream extends InputStream {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(ByteBufferInputStream.class);
+
+ /** Size of the buffer. */
+ private final int size;
+
+ /**
+ * Not final so that in close() it will be set to null, which
+ * may result in faster cleanup of the buffer.
+ */
+ private ByteBuffer byteBuffer;
+
+ public ByteBufferInputStream(int size,
+ ByteBuffer byteBuffer) {
+ LOG.debug("Creating ByteBufferInputStream of size {}", size);
+ this.size = size;
+ this.byteBuffer = byteBuffer;
+ }
+
+ /**
+ * After the stream is closed, set the local reference to the byte
+ * buffer to null; this guarantees that future attempts to use
+ * stream methods will fail.
+ */
+ @Override
+ public synchronized void close() {
+ LOG.debug("ByteBufferInputStream.close()");
+ byteBuffer = null;
+ }
+
+ /**
+ * Is the stream open?
+ * @return true if the stream has not been closed.
+ */
+ public synchronized boolean isOpen() {
+ return byteBuffer != null;
+ }
+
+ /**
+ * Verify that the stream is open.
+ * @throws IOException if the stream is closed
+ */
+ private void verifyOpen() throws IOException {
+ if (byteBuffer == null) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+ }
+
+ /**
+ * Check the open state.
+ * @throws IllegalStateException if the stream is closed.
+ */
+ private void checkOpenState() {
+ Preconditions.checkState(isOpen(),
+ FSExceptionMessages.STREAM_IS_CLOSED);
+ }
+
+ public synchronized int read() throws IOException {
+ if (available() > 0) {
+ return byteBuffer.get() & 0xFF;
+ } else {
+ return -1;
+ }
+ }
+
+ @Override
+ public synchronized long skip(long offset) throws IOException {
+ verifyOpen();
+ long newPos = position() + offset;
+ if (newPos < 0) {
+ throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+ }
+ if (newPos > size) {
+ throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+ }
+ byteBuffer.position((int) newPos);
+ return newPos;
+ }
+
+ @Override
+ public synchronized int available() {
+ checkOpenState();
+ return byteBuffer.remaining();
+ }
+
+ /**
+ * Get the current buffer position.
+ * @return the buffer position
+ */
+ public synchronized int position() {
+ checkOpenState();
+ return byteBuffer.position();
+ }
+
+ /**
+ * Check if there is data left.
+ * @return true if there is data remaining in the buffer.
+ */
+ public synchronized boolean hasRemaining() {
+ checkOpenState();
+ return byteBuffer.hasRemaining();
+ }
+
+ @Override
+ public synchronized void mark(int readlimit) {
+ LOG.debug("mark at {}", position());
+ checkOpenState();
+ byteBuffer.mark();
+ }
+
+ @Override
+ public synchronized void reset() throws IOException {
+ LOG.debug("reset");
+ checkOpenState();
+ byteBuffer.reset();
+ }
+
+ @Override
+ public boolean markSupported() {
+ return true;
+ }
+
+ /**
+ * Read in data.
+ * @param b destination buffer.
+ * @param offset offset within the buffer.
+ * @param length length of bytes to read.
+ * @throws EOFException if the position is negative
+ * @throws IndexOutOfBoundsException if there isn't space for the
+ * amount of data requested.
+ * @throws IllegalArgumentException other arguments are invalid.
+ */
+ @SuppressWarnings("NullableProblems")
+ public synchronized int read(byte[] b, int offset, int length)
+ throws IOException {
+ Preconditions.checkArgument(length >= 0, "length is negative");
+ Preconditions.checkArgument(b != null, "Null buffer");
+ if (b.length - offset < length) {
+ throw new IndexOutOfBoundsException(
+ FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
+ + ": request length =" + length
+ + ", with offset =" + offset
+ + "; buffer capacity =" + (b.length - offset));
+ }
+ verifyOpen();
+ if (!hasRemaining()) {
+ return -1;
+ }
+
+ int toRead = Math.min(length, available());
+ byteBuffer.get(b, offset, toRead);
+ return toRead;
+ }
+
+ @Override
+ public String toString() {
+ return "ByteBufferInputStream{" +
+ "size=" + size +
+ ", byteBuffer=" + byteBuffer +
+ ((byteBuffer != null) ? ", available=" + byteBuffer.remaining() : "") +
+ "} " + super.toString();
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java
index 0ae9ee6378b57..e8b6684f12015 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java
@@ -22,7 +22,6 @@
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
-import java.io.EOFException;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
@@ -40,7 +39,6 @@
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.DirectBufferPool;
@@ -777,158 +775,8 @@ public String toString() {
'}';
}
- /**
- * Provide an input stream from a byte buffer; supporting
- * {@link #mark(int)}, which is required to enable replay of failed
- * PUT attempts.
- */
- class ByteBufferInputStream extends InputStream {
-
- private final int size;
- private ByteBuffer byteBuffer;
-
- ByteBufferInputStream(int size,
- ByteBuffer byteBuffer) {
- LOG.debug("Creating ByteBufferInputStream of size {}", size);
- this.size = size;
- this.byteBuffer = byteBuffer;
- }
-
- /**
- * After the stream is closed, set the local reference to the byte
- * buffer to null; this guarantees that future attempts to use
- * stream methods will fail.
- */
- @Override
- public synchronized void close() {
- LOG.debug("ByteBufferInputStream.close() for {}",
- ByteBufferBlock.super.toString());
- byteBuffer = null;
- }
-
- /**
- * Verify that the stream is open.
- *
- * @throws IOException if the stream is closed
- */
- private void verifyOpen() throws IOException {
- if (byteBuffer == null) {
- throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
- }
- }
-
- public synchronized int read() throws IOException {
- if (available() > 0) {
- return byteBuffer.get() & 0xFF;
- } else {
- return -1;
- }
- }
-
- @Override
- public synchronized long skip(long offset) throws IOException {
- verifyOpen();
- long newPos = position() + offset;
- if (newPos < 0) {
- throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
- }
- if (newPos > size) {
- throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
- }
- byteBuffer.position((int) newPos);
- return newPos;
- }
-
- @Override
- public synchronized int available() {
- Preconditions.checkState(byteBuffer != null,
- FSExceptionMessages.STREAM_IS_CLOSED);
- return byteBuffer.remaining();
- }
-
- /**
- * Get the current buffer position.
- *
- * @return the buffer position
- */
- public synchronized int position() {
- return byteBuffer.position();
- }
-
- /**
- * Check if there is data left.
- *
- * @return true if there is data remaining in the buffer.
- */
- public synchronized boolean hasRemaining() {
- return byteBuffer.hasRemaining();
- }
-
- @Override
- public synchronized void mark(int readlimit) {
- LOG.debug("mark at {}", position());
- byteBuffer.mark();
- }
-
- @Override
- public synchronized void reset() throws IOException {
- LOG.debug("reset");
- byteBuffer.reset();
- }
-
- @Override
- public boolean markSupported() {
- return true;
- }
-
- /**
- * Read in data.
- *
- * @param b destination buffer.
- * @param offset offset within the buffer.
- * @param length length of bytes to read.
- * @throws EOFException if the position is negative
- * @throws IndexOutOfBoundsException if there isn't space for the
- * amount of data requested.
- * @throws IllegalArgumentException other arguments are invalid.
- */
- @SuppressWarnings("NullableProblems")
- public synchronized int read(byte[] b, int offset, int length)
- throws IOException {
- Preconditions.checkArgument(length >= 0, "length is negative");
- Preconditions.checkArgument(b != null, "Null buffer");
- if (b.length - offset < length) {
- throw new IndexOutOfBoundsException(
- FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER
- + ": request length =" + length
- + ", with offset =" + offset
- + "; buffer capacity =" + (b.length - offset));
- }
- verifyOpen();
- if (!hasRemaining()) {
- return -1;
- }
-
- int toRead = Math.min(length, available());
- byteBuffer.get(b, offset, toRead);
- return toRead;
- }
-
- @Override
- public String toString() {
- final StringBuilder sb = new StringBuilder(
- "ByteBufferInputStream{");
- sb.append("size=").append(size);
- ByteBuffer buf = this.byteBuffer;
- if (buf != null) {
- sb.append(", available=").append(buf.remaining());
- }
- sb.append(", ").append(ByteBufferBlock.super.toString());
- sb.append('}');
- return sb.toString();
- }
- }
}
+
}
// ====================================================================
@@ -1124,4 +972,5 @@ void closeBlock() {
}
}
}
+
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java
index 0a0d023d931d0..fca521a5b8689 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java
@@ -26,6 +26,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
@@ -33,10 +34,14 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSBuilder;
+import org.apache.hadoop.util.Time;
/**
* Future IO Helper methods.
@@ -53,12 +58,18 @@
* {@code UncheckedIOException} raised in the future.
* This makes it somewhat easier to execute IOException-raising
* code inside futures.
- *
+ *
+ * Important: any {@code CancellationException} raised by the future
+ * is rethrown unchanged. This has been the implicit behavior since
+ * this code was first written, and is now explicitly documented.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public final class FutureIO {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(FutureIO.class);
+
private FutureIO() {
}
@@ -68,17 +79,28 @@ private FutureIO() {
* Any exception generated in the future is
* extracted and rethrown.
*
+ * If this thread is interrupted while waiting for the future to complete,
+ * an {@code InterruptedIOException} is raised.
+ * However, if the future is cancelled, a {@code CancellationException}
+ * is raised in the {code Future.get()} call. This is
+ * passed up as is -so allowing the caller to distinguish between
+ * thread interruption (such as when speculative task execution is aborted)
+ * and future cancellation.
* @param future future to evaluate
* @param type of the result.
* @return the result, if all went well.
- * @throws InterruptedIOException future was interrupted
+ * @throws InterruptedIOException waiting for future completion was interrupted
+ * @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
public static T awaitFuture(final Future future)
- throws InterruptedIOException, IOException, RuntimeException {
+ throws InterruptedIOException, IOException, CancellationException, RuntimeException {
try {
return future.get();
+ } catch (CancellationException e) {
+ LOG.debug("Future {} was cancelled", future, e);
+ throw e;
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
@@ -94,11 +116,12 @@ public static T awaitFuture(final Future future)
* extracted and rethrown.
*
* @param future future to evaluate
- * @param timeout timeout to wait
+ * @param timeout timeout to wait.
* @param unit time unit.
* @param type of the result.
* @return the result, if all went well.
- * @throws InterruptedIOException future was interrupted
+ * @throws InterruptedIOException waiting for future completion was interrupted
+ * @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
* @throws TimeoutException the future timed out.
@@ -106,10 +129,13 @@ public static T awaitFuture(final Future future)
public static T awaitFuture(final Future future,
final long timeout,
final TimeUnit unit)
- throws InterruptedIOException, IOException, RuntimeException,
+ throws InterruptedIOException, IOException, CancellationException, RuntimeException,
TimeoutException {
try {
return future.get(timeout, unit);
+ } catch (CancellationException e) {
+ LOG.debug("Future {} was cancelled", future, e);
+ throw e;
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString())
.initCause(e);
@@ -128,12 +154,13 @@ public static T awaitFuture(final Future future,
* @param collection collection of futures to be evaluated
* @param type of the result.
* @return the list of future's result, if all went well.
- * @throws InterruptedIOException future was interrupted
+ * @throws InterruptedIOException waiting for future completion was interrupted
+ * @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
public static List awaitAllFutures(final Collection> collection)
- throws InterruptedIOException, IOException, RuntimeException {
+ throws InterruptedIOException, IOException, CancellationException, RuntimeException {
List results = new ArrayList<>();
for (Future future : collection) {
results.add(awaitFuture(future));
@@ -148,19 +175,19 @@ public static List awaitAllFutures(final Collection> collection
* This method blocks until all futures in the collection have completed or
* the timeout expires, whichever happens first. If any future throws an
* exception during its execution, this method extracts and rethrows that exception.
- *
* @param collection collection of futures to be evaluated
* @param duration timeout duration
* @param type of the result.
* @return the list of future's result, if all went well.
- * @throws InterruptedIOException future was interrupted
+ * @throws InterruptedIOException waiting for future completion was interrupted
+ * @throws CancellationException if the future itself was cancelled
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
* @throws TimeoutException the future timed out.
*/
public static List awaitAllFutures(final Collection> collection,
final Duration duration)
- throws InterruptedIOException, IOException, RuntimeException,
+ throws InterruptedIOException, IOException, CancellationException, RuntimeException,
TimeoutException {
List results = new ArrayList<>();
for (Future future : collection) {
@@ -169,6 +196,48 @@ public static List awaitAllFutures(final Collection> collection
return results;
}
+ /**
+ * Cancels a collection of futures and awaits the specified duration for their completion.
+ *
+ * This method blocks until all futures in the collection have completed or
+ * the timeout expires, whichever happens first.
+ * All exceptions thrown by the futures are ignored. as is any TimeoutException.
+ * @param collection collection of futures to be evaluated
+ * @param interruptIfRunning should the cancel interrupt any active futures?
+ * @param duration total timeout duration
+ * @param type of the result.
+ * @return all futures which completed successfully.
+ */
+ public static List cancelAllFuturesAndAwaitCompletion(
+ final Collection> collection,
+ final boolean interruptIfRunning,
+ final Duration duration) {
+
+ for (Future future : collection) {
+ future.cancel(interruptIfRunning);
+ }
+ // timeout is relative to the start of the operation
+ long timeout = duration.toMillis();
+ List results = new ArrayList<>();
+ for (Future future : collection) {
+ long start = Time.now();
+ try {
+ results.add(awaitFuture(future, timeout, TimeUnit.MILLISECONDS));
+ } catch (CancellationException | IOException | TimeoutException e) {
+ // swallow
+ LOG.debug("Ignoring exception of cancelled future", e);
+ }
+ // measure wait time and reduce timeout accordingly
+ long waited = Time.now() - start;
+ timeout -= waited;
+ if (timeout < 0) {
+ // very brief timeout always
+ timeout = 0;
+ }
+ }
+ return results;
+ }
+
/**
* From the inner cause of an execution exception, extract the inner cause
* if it is an IOE or RTE.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
index f7c72f8530959..fa942efc20f9a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java
@@ -22,12 +22,19 @@
/**
* A 5xx response came back from a service.
- * The 500 error considered retriable by the AWS SDK, which will have already
- * tried it {@code fs.s3a.attempts.maximum} times before reaching s3a
+ *
+ * The 500 error is considered retryable by the AWS SDK, which will have already
+ * retried it {@code fs.s3a.attempts.maximum} times before reaching s3a
* code.
- * How it handles other 5xx errors is unknown: S3A FS code will treat them
- * as unrecoverable on the basis that they indicate some third-party store
- * or gateway problem.
+ *
+ * These are rare, but can occur; they are considered retryable.
+ * Note that HADOOP-19221 shows a failure condition where the
+ * SDK itself did not recover on retry from the error.
+ * In S3A code, retries happen if the retry policy configuration
+ * {@code fs.s3a.retry.http.5xx.errors} is {@code true}.
+ *
+ * In third party stores it may have a similar meaning -though it
+ * can often just mean "misconfigured server".
*/
public class AWSStatus500Exception extends AWSServiceIOException {
public AWSStatus500Exception(String operation,
@@ -35,8 +42,4 @@ public AWSStatus500Exception(String operation,
super(operation, cause);
}
- @Override
- public boolean retryable() {
- return false;
- }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 7e614bc11d6ee..cf7bc3ddcf248 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -1119,6 +1119,22 @@ private Constants() {
*/
public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms";
+
+ /**
+ * Should S3A connector retry on all 5xx errors which don't have
+ * explicit support: {@value}?
+ *
+ * This is in addition to any retries the AWS SDK itself does, which
+ * is known to retry on many of these (e.g. 500).
+ */
+ public static final String RETRY_HTTP_5XX_ERRORS =
+ "fs.s3a.retry.http.5xx.errors";
+
+ /**
+ * Default value for {@link #RETRY_HTTP_5XX_ERRORS}: {@value}.
+ */
+ public static final boolean DEFAULT_RETRY_HTTP_5XX_ERRORS = true;
+
/**
* Should etags be exposed as checksums?
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
index 7ee6c55c191b7..25b5d774cdf7f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
@@ -29,21 +29,21 @@
*/
public class ProgressableProgressListener implements TransferListener {
private static final Logger LOG = S3AFileSystem.LOG;
- private final S3AFileSystem fs;
+ private final S3AStore store;
private final String key;
private final Progressable progress;
private long lastBytesTransferred;
/**
* Instantiate.
- * @param fs filesystem: will be invoked with statistics updates
+ * @param store store: will be invoked with statistics updates
* @param key key for the upload
* @param progress optional callback for progress.
*/
- public ProgressableProgressListener(S3AFileSystem fs,
+ public ProgressableProgressListener(S3AStore store,
String key,
Progressable progress) {
- this.fs = fs;
+ this.store = store;
this.key = key;
this.progress = progress;
this.lastBytesTransferred = 0;
@@ -51,12 +51,12 @@ public ProgressableProgressListener(S3AFileSystem fs,
@Override
public void transferInitiated(TransferListener.Context.TransferInitiated context) {
- fs.incrementWriteOperations();
+ store.incrementWriteOperations();
}
@Override
public void transferComplete(TransferListener.Context.TransferComplete context) {
- fs.incrementWriteOperations();
+ store.incrementWriteOperations();
}
@Override
@@ -68,7 +68,7 @@ public void bytesTransferred(TransferListener.Context.BytesTransferred context)
long transferred = context.progressSnapshot().transferredBytes();
long delta = transferred - lastBytesTransferred;
- fs.incrementPutProgressStatistics(key, delta);
+ store.incrementPutProgressStatistics(key, delta);
lastBytesTransferred = transferred;
}
@@ -84,7 +84,7 @@ public long uploadCompleted(ObjectTransfer upload) {
upload.progress().snapshot().transferredBytes() - lastBytesTransferred;
if (delta > 0) {
LOG.debug("S3A write delta changed after finished: {} bytes", delta);
- fs.incrementPutProgressStatistics(key, delta);
+ store.incrementPutProgressStatistics(key, delta);
}
return delta;
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 5fe39ac6ea336..741a78a0537f2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -18,33 +18,41 @@
package org.apache.hadoop.fs.s3a;
+import java.io.FileNotFoundException;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.Locale;
+import java.util.Map;
import java.util.StringJoiner;
-import java.util.concurrent.ExecutionException;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
-import software.amazon.awssdk.core.exception.SdkException;
+import javax.annotation.Nonnull;
+
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.model.CompletedPart;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
-import software.amazon.awssdk.services.s3.model.PutObjectResponse;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.ClosedIOException;
import org.apache.hadoop.fs.s3a.impl.ProgressListener;
import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
import org.apache.hadoop.util.Preconditions;
-import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
@@ -68,25 +76,59 @@
import org.apache.hadoop.util.Progressable;
import static java.util.Objects.requireNonNull;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.*;
import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+import static org.apache.hadoop.util.functional.FutureIO.awaitAllFutures;
+import static org.apache.hadoop.util.functional.FutureIO.cancelAllFuturesAndAwaitCompletion;
/**
* Upload files/parts directly via different buffering mechanisms:
* including memory and disk.
+ *
+ * Key Features
+ *
+ *
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 ea2800b3ab8f5..fb936f1848d1d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -21,7 +21,6 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
-import java.io.InputStream;
import java.io.InterruptedIOException;
import java.io.UncheckedIOException;
import java.net.URI;
@@ -43,6 +42,7 @@
import java.util.Set;
import java.util.Objects;
import java.util.TreeSet;
+import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutorService;
@@ -85,11 +85,8 @@
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import software.amazon.awssdk.transfer.s3.model.CompletedCopy;
-import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
import software.amazon.awssdk.transfer.s3.model.Copy;
import software.amazon.awssdk.transfer.s3.model.CopyRequest;
-import software.amazon.awssdk.transfer.s3.model.FileUpload;
-import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.slf4j.Logger;
@@ -149,6 +146,7 @@
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
import org.apache.hadoop.fs.s3a.impl.StoreContextFactory;
+import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
@@ -248,6 +246,7 @@
import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_PERFORMANCE;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_REQUIRED_EXCEPTION;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.ARN_BUCKET_OPTION;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
@@ -356,8 +355,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
/** Log to warn of storage class configuration problems. */
private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG);
- private static final Logger PROGRESS =
- LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
private LocalDirAllocator directoryAllocator;
private String cannedACL;
@@ -729,7 +726,7 @@ public void initialize(URI name, Configuration originalConf)
}
blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
DEFAULT_FAST_UPLOAD_BUFFER);
- blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
+ blockFactory = S3ADataBlocks.createFactory(createStoreContext(), blockOutputBuffer);
blockOutputActiveBlocks = intOption(conf,
FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
// If CSE is enabled, do multipart uploads serially.
@@ -829,12 +826,13 @@ public void initialize(URI name, Configuration originalConf)
protected S3AStore createS3AStore(final ClientManager clientManager,
final int rateLimitCapacity) {
return new S3AStoreBuilder()
+ .withAuditSpanSource(getAuditManager())
.withClientManager(clientManager)
.withDurationTrackerFactory(getDurationTrackerFactory())
- .withStoreContextFactory(this)
- .withAuditSpanSource(getAuditManager())
+ .withFsStatistics(getFsStatistics())
.withInstrumentation(getInstrumentation())
.withStatisticsContext(statisticsContext)
+ .withStoreContextFactory(this)
.withStorageStatistics(getStorageStatistics())
.withReadRateLimiter(unlimitedRate())
.withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity))
@@ -1960,9 +1958,48 @@ private final class WriteOperationHelperCallbacksImpl
implements WriteOperationHelper.WriteOperationHelperCallbacks {
@Override
+ @Retries.OnceRaw
public CompleteMultipartUploadResponse completeMultipartUpload(
CompleteMultipartUploadRequest request) {
- return getS3Client().completeMultipartUpload(request);
+ return store.completeMultipartUpload(request);
+ }
+
+ @Override
+ @Retries.OnceRaw
+ public UploadPartResponse uploadPart(
+ final UploadPartRequest request,
+ final RequestBody body,
+ final DurationTrackerFactory durationTrackerFactory)
+ throws AwsServiceException, UncheckedIOException {
+ return store.uploadPart(request, body, durationTrackerFactory);
+ }
+
+ /**
+ * Perform post-write actions.
+ *
+ * This operation MUST be called after any PUT/multipart PUT completes
+ * successfully.
+ *
+ * The actions include calling
+ * {@link #deleteUnnecessaryFakeDirectories(Path)}
+ * if directory markers are not being retained.
+ * @param eTag eTag of the written object
+ * @param versionId S3 object versionId of the written object
+ * @param key key written to
+ * @param length total length of file written
+ * @param putOptions put object options
+ */
+ @Override
+ @Retries.RetryExceptionsSwallowed
+ public void finishedWrite(
+ String key,
+ long length,
+ PutObjectOptions putOptions) {
+ S3AFileSystem.this.finishedWrite(
+ key,
+ length,
+ putOptions);
+
}
}
@@ -2921,7 +2958,7 @@ public S3AStorageStatistics getStorageStatistics() {
/**
* Get the instrumentation's IOStatistics.
- * @return statistics
+ * @return statistics or null if instrumentation has not yet been instantiated.
*/
@Override
public IOStatistics getIOStatistics() {
@@ -2950,9 +2987,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() {
*/
protected DurationTrackerFactory nonNullDurationTrackerFactory(
DurationTrackerFactory factory) {
- return factory != null
- ? factory
- : getDurationTrackerFactory();
+ return store.nonNullDurationTrackerFactory(factory);
}
/**
@@ -3267,18 +3302,7 @@ public PutObjectRequest.Builder newPutObjectRequestBuilder(String key,
@Retries.OnceRaw
public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
ProgressableProgressListener listener) throws IOException {
- long len = getPutRequestLength(putObjectRequest);
- LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key());
- incrementPutStartStatistics(len);
-
- FileUpload upload = store.getOrCreateTransferManager().uploadFile(
- UploadFileRequest.builder()
- .putObjectRequest(putObjectRequest)
- .source(file)
- .addTransferListener(listener)
- .build());
-
- return new UploadInfo(upload, len);
+ return store.putObject(putObjectRequest, file, listener);
}
/**
@@ -3291,9 +3315,8 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
* Important: this call will close any input stream in the request.
* @param putObjectRequest the request
* @param putOptions put object options
- * @param durationTrackerFactory factory for duration tracking
* @param uploadData data to be uploaded
- * @param isFile represents if data to be uploaded is a file
+ * @param durationTrackerFactory factory for duration tracking
* @return the upload initiated
* @throws SdkException on problems
*/
@@ -3301,27 +3324,27 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
@Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed")
PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest,
PutObjectOptions putOptions,
- S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
+ S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws SdkException {
+
long len = getPutRequestLength(putObjectRequest);
LOG.debug("PUT {} bytes to {}", len, putObjectRequest.key());
incrementPutStartStatistics(len);
+ final UploadContentProviders.BaseContentProvider provider =
+ uploadData.getContentProvider();
try {
PutObjectResponse response =
trackDurationOfSupplier(nonNullDurationTrackerFactory(durationTrackerFactory),
OBJECT_PUT_REQUESTS.getSymbol(),
- () -> isFile
- ? getS3Client().putObject(putObjectRequest,
- RequestBody.fromFile(uploadData.getFile()))
- : getS3Client().putObject(putObjectRequest,
- RequestBody.fromInputStream(uploadData.getUploadStream(),
- putObjectRequest.contentLength())));
+ () -> getS3Client().putObject(putObjectRequest,
+ RequestBody.fromContentProvider(
+ provider,
+ provider.getSize(),
+ CONTENT_TYPE_OCTET_STREAM)));
incrementPutCompletedStatistics(true, len);
// apply any post-write actions.
- finishedWrite(putObjectRequest.key(), len,
- response.eTag(), response.versionId(),
- putOptions);
+ finishedWrite(putObjectRequest.key(), len, putOptions);
return response;
} catch (SdkException e) {
incrementPutCompletedStatistics(false, len);
@@ -3379,13 +3402,8 @@ UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body,
*
* @param bytes bytes in the request.
*/
- public void incrementPutStartStatistics(long bytes) {
- LOG.debug("PUT start {} bytes", bytes);
- incrementWriteOperations();
- incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
- if (bytes > 0) {
- incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
- }
+ protected void incrementPutStartStatistics(long bytes) {
+ store.incrementPutStartStatistics(bytes);
}
/**
@@ -3395,14 +3413,8 @@ public void incrementPutStartStatistics(long bytes) {
* @param success did the operation succeed?
* @param bytes bytes in the request.
*/
- public void incrementPutCompletedStatistics(boolean success, long bytes) {
- LOG.debug("PUT completed success={}; {} bytes", success, bytes);
- if (bytes > 0) {
- incrementStatistic(OBJECT_PUT_BYTES, bytes);
- decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
- }
- incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
- decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
+ protected void incrementPutCompletedStatistics(boolean success, long bytes) {
+ store.incrementPutCompletedStatistics(success, bytes);
}
/**
@@ -3412,12 +3424,8 @@ public void incrementPutCompletedStatistics(boolean success, long bytes) {
* @param key key to file that is being written (for logging)
* @param bytes bytes successfully uploaded.
*/
- public void incrementPutProgressStatistics(String key, long bytes) {
- PROGRESS.debug("PUT {}: {} bytes", key, bytes);
- incrementWriteOperations();
- if (bytes > 0) {
- statistics.incrementBytesWritten(bytes);
- }
+ protected void incrementPutProgressStatistics(String key, long bytes) {
+ store.incrementPutProgressStatistics(key, bytes);
}
/**
@@ -4248,6 +4256,7 @@ public boolean deleteLocal(Path path, boolean recursive) throws IOException {
}
@Override
+ @Retries.RetryTranslated
public void copyLocalFileFromTo(File file, Path from, Path to) throws IOException {
// the duration of the put is measured, but the active span is the
// constructor-supplied one -this ensures all audit log events are grouped correctly
@@ -4264,11 +4273,13 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio
}
@Override
+ @Retries.RetryTranslated
public FileStatus getFileStatus(Path f) throws IOException {
return S3AFileSystem.this.getFileStatus(f);
}
@Override
+ @Retries.RetryTranslated
public boolean createEmptyDir(Path path, StoreContext storeContext)
throws IOException {
return trackDuration(getDurationTrackerFactory(),
@@ -4289,8 +4300,9 @@ public boolean createEmptyDir(Path path, StoreContext storeContext)
* @param putOptions put object options
* @return the upload result
* @throws IOException IO failure
+ * @throws CancellationException if the wait() was cancelled
*/
- @Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed")
+ @Retries.OnceTranslated("For PUT; post-PUT actions are RetrySwallowed")
PutObjectResponse executePut(
final PutObjectRequest putObjectRequest,
final Progressable progress,
@@ -4300,49 +4312,21 @@ PutObjectResponse executePut(
String key = putObjectRequest.key();
long len = getPutRequestLength(putObjectRequest);
ProgressableProgressListener listener =
- new ProgressableProgressListener(this, putObjectRequest.key(), progress);
+ new ProgressableProgressListener(store, putObjectRequest.key(), progress);
UploadInfo info = putObject(putObjectRequest, file, listener);
- PutObjectResponse result = waitForUploadCompletion(key, info).response();
+ PutObjectResponse result = store.waitForUploadCompletion(key, info).response();
listener.uploadCompleted(info.getFileUpload());
// post-write actions
- finishedWrite(key, len,
- result.eTag(), result.versionId(), putOptions);
+ finishedWrite(key, len, putOptions);
return result;
}
- /**
- * Wait for an upload to complete.
- * If the upload (or its result collection) failed, this is where
- * the failure is raised as an AWS exception.
- * Calls {@link #incrementPutCompletedStatistics(boolean, long)}
- * to update the statistics.
- * @param key destination key
- * @param uploadInfo upload to wait for
- * @return the upload result
- * @throws IOException IO failure
- */
- @Retries.OnceRaw
- CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
- throws IOException {
- FileUpload upload = uploadInfo.getFileUpload();
- try {
- CompletedFileUpload result = upload.completionFuture().join();
- incrementPutCompletedStatistics(true, uploadInfo.getLength());
- return result;
- } catch (CompletionException e) {
- LOG.info("Interrupted: aborting upload");
- incrementPutCompletedStatistics(false, uploadInfo.getLength());
- throw extractException("upload", key, e);
- }
- }
-
/**
* This override bypasses checking for existence.
*
* @param f the path to delete; this may be unqualified.
- * @return true, always. * @param f the path to delete.
- * @return true if deleteOnExit is successful, otherwise false.
+ * @return true, always.
* @throws IOException IO failure
*/
@Override
@@ -4723,9 +4707,7 @@ CreateMultipartUploadResponse initiateMultipartUpload(
* {@link #deleteUnnecessaryFakeDirectories(Path)}
* if directory markers are not being retained.
* @param key key written to
- * @param length total length of file written
- * @param eTag eTag of the written object
- * @param versionId S3 object versionId of the written object
+ * @param length total length of file written
* @param putOptions put object options
*/
@InterfaceAudience.Private
@@ -4733,11 +4715,9 @@ CreateMultipartUploadResponse initiateMultipartUpload(
void finishedWrite(
String key,
long length,
- String eTag,
- String versionId,
PutObjectOptions putOptions) {
- LOG.debug("Finished write to {}, len {}. etag {}, version {}",
- key, length, eTag, versionId);
+ LOG.debug("Finished write to {}, len {}.",
+ key, length);
Preconditions.checkArgument(length >= 0, "content length is negative");
if (!putOptions.isKeepMarkers()) {
Path p = keyToQualifiedPath(key);
@@ -4831,18 +4811,16 @@ private void createFakeDirectory(final String objectName,
@Retries.RetryTranslated
private void createEmptyObject(final String objectName, PutObjectOptions putOptions)
throws IOException {
- final InputStream im = new InputStream() {
- @Override
- public int read() throws IOException {
- return -1;
- }
- };
- S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(im);
+ S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(
+ new byte[0], 0, 0, null);
invoker.retry("PUT 0-byte object ", objectName, true,
- () -> putObjectDirect(getRequestFactory().newDirectoryMarkerRequest(objectName).build(),
- putOptions, uploadData, false, getDurationTrackerFactory()));
+ () -> putObjectDirect(
+ getRequestFactory().newDirectoryMarkerRequest(objectName).build(),
+ putOptions,
+ uploadData,
+ getDurationTrackerFactory()));
incrementPutProgressStatistics(objectName, 0);
instrumentation.directoryCreated();
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 9d34457ab9443..e3bef9f470727 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -1505,6 +1505,7 @@ private OutputStreamStatistics(
INVOCATION_HFLUSH.getSymbol(),
INVOCATION_HSYNC.getSymbol())
.withGauges(
+ STREAM_WRITE_BLOCK_UPLOADS_ACTIVE.getSymbol(),
STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(),
STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol())
.withDurationTracking(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
index faf105c8e2c86..aa3d604cc4f83 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java
@@ -125,6 +125,11 @@ public class S3ARetryPolicy implements RetryPolicy {
*/
protected final RetryPolicy retryAwsClientExceptions;
+ /**
+ * Retry policy for all http 5xx errors not handled explicitly.
+ */
+ protected final RetryPolicy http5xxRetryPolicy;
+
/**
* Instantiate.
* @param conf configuration to read.
@@ -164,6 +169,13 @@ public S3ARetryPolicy(Configuration conf) {
// client connectivity: fixed retries without care for idempotency
connectivityFailure = baseExponentialRetry;
+ boolean retry5xxHttpErrors =
+ conf.getBoolean(RETRY_HTTP_5XX_ERRORS, DEFAULT_RETRY_HTTP_5XX_ERRORS);
+
+ http5xxRetryPolicy = retry5xxHttpErrors
+ ? retryAwsClientExceptions
+ : fail;
+
Map, RetryPolicy> policyMap =
createExceptionMap();
retryPolicy = retryByException(retryIdempotentCalls, policyMap);
@@ -228,15 +240,13 @@ protected Map, RetryPolicy> createExceptionMap() {
// throttled requests are can be retried, always
policyMap.put(AWSServiceThrottledException.class, throttlePolicy);
- // Status 5xx error code is an immediate failure
+ // Status 5xx error code has historically been treated as an immediate failure
// this is sign of a server-side problem, and while
// rare in AWS S3, it does happen on third party stores.
// (out of disk space, etc).
// by the time we get here, the aws sdk will have
// already retried.
- // there is specific handling for some 5XX codes (501, 503);
- // this is for everything else
- policyMap.put(AWSStatus500Exception.class, fail);
+ policyMap.put(AWSStatus500Exception.class, http5xxRetryPolicy);
// subclass of AWSServiceIOException whose cause is always S3Exception
policyMap.put(AWSS3IOException.class, retryIdempotentCalls);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java
index a11ed19670587..aed4442716963 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java
@@ -18,16 +18,27 @@
package org.apache.hadoop.fs.s3a;
+import java.io.File;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.time.Duration;
import java.util.Map;
import java.util.Optional;
+import java.util.concurrent.CancellationException;
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.core.sync.RequestBody;
+import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
+import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse;
+import software.amazon.awssdk.services.s3.model.PutObjectRequest;
+import software.amazon.awssdk.services.s3.model.UploadPartRequest;
+import software.amazon.awssdk.services.s3.model.UploadPartResponse;
+import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -78,6 +89,55 @@ public interface S3AStore extends IOStatisticsSource, ClientManager {
ClientManager clientManager();
+ /**
+ * Increment read operations.
+ */
+ void incrementReadOperations();
+
+ /**
+ * Increment the write operation counter.
+ * This is somewhat inaccurate, as it appears to be invoked more
+ * often than needed in progress callbacks.
+ */
+ void incrementWriteOperations();
+
+ /**
+ * At the start of a put/multipart upload operation, update the
+ * relevant counters.
+ *
+ * @param bytes bytes in the request.
+ */
+ void incrementPutStartStatistics(long bytes);
+
+ /**
+ * At the end of a put/multipart upload operation, update the
+ * relevant counters and gauges.
+ *
+ * @param success did the operation succeed?
+ * @param bytes bytes in the request.
+ */
+ void incrementPutCompletedStatistics(boolean success, long bytes);
+
+ /**
+ * Callback for use in progress callbacks from put/multipart upload events.
+ * Increments those statistics which are expected to be updated during
+ * the ongoing upload operation.
+ * @param key key to file that is being written (for logging)
+ * @param bytes bytes successfully uploaded.
+ */
+ void incrementPutProgressStatistics(String key, long bytes);
+
+ /**
+ * Given a possibly null duration tracker factory, return a non-null
+ * one for use in tracking durations -either that or the FS tracker
+ * itself.
+ *
+ * @param factory factory.
+ * @return a non-null factory.
+ */
+ DurationTrackerFactory nonNullDurationTrackerFactory(
+ DurationTrackerFactory factory);
+
/**
* Perform a bulk object delete operation against S3.
* Increments the {@code OBJECT_DELETE_REQUESTS} and write
@@ -133,4 +193,75 @@ Map.Entry deleteObjects(DeleteObjectsRequest de
Map.Entry> deleteObject(
DeleteObjectRequest request) throws SdkException;
+ /**
+ * Upload part of a multi-partition file.
+ * Increments the write and put counters.
+ * Important: this call does not close any input stream in the body.
+ *
+ * Retry Policy: none.
+ * @param durationTrackerFactory duration tracker factory for operation
+ * @param request the upload part request.
+ * @param body the request body.
+ * @return the result of the operation.
+ * @throws AwsServiceException on problems
+ * @throws UncheckedIOException failure to instantiate the s3 client
+ */
+ @Retries.OnceRaw
+ UploadPartResponse uploadPart(
+ UploadPartRequest request,
+ RequestBody body,
+ DurationTrackerFactory durationTrackerFactory)
+ throws AwsServiceException, UncheckedIOException;
+
+ /**
+ * Start a transfer-manager managed async PUT of an object,
+ * incrementing the put requests and put bytes
+ * counters.
+ *
+ * It does not update the other counters,
+ * as existing code does that as progress callbacks come in.
+ * Byte length is calculated from the file length, or, if there is no
+ * file, from the content length of the header.
+ *
+ * Because the operation is async, any stream supplied in the request
+ * must reference data (files, buffers) which stay valid until the upload
+ * completes.
+ * Retry policy: N/A: the transfer manager is performing the upload.
+ * Auditing: must be inside an audit span.
+ * @param putObjectRequest the request
+ * @param file the file to be uploaded
+ * @param listener the progress listener for the request
+ * @return the upload initiated
+ * @throws IOException if transfer manager creation failed.
+ */
+ @Retries.OnceRaw
+ UploadInfo putObject(
+ PutObjectRequest putObjectRequest,
+ File file,
+ ProgressableProgressListener listener) throws IOException;
+
+ /**
+ * Wait for an upload to complete.
+ * If the upload (or its result collection) failed, this is where
+ * the failure is raised as an AWS exception.
+ * Calls {@link S3AStore#incrementPutCompletedStatistics(boolean, long)}
+ * to update the statistics.
+ * @param key destination key
+ * @param uploadInfo upload to wait for
+ * @return the upload result
+ * @throws IOException IO failure
+ * @throws CancellationException if the wait() was cancelled
+ */
+ @Retries.OnceTranslated
+ CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
+ throws IOException;
+
+ /**
+ * Complete a multipart upload.
+ * @param request request
+ * @return the response
+ */
+ @Retries.OnceRaw
+ CompleteMultipartUploadResponse completeMultipartUpload(
+ CompleteMultipartUploadRequest request);
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 2335f09c51037..1b858f5c1e8a7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -24,6 +24,7 @@
import software.amazon.awssdk.core.exception.ApiCallTimeoutException;
import software.amazon.awssdk.core.exception.SdkException;
import software.amazon.awssdk.core.retry.RetryUtils;
+import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.S3Exception;
import software.amazon.awssdk.services.s3.model.S3Object;
@@ -297,7 +298,7 @@ public static IOException translateException(@Nullable String operation,
case SC_405_METHOD_NOT_ALLOWED:
case SC_415_UNSUPPORTED_MEDIA_TYPE:
case SC_501_NOT_IMPLEMENTED:
- ioe = new AWSUnsupportedFeatureException(message, s3Exception);
+ ioe = new AWSUnsupportedFeatureException(message, ase);
break;
// precondition failure: the object is there, but the precondition
@@ -1176,6 +1177,19 @@ public static S3AFileStatus[] iteratorToStatuses(
return statuses;
}
+ /**
+ * Get the length of the PUT, verifying that the length is known.
+ * @param putObjectRequest a request bound to a file or a stream.
+ * @return the request length
+ * @throws IllegalArgumentException if the length is negative
+ */
+ public static long getPutRequestLength(PutObjectRequest putObjectRequest) {
+ long len = putObjectRequest.contentLength();
+
+ Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length");
+ return len;
+ }
+
/**
* An interface for use in lambda-expressions working with
* directory tree listings.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index 3bee1008ce42b..0bcdb29330d56 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -65,6 +65,38 @@ public enum Statistic {
"GET request.",
TYPE_DURATION),
+ /* Http error responses */
+ HTTP_RESPONSE_400(
+ StoreStatisticNames.HTTP_RESPONSE_400,
+ "400 response.",
+ TYPE_COUNTER),
+
+ HTTP_RESPONSE_429(
+ StoreStatisticNames.HTTP_RESPONSE_429,
+ "429 response.",
+ TYPE_COUNTER),
+
+ HTTP_RESPONSE_4XX(
+ StoreStatisticNames.HTTP_RESPONSE_4XX,
+ "4XX response.",
+ TYPE_COUNTER),
+
+ HTTP_RESPONSE_500(
+ StoreStatisticNames.HTTP_RESPONSE_500,
+ "500 response.",
+ TYPE_COUNTER),
+
+ HTTP_RESPONSE_503(
+ StoreStatisticNames.HTTP_RESPONSE_503,
+ "503 response.",
+ TYPE_COUNTER),
+
+ HTTP_RESPONSE_5XX(
+ StoreStatisticNames.HTTP_RESPONSE_5XX,
+ "5XX response.",
+ TYPE_COUNTER),
+
+
/* FileSystem Level statistics */
FILESYSTEM_INITIALIZATION(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 3bbe000bf5b6e..b7387fc12e140 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -21,9 +21,11 @@
import javax.annotation.Nullable;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
@@ -233,14 +235,12 @@ private void deactivateAuditSpan() {
* @param destKey destination key
* @param length size, if known. Use -1 for not known
* @param options options for the request
- * @param isFile is data to be uploaded a file
* @return the request
*/
@Retries.OnceRaw
public PutObjectRequest createPutObjectRequest(String destKey,
long length,
- final PutObjectOptions options,
- boolean isFile) {
+ final PutObjectOptions options) {
activateAuditSpan();
@@ -289,7 +289,7 @@ public String initiateMultiPartUpload(
/**
* Finalize a multipart PUT operation.
* This completes the upload, and, if that works, calls
- * {@link S3AFileSystem#finishedWrite(String, long, String, String, org.apache.hadoop.fs.s3a.impl.PutObjectOptions)}
+ * {@link WriteOperationHelperCallbacks#finishedWrite(String, long, PutObjectOptions)}
* to update the filesystem.
* Retry policy: retrying, translated.
* @param destKey destination of the commit
@@ -324,8 +324,7 @@ private CompleteMultipartUploadResponse finalizeMultipartUpload(
destKey, uploadId, partETags);
return writeOperationHelperCallbacks.completeMultipartUpload(requestBuilder.build());
});
- owner.finishedWrite(destKey, length, uploadResult.eTag(),
- uploadResult.versionId(),
+ writeOperationHelperCallbacks.finishedWrite(destKey, length,
putOptions);
return uploadResult;
}
@@ -404,11 +403,12 @@ public void abortMultipartUpload(String destKey, String uploadId,
/**
* Abort a multipart commit operation.
* @param upload upload to abort.
+ * @throws FileNotFoundException if the upload is unknown
* @throws IOException on problems.
*/
@Retries.RetryTranslated
public void abortMultipartUpload(MultipartUpload upload)
- throws IOException {
+ throws FileNotFoundException, IOException {
invoker.retry("Aborting multipart commit", upload.key(), true,
withinAuditSpan(getAuditSpan(),
() -> owner.abortMultipartUpload(upload)));
@@ -508,20 +508,19 @@ public String toString() {
* file, from the content length of the header.
* @param putObjectRequest the request
* @param putOptions put object options
- * @param durationTrackerFactory factory for duration tracking
* @param uploadData data to be uploaded
- * @param isFile is data to be uploaded a file
- *
+ * @param durationTrackerFactory factory for duration tracking
* @return the upload initiated
* @throws IOException on problems
*/
@Retries.RetryTranslated
public PutObjectResponse putObject(PutObjectRequest putObjectRequest,
- PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
+ PutObjectOptions putOptions,
+ S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws IOException {
return retry("Writing Object", putObjectRequest.key(), true, withinAuditSpan(getAuditSpan(),
- () -> owner.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile,
+ () -> owner.putObjectDirect(putObjectRequest, putOptions, uploadData,
durationTrackerFactory)));
}
@@ -578,7 +577,6 @@ public CompleteMultipartUploadResponse commitUpload(
/**
* Upload part of a multi-partition file.
- * @param request request
* @param durationTrackerFactory duration tracker factory for operation
* @param request the upload part request.
* @param body the request body.
@@ -594,7 +592,9 @@ public UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body
request.key(),
true,
withinAuditSpan(getAuditSpan(),
- () -> owner.uploadPart(request, body, durationTrackerFactory)));
+ () -> writeOperationHelperCallbacks.uploadPart(request,
+ body,
+ durationTrackerFactory)));
}
/**
@@ -644,8 +644,44 @@ public interface WriteOperationHelperCallbacks {
* @param request Complete multi-part upload request
* @return completeMultipartUploadResult
*/
- CompleteMultipartUploadResponse completeMultipartUpload(CompleteMultipartUploadRequest request);
+ @Retries.OnceRaw
+ CompleteMultipartUploadResponse completeMultipartUpload(
+ CompleteMultipartUploadRequest request);
+ /**
+ * Upload part of a multi-partition file.
+ * Increments the write and put counters.
+ * Important: this call does not close any input stream in the body.
+ *
+ * Retry Policy: none.
+ * @param durationTrackerFactory duration tracker factory for operation
+ * @param request the upload part request.
+ * @param body the request body.
+ * @return the result of the operation.
+ * @throws AwsServiceException on problems
+ * @throws UncheckedIOException failure to instantiate the s3 client
+ */
+ @Retries.OnceRaw
+ UploadPartResponse uploadPart(
+ UploadPartRequest request,
+ RequestBody body,
+ DurationTrackerFactory durationTrackerFactory)
+ throws AwsServiceException, UncheckedIOException;
+
+ /**
+ * Perform post-write actions.
+ *
+ * This operation MUST be called after any PUT/multipart PUT completes
+ * successfully.
+ * @param key key written to
+ * @param length total length of file written
+ * @param putOptions put object options
+ */
+ @Retries.RetryExceptionsSwallowed
+ void finishedWrite(
+ String key,
+ long length,
+ PutObjectOptions putOptions);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
index 5ad9c9f9b6482..68709c40f45ca 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java
@@ -74,13 +74,11 @@ T retry(String action,
* @param destKey destination key
* @param length size, if known. Use -1 for not known
* @param options options for the request
- * @param isFile is data to be uploaded a file
* @return the request
*/
PutObjectRequest createPutObjectRequest(String destKey,
long length,
- @Nullable PutObjectOptions options,
- boolean isFile);
+ @Nullable PutObjectOptions options);
/**
* Callback on a successful write.
@@ -148,6 +146,7 @@ void abortMultipartUpload(String destKey, String uploadId,
/**
* Abort a multipart commit operation.
* @param upload upload to abort.
+ * @throws FileNotFoundException if the upload is unknown
* @throws IOException on problems.
*/
@Retries.RetryTranslated
@@ -208,15 +207,15 @@ UploadPartRequest.Builder newUploadPartRequestBuilder(
* file, from the content length of the header.
* @param putObjectRequest the request
* @param putOptions put object options
- * @param durationTrackerFactory factory for duration tracking
* @param uploadData data to be uploaded
- * @param isFile is data to be uploaded a file
+ * @param durationTrackerFactory factory for duration tracking
* @return the upload initiated
* @throws IOException on problems
*/
@Retries.RetryTranslated
PutObjectResponse putObject(PutObjectRequest putObjectRequest,
- PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
+ PutObjectOptions putOptions,
+ S3ADataBlocks.BlockUploadData uploadData,
DurationTrackerFactory durationTrackerFactory)
throws IOException;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java
index 97ee92a20b1e3..c5ce1a2c9e4b8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java
@@ -26,6 +26,8 @@
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.service.AbstractService;
+import static java.util.Objects.requireNonNull;
+
/**
* This is a long-lived service which is created in S3A FS initialize
* (make it fast!) which provides context for tracking operations made to S3.
@@ -85,7 +87,7 @@ protected AbstractOperationAuditor(final String name) {
@Override
public void init(final OperationAuditorOptions opts) {
this.options = opts;
- this.iostatistics = opts.getIoStatisticsStore();
+ this.iostatistics = requireNonNull(opts.getIoStatisticsStore());
init(opts.getConfiguration());
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
index 3a2d9d7f823ee..5f93454795a94 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
@@ -23,12 +23,14 @@
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
+import java.util.Optional;
import software.amazon.awssdk.awscore.AwsExecutionAttribute;
import software.amazon.awssdk.core.SdkRequest;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.http.SdkHttpRequest;
+import software.amazon.awssdk.http.SdkHttpResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
@@ -66,6 +68,7 @@
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION;
import static org.apache.hadoop.fs.s3a.commit.CommitUtils.extractJobID;
import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.HEADER_REFERRER;
+import static org.apache.hadoop.fs.s3a.statistics.impl.StatisticsFromAwsSdkImpl.mapErrorStatusCodeToStatisticName;
/**
* The LoggingAuditor logs operations at DEBUG (in SDK Request) and
@@ -444,6 +447,22 @@ public String toString() {
HttpReferrerAuditHeader getReferrer() {
return referrer;
}
+
+ /**
+ * Execution failure: extract an error code and if this maps to
+ * a statistic name, update that counter.
+ */
+ @Override
+ public void onExecutionFailure(final Context.FailedExecution context,
+ final ExecutionAttributes executionAttributes) {
+ final Optional response = context.httpResponse();
+ int sc = response.map(SdkHttpResponse::statusCode).orElse(0);
+ String stat = mapErrorStatusCodeToStatisticName(sc);
+ if (stat != null) {
+ LOG.debug("Incrementing error statistic {}", stat);
+ getIOStatistics().incrementCounter(stat);
+ }
+ }
}
/**
@@ -522,4 +541,5 @@ public void beforeExecution(Context.BeforeExecution context,
super.beforeExecution(context, executionAttributes);
}
}
+
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
index d1943fa47773f..f33d94ce84fef 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
@@ -21,8 +21,6 @@
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@@ -62,6 +60,7 @@
import org.apache.hadoop.fs.s3a.impl.HeaderProcessing;
import org.apache.hadoop.fs.s3a.impl.InternalConstants;
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
+import org.apache.hadoop.fs.s3a.impl.UploadContentProviders;
import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.IOStatistics;
@@ -81,6 +80,7 @@
import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS;
+import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator;
@@ -88,11 +88,17 @@
* The implementation of the various actions a committer needs.
* This doesn't implement the protocol/binding to a specific execution engine,
* just the operations needed to to build one.
- *
+ *
* When invoking FS operations, it assumes that the underlying FS is
* handling retries and exception translation: it does not attempt to
* duplicate that work.
- *
+ *
+ * It does use {@link UploadContentProviders} to create a content provider
+ * for the request body which is capable of restarting a failed upload.
+ * This is not currently provided by the default AWS SDK implementation
+ * of {@code RequestBody#fromFile()}.
+ *
+ * See HADOOP-19221 for details.
*/
public class CommitOperations extends AbstractStoreOperation
implements IOStatisticsSource {
@@ -553,7 +559,6 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
commitData.setText(partition != null ? "partition: " + partition : "");
commitData.setLength(length);
- long offset = 0;
long numParts = (length / uploadPartSize +
((length % uploadPartSize) > 0 ? 1 : 0));
// always write one part, even if it is just an empty one
@@ -570,31 +575,19 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
numParts, length));
}
- List parts = new ArrayList<>((int) numParts);
-
+ final int partCount = (int) numParts;
LOG.debug("File size is {}, number of parts to upload = {}",
- length, numParts);
+ length, partCount);
// Open the file to upload.
- try (InputStream fileStream = Files.newInputStream(localFile.toPath())) {
- for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
- progress.progress();
- long size = Math.min(length - offset, uploadPartSize);
- UploadPartRequest part = writeOperations.newUploadPartRequestBuilder(
- destKey,
- uploadId,
- partNumber,
- size).build();
- // Read from the file input stream at current position.
- RequestBody body = RequestBody.fromInputStream(fileStream, size);
- UploadPartResponse response = writeOperations.uploadPart(part, body, statistics);
- offset += uploadPartSize;
- parts.add(CompletedPart.builder()
- .partNumber(partNumber)
- .eTag(response.eTag())
- .build());
- }
- }
+ List parts = uploadFileData(
+ uploadId,
+ localFile,
+ destKey,
+ progress,
+ length,
+ partCount,
+ uploadPartSize);
commitData.bindCommitData(parts);
statistics.commitUploaded(length);
@@ -617,6 +610,55 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile,
}
}
+ /**
+ * Upload file data using content provider API.
+ * This a rewrite of the previous code to address HADOOP-19221;
+ * our own {@link UploadContentProviders} file content provider
+ * is used to upload each part of the file.
+ * @param uploadId upload ID
+ * @param localFile locally staged file
+ * @param destKey destination path
+ * @param progress progress callback
+ * @param length file length
+ * @param numParts number of parts to upload
+ * @param uploadPartSize max size of a part
+ * @return the ordered list of parts
+ * @throws IOException IO failure
+ */
+ private List uploadFileData(
+ final String uploadId,
+ final File localFile,
+ final String destKey,
+ final Progressable progress,
+ final long length,
+ final int numParts,
+ final long uploadPartSize) throws IOException {
+ List parts = new ArrayList<>(numParts);
+ long offset = 0;
+ for (int partNumber = 1; partNumber <= numParts; partNumber++) {
+ progress.progress();
+ int size = (int)Math.min(length - offset, uploadPartSize);
+ UploadPartRequest part = writeOperations.newUploadPartRequestBuilder(
+ destKey,
+ uploadId,
+ partNumber,
+ size).build();
+ // Create a file content provider starting at the current offset.
+ RequestBody body = RequestBody.fromContentProvider(
+ UploadContentProviders.fileContentProvider(localFile, offset, size),
+ size,
+ CONTENT_TYPE_OCTET_STREAM);
+ UploadPartResponse response = writeOperations.uploadPart(part, body, statistics);
+ offset += uploadPartSize;
+ parts.add(CompletedPart.builder()
+ .partNumber(partNumber)
+ .eTag(response.eTag())
+ .build());
+ }
+ return parts;
+ }
+
+
/**
* Add the filesystem statistics to the map; overwriting anything
* with the same name.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java
index 0ab3cee5201e7..1f6c9123bae62 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/S3MagicCommitTracker.java
@@ -18,9 +18,7 @@
package org.apache.hadoop.fs.s3a.commit.magic;
-import java.io.ByteArrayInputStream;
import java.io.IOException;
-import java.io.InputStream;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -81,8 +79,8 @@ public boolean aboutToComplete(String uploadId,
PutObjectRequest originalDestPut = getWriter().createPutObjectRequest(
getOriginalDestKey(),
0,
- new PutObjectOptions(true, null, headers), false);
- upload(originalDestPut, new ByteArrayInputStream(EMPTY));
+ new PutObjectOptions(true, null, headers));
+ upload(originalDestPut, EMPTY);
// build the commit summary
SinglePendingCommit commitData = new SinglePendingCommit();
@@ -105,8 +103,8 @@ public boolean aboutToComplete(String uploadId,
getPath(), getPendingPartKey(), commitData);
PutObjectRequest put = getWriter().createPutObjectRequest(
getPendingPartKey(),
- bytes.length, null, false);
- upload(put, new ByteArrayInputStream(bytes));
+ bytes.length, null);
+ upload(put, bytes);
return false;
}
@@ -117,9 +115,9 @@ public boolean aboutToComplete(String uploadId,
* @throws IOException on problems
*/
@Retries.RetryTranslated
- private void upload(PutObjectRequest request, InputStream inputStream) throws IOException {
+ private void upload(PutObjectRequest request, byte[] bytes) throws IOException {
trackDurationOfInvocation(getTrackerStatistics(), COMMITTER_MAGIC_MARKER_PUT.getSymbol(),
() -> getWriter().putObject(request, PutObjectOptions.keepingDirs(),
- new S3ADataBlocks.BlockUploadData(inputStream), false, null));
+ new S3ADataBlocks.BlockUploadData(bytes, null), null));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java
index 84770861cc489..7fadac8623d50 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java
@@ -20,6 +20,7 @@
import java.io.Closeable;
import java.io.IOException;
+import java.io.UncheckedIOException;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.S3Client;
@@ -39,10 +40,34 @@ public interface ClientManager extends Closeable {
S3TransferManager getOrCreateTransferManager()
throws IOException;
+ /**
+ * Get the S3Client, raising a failure to create as an IOException.
+ * @return the S3 client
+ * @throws IOException failure to create the client.
+ */
S3Client getOrCreateS3Client() throws IOException;
+ /**
+ * Get the S3Client, raising a failure to create as an UncheckedIOException.
+ * @return the S3 client
+ * @throws UncheckedIOException failure to create the client.
+ */
+ S3Client getOrCreateS3ClientUnchecked() throws UncheckedIOException;
+
+ /**
+ * Get the Async S3Client,raising a failure to create as an IOException.
+ * @return the Async S3 client
+ * @throws IOException failure to create the client.
+ */
S3AsyncClient getOrCreateAsyncClient() throws IOException;
+ /**
+ * Get the AsyncS3Client, raising a failure to create as an UncheckedIOException.
+ * @return the S3 client
+ * @throws UncheckedIOException failure to create the client.
+ */
+ S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException;
+
/**
* Close operation is required to not raise exceptions.
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java
index ff6748e66d1d2..4b2fc1c599b26 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.fs.s3a.impl;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
@@ -147,12 +148,34 @@ public synchronized S3Client getOrCreateS3Client() throws IOException {
return s3Client.eval();
}
+ /**
+ * Get the S3Client, raising a failure to create as an UncheckedIOException.
+ * @return the S3 client
+ * @throws UncheckedIOException failure to create the client.
+ */
+ @Override
+ public synchronized S3Client getOrCreateS3ClientUnchecked() throws UncheckedIOException {
+ checkNotClosed();
+ return s3Client.get();
+ }
+
@Override
public synchronized S3AsyncClient getOrCreateAsyncClient() throws IOException {
checkNotClosed();
return s3AsyncClient.eval();
}
+ /**
+ * Get the AsyncS3Client, raising a failure to create as an UncheckedIOException.
+ * @return the S3 client
+ * @throws UncheckedIOException failure to create the client.
+ */
+ @Override
+ public synchronized S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException {
+ checkNotClosed();
+ return s3Client.get();
+ }
+
@Override
public synchronized S3TransferManager getOrCreateTransferManager() throws IOException {
checkNotClosed();
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
index 1d12a41008b6b..ddbcad6dc047f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
@@ -292,4 +292,11 @@ private InternalConstants() {
*/
public static final String AUTH_SCHEME_AWS_SIGV_4 = "aws.auth#sigv4";
+
+ /**
+ * Progress logge name; fairly noisy.
+ */
+ public static final String UPLOAD_PROGRESS_LOG_NAME =
+ "org.apache.hadoop.fs.s3a.S3AFileSystem.Progress";
+
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java
index f3f9fb61e434d..391e11d956212 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java
@@ -20,10 +20,72 @@
/**
* Enum for progress listener events.
+ * Some are used in the {@code S3ABlockOutputStream}
+ * class to manage progress; others are to assist
+ * testing.
*/
public enum ProgressListenerEvent {
+
+ /**
+ * Stream has been closed.
+ */
+ CLOSE_EVENT,
+
+ /** PUT operation completed successfully. */
+ PUT_COMPLETED_EVENT,
+
+ /** PUT operation was interrupted. */
+ PUT_INTERRUPTED_EVENT,
+
+ /** PUT operation was interrupted. */
+ PUT_FAILED_EVENT,
+
+ /** A PUT operation was started. */
+ PUT_STARTED_EVENT,
+
+ /** Bytes were transferred. */
REQUEST_BYTE_TRANSFER_EVENT,
+
+ /**
+ * A multipart upload was initiated.
+ */
+ TRANSFER_MULTIPART_INITIATED_EVENT,
+
+ /**
+ * A multipart upload was aborted.
+ */
+ TRANSFER_MULTIPART_ABORTED_EVENT,
+
+ /**
+ * A multipart upload was successfully.
+ */
+ TRANSFER_MULTIPART_COMPLETED_EVENT,
+
+ /**
+ * An upload of a part of a multipart upload was started.
+ */
TRANSFER_PART_STARTED_EVENT,
+
+ /**
+ * An upload of a part of a multipart upload was completed.
+ * This does not indicate the upload was successful.
+ */
TRANSFER_PART_COMPLETED_EVENT,
- TRANSFER_PART_FAILED_EVENT;
+
+ /**
+ * An upload of a part of a multipart upload was completed
+ * successfully.
+ */
+ TRANSFER_PART_SUCCESS_EVENT,
+
+ /**
+ * An upload of a part of a multipart upload was abported.
+ */
+ TRANSFER_PART_ABORTED_EVENT,
+
+ /**
+ * An upload of a part of a multipart upload failed.
+ */
+ TRANSFER_PART_FAILED_EVENT,
+
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java
index ca629f16be992..a7565fe046e3e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a.impl;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.S3AStore;
@@ -50,6 +51,13 @@ public class S3AStoreBuilder {
private AuditSpanSource auditSpanSource;
+ /**
+ * The original file system statistics: fairly minimal but broadly
+ * collected so it is important to pick up.
+ * This may be null.
+ */
+ private FileSystem.Statistics fsStatistics;
+
public S3AStoreBuilder withStoreContextFactory(
final StoreContextFactory storeContextFactoryValue) {
this.storeContextFactory = storeContextFactoryValue;
@@ -104,6 +112,11 @@ public S3AStoreBuilder withAuditSpanSource(
return this;
}
+ public S3AStoreBuilder withFsStatistics(final FileSystem.Statistics value) {
+ this.fsStatistics = value;
+ return this;
+ }
+
public S3AStore build() {
return new S3AStoreImpl(storeContextFactory,
clientManager,
@@ -113,6 +126,7 @@ public S3AStore build() {
storageStatistics,
readRateLimiter,
writeRateLimiter,
- auditSpanSource);
+ auditSpanSource,
+ fsStatistics);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java
index d0871e7af2388..385023598c559 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java
@@ -18,34 +18,49 @@
package org.apache.hadoop.fs.s3a.impl;
+import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletionException;
import javax.annotation.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.core.sync.RequestBody;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
+import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse;
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
+import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.S3Error;
+import software.amazon.awssdk.services.s3.model.UploadPartRequest;
+import software.amazon.awssdk.services.s3.model.UploadPartResponse;
import software.amazon.awssdk.transfer.s3.S3TransferManager;
+import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
+import software.amazon.awssdk.transfer.s3.model.FileUpload;
+import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.ProgressableProgressListener;
import org.apache.hadoop.fs.s3a.Retries;
import org.apache.hadoop.fs.s3a.S3AInstrumentation;
import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
import org.apache.hadoop.fs.s3a.S3AStore;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.UploadInfo;
import org.apache.hadoop.fs.s3a.api.RequestFactory;
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
@@ -57,11 +72,18 @@
import org.apache.hadoop.util.functional.Tuples;
import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.s3a.S3AUtils.extractException;
+import static org.apache.hadoop.fs.s3a.S3AUtils.getPutRequestLength;
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS;
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_PART_PUT;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS;
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST;
+import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_BYTES;
+import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_BYTES_PENDING;
+import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
+import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS_COMPLETED;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RATE_LIMITED;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
@@ -69,6 +91,7 @@
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
import static org.apache.hadoop.util.Preconditions.checkArgument;
/**
@@ -80,6 +103,12 @@ public class S3AStoreImpl implements S3AStore {
private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class);
+ /**
+ * Progress logger; fairly noisy.
+ */
+ private static final Logger PROGRESS =
+ LoggerFactory.getLogger(InternalConstants.UPLOAD_PROGRESS_LOG_NAME);
+
/** Factory to create store contexts. */
private final StoreContextFactory storeContextFactory;
@@ -119,6 +148,13 @@ public class S3AStoreImpl implements S3AStore {
/** Audit span source. */
private final AuditSpanSource auditSpanSource;
+ /**
+ * The original file system statistics: fairly minimal but broadly
+ * collected so it is important to pick up.
+ * This may be null.
+ */
+ private final FileSystem.Statistics fsStatistics;
+
/** Constructor to create S3A store. */
S3AStoreImpl(StoreContextFactory storeContextFactory,
ClientManager clientManager,
@@ -128,7 +164,8 @@ public class S3AStoreImpl implements S3AStore {
S3AStorageStatistics storageStatistics,
RateLimiting readRateLimiter,
RateLimiting writeRateLimiter,
- AuditSpanSource auditSpanSource) {
+ AuditSpanSource auditSpanSource,
+ @Nullable FileSystem.Statistics fsStatistics) {
this.storeContextFactory = requireNonNull(storeContextFactory);
this.clientManager = requireNonNull(clientManager);
this.durationTrackerFactory = requireNonNull(durationTrackerFactory);
@@ -139,6 +176,7 @@ public class S3AStoreImpl implements S3AStore {
this.writeRateLimiter = requireNonNull(writeRateLimiter);
this.auditSpanSource = requireNonNull(auditSpanSource);
this.storeContext = requireNonNull(storeContextFactory.createStoreContext());
+ this.fsStatistics = fsStatistics;
this.invoker = storeContext.getInvoker();
this.bucket = storeContext.getBucket();
this.requestFactory = storeContext.getRequestFactory();
@@ -178,10 +216,10 @@ public StoreContext getStoreContext() {
/**
* Get the S3 client.
* @return the S3 client.
- * @throws IOException on any failure to create the client.
+ * @throws UncheckedIOException on any failure to create the client.
*/
- private S3Client getS3Client() throws IOException {
- return clientManager.getOrCreateS3Client();
+ private S3Client getS3Client() throws UncheckedIOException {
+ return clientManager.getOrCreateS3ClientUnchecked();
}
@Override
@@ -199,6 +237,16 @@ public S3AsyncClient getOrCreateAsyncClient() throws IOException {
return clientManager.getOrCreateAsyncClient();
}
+ @Override
+ public S3Client getOrCreateS3ClientUnchecked() throws UncheckedIOException {
+ return clientManager.getOrCreateS3ClientUnchecked();
+ }
+
+ @Override
+ public S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException {
+ return clientManager.getOrCreateAsyncS3ClientUnchecked();
+ }
+
@Override
public DurationTrackerFactory getDurationTrackerFactory() {
return durationTrackerFactory;
@@ -305,6 +353,105 @@ public IOStatistics getIOStatistics() {
return instrumentation.getIOStatistics();
}
+ /**
+ * Increment read operations.
+ */
+ @Override
+ public void incrementReadOperations() {
+ if (fsStatistics != null) {
+ fsStatistics.incrementReadOps(1);
+ }
+ }
+
+ /**
+ * Increment the write operation counter.
+ * This is somewhat inaccurate, as it appears to be invoked more
+ * often than needed in progress callbacks.
+ */
+ @Override
+ public void incrementWriteOperations() {
+ if (fsStatistics != null) {
+ fsStatistics.incrementWriteOps(1);
+ }
+ }
+
+
+ /**
+ * Increment the bytes written statistic.
+ * @param bytes number of bytes written.
+ */
+ private void incrementBytesWritten(final long bytes) {
+ if (fsStatistics != null) {
+ fsStatistics.incrementBytesWritten(bytes);
+ }
+ }
+
+ /**
+ * At the start of a put/multipart upload operation, update the
+ * relevant counters.
+ *
+ * @param bytes bytes in the request.
+ */
+ @Override
+ public void incrementPutStartStatistics(long bytes) {
+ LOG.debug("PUT start {} bytes", bytes);
+ incrementWriteOperations();
+ incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
+ if (bytes > 0) {
+ incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
+ }
+ }
+
+ /**
+ * At the end of a put/multipart upload operation, update the
+ * relevant counters and gauges.
+ *
+ * @param success did the operation succeed?
+ * @param bytes bytes in the request.
+ */
+ @Override
+ public void incrementPutCompletedStatistics(boolean success, long bytes) {
+ LOG.debug("PUT completed success={}; {} bytes", success, bytes);
+ if (bytes > 0) {
+ incrementStatistic(OBJECT_PUT_BYTES, bytes);
+ decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes);
+ }
+ incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED);
+ decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1);
+ }
+
+ /**
+ * Callback for use in progress callbacks from put/multipart upload events.
+ * Increments those statistics which are expected to be updated during
+ * the ongoing upload operation.
+ * @param key key to file that is being written (for logging)
+ * @param bytes bytes successfully uploaded.
+ */
+ @Override
+ public void incrementPutProgressStatistics(String key, long bytes) {
+ PROGRESS.debug("PUT {}: {} bytes", key, bytes);
+ incrementWriteOperations();
+ if (bytes > 0) {
+ incrementBytesWritten(bytes);
+ }
+ }
+
+ /**
+ * Given a possibly null duration tracker factory, return a non-null
+ * one for use in tracking durations -either that or the FS tracker
+ * itself.
+ *
+ * @param factory factory.
+ * @return a non-null factory.
+ */
+ @Override
+ public DurationTrackerFactory nonNullDurationTrackerFactory(
+ DurationTrackerFactory factory) {
+ return factory != null
+ ? factory
+ : getDurationTrackerFactory();
+ }
+
/**
* Start an operation; this informs the audit service of the event
* and then sets it as the active span.
@@ -388,7 +535,6 @@ public Map.Entry deleteObjects(
return Tuples.pair(d.asDuration(), response);
} catch (IOException e) {
- // this is part of the retry signature, nothing else.
// convert to unchecked.
throw new UncheckedIOException(e);
}
@@ -430,10 +576,125 @@ public Map.Entry> deleteObject(
d.close();
return Tuples.pair(d.asDuration(), Optional.empty());
} catch (IOException e) {
- // this is part of the retry signature, nothing else.
// convert to unchecked.
throw new UncheckedIOException(e);
}
}
+ /**
+ * Upload part of a multi-partition file.
+ * Increments the write and put counters.
+ * Important: this call does not close any input stream in the body.
+ *
+ * Retry Policy: none.
+ * @param trackerFactory duration tracker factory for operation
+ * @param request the upload part request.
+ * @param body the request body.
+ * @return the result of the operation.
+ * @throws AwsServiceException on problems
+ * @throws UncheckedIOException failure to instantiate the s3 client
+ */
+ @Override
+ @Retries.OnceRaw
+ public UploadPartResponse uploadPart(
+ final UploadPartRequest request,
+ final RequestBody body,
+ @Nullable final DurationTrackerFactory trackerFactory)
+ throws AwsServiceException, UncheckedIOException {
+ long len = request.contentLength();
+ incrementPutStartStatistics(len);
+ try {
+ UploadPartResponse uploadPartResponse = trackDurationOfSupplier(
+ nonNullDurationTrackerFactory(trackerFactory),
+ MULTIPART_UPLOAD_PART_PUT.getSymbol(), () ->
+ getS3Client().uploadPart(request, body));
+ incrementPutCompletedStatistics(true, len);
+ return uploadPartResponse;
+ } catch (AwsServiceException e) {
+ incrementPutCompletedStatistics(false, len);
+ throw e;
+ }
+ }
+
+ /**
+ * Start a transfer-manager managed async PUT of an object,
+ * incrementing the put requests and put bytes
+ * counters.
+ *
+ * It does not update the other counters,
+ * as existing code does that as progress callbacks come in.
+ * Byte length is calculated from the file length, or, if there is no
+ * file, from the content length of the header.
+ *
+ * Because the operation is async, any stream supplied in the request
+ * must reference data (files, buffers) which stay valid until the upload
+ * completes.
+ * Retry policy: N/A: the transfer manager is performing the upload.
+ * Auditing: must be inside an audit span.
+ * @param putObjectRequest the request
+ * @param file the file to be uploaded
+ * @param listener the progress listener for the request
+ * @return the upload initiated
+ * @throws IOException if transfer manager creation failed.
+ */
+ @Override
+ @Retries.OnceRaw
+ public UploadInfo putObject(
+ PutObjectRequest putObjectRequest,
+ File file,
+ ProgressableProgressListener listener) throws IOException {
+ long len = getPutRequestLength(putObjectRequest);
+ LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key());
+ incrementPutStartStatistics(len);
+
+ FileUpload upload = getOrCreateTransferManager().uploadFile(
+ UploadFileRequest.builder()
+ .putObjectRequest(putObjectRequest)
+ .source(file)
+ .addTransferListener(listener)
+ .build());
+
+ return new UploadInfo(upload, len);
+ }
+
+ /**
+ * Wait for an upload to complete.
+ * If the upload (or its result collection) failed, this is where
+ * the failure is raised as an AWS exception.
+ * Calls {@link S3AStore#incrementPutCompletedStatistics(boolean, long)}
+ * to update the statistics.
+ * @param key destination key
+ * @param uploadInfo upload to wait for
+ * @return the upload result
+ * @throws IOException IO failure
+ * @throws CancellationException if the wait() was cancelled
+ */
+ @Override
+ @Retries.OnceTranslated
+ public CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo)
+ throws IOException {
+ FileUpload upload = uploadInfo.getFileUpload();
+ try {
+ CompletedFileUpload result = upload.completionFuture().join();
+ incrementPutCompletedStatistics(true, uploadInfo.getLength());
+ return result;
+ } catch (CompletionException e) {
+ LOG.info("Interrupted: aborting upload");
+ incrementPutCompletedStatistics(false, uploadInfo.getLength());
+ throw extractException("upload", key, e);
+ }
+ }
+
+ /**
+ * Complete a multipart upload.
+ * @param request request
+ * @return the response
+ */
+ @Override
+ @Retries.OnceRaw
+ public CompleteMultipartUploadResponse completeMultipartUpload(
+ CompleteMultipartUploadRequest request) {
+ return getS3Client().completeMultipartUpload(request);
+ }
+
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java
new file mode 100644
index 0000000000000..5676e67cde2fa
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/UploadContentProviders.java
@@ -0,0 +1,549 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.http.ContentStreamProvider;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.store.ByteBufferInputStream;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+import static org.apache.hadoop.util.Preconditions.checkState;
+import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions;
+
+/**
+ * Implementations of {@code software.amazon.awssdk.http.ContentStreamProvider}.
+ *
+ * These are required to ensure that retry of multipart uploads are reliable,
+ * while also avoiding memory copy/consumption overhead.
+ *
+ * For these reasons the providers built in to the AWS SDK are not used.
+ *
+ * See HADOOP-19221 for details.
+ */
+public final class UploadContentProviders {
+
+ public static final Logger LOG = LoggerFactory.getLogger(UploadContentProviders.class);
+
+ private UploadContentProviders() {
+ }
+
+ /**
+ * Create a content provider from a file.
+ * @param file file to read.
+ * @param offset offset in file.
+ * @param size of data.
+ * @return the provider
+ * @throws IllegalArgumentException if the offset is negative.
+ */
+ public static BaseContentProvider fileContentProvider(
+ File file,
+ long offset,
+ final int size) {
+
+ return new FileWithOffsetContentProvider(file, offset, size);
+ }
+
+ /**
+ * Create a content provider from a file.
+ * @param file file to read.
+ * @param offset offset in file.
+ * @param size of data.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @return the provider
+ * @throws IllegalArgumentException if the offset is negative.
+ */
+ public static BaseContentProvider fileContentProvider(
+ File file,
+ long offset,
+ final int size,
+ final Supplier isOpen) {
+
+ return new FileWithOffsetContentProvider(file, offset, size, isOpen);
+ }
+
+ /**
+ * Create a content provider from a byte buffer.
+ * The buffer is not copied and MUST NOT be modified while
+ * the upload is taking place.
+ * @param byteBuffer buffer to read.
+ * @param size size of the data.
+ * @return the provider
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null
+ */
+ public static BaseContentProvider byteBufferContentProvider(
+ final ByteBuffer byteBuffer,
+ final int size) {
+ return new ByteBufferContentProvider(byteBuffer, size);
+ }
+
+ /**
+ * Create a content provider from a byte buffer.
+ * The buffer is not copied and MUST NOT be modified while
+ * the upload is taking place.
+ * @param byteBuffer buffer to read.
+ * @param size size of the data.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @return the provider
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null
+ */
+ public static BaseContentProvider byteBufferContentProvider(
+ final ByteBuffer byteBuffer,
+ final int size,
+ final @Nullable Supplier isOpen) {
+
+ return new ByteBufferContentProvider(byteBuffer, size, isOpen);
+ }
+
+ /**
+ * Create a content provider for all or part of a byte array.
+ * The buffer is not copied and MUST NOT be modified while
+ * the upload is taking place.
+ * @param bytes buffer to read.
+ * @param offset offset in buffer.
+ * @param size size of the data.
+ * @return the provider
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null.
+ */
+ public static BaseContentProvider byteArrayContentProvider(
+ final byte[] bytes, final int offset, final int size) {
+ return new ByteArrayContentProvider(bytes, offset, size);
+ }
+
+ /**
+ * Create a content provider for all or part of a byte array.
+ * The buffer is not copied and MUST NOT be modified while
+ * the upload is taking place.
+ * @param bytes buffer to read.
+ * @param offset offset in buffer.
+ * @param size size of the data.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @return the provider
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null.
+ */
+ public static BaseContentProvider byteArrayContentProvider(
+ final byte[] bytes,
+ final int offset,
+ final int size,
+ final @Nullable Supplier isOpen) {
+ return new ByteArrayContentProvider(bytes, offset, size, isOpen);
+ }
+
+ /**
+ * Create a content provider for all of a byte array.
+ * @param bytes buffer to read.
+ * @return the provider
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null.
+ */
+ public static BaseContentProvider byteArrayContentProvider(
+ final byte[] bytes) {
+ return byteArrayContentProvider(bytes, 0, bytes.length);
+ }
+
+ /**
+ * Create a content provider for all of a byte array.
+ * @param bytes buffer to read.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @return the provider
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null.
+ */
+ public static BaseContentProvider byteArrayContentProvider(
+ final byte[] bytes,
+ final @Nullable Supplier isOpen) {
+ return byteArrayContentProvider(bytes, 0, bytes.length, isOpen);
+ }
+
+ /**
+ * Base class for content providers; tracks the number of times a stream
+ * has been opened.
+ * @param type of stream created.
+ */
+ @VisibleForTesting
+ public static abstract class BaseContentProvider
+ implements ContentStreamProvider, Closeable {
+
+ /**
+ * Size of the data.
+ */
+ private final int size;
+
+ /**
+ * Probe to check if the stream is open.
+ * Invoked in {@link #checkOpen()}, which is itself
+ * invoked in {@link #newStream()}.
+ */
+ private final Supplier isOpen;
+
+ /**
+ * How many times has a stream been created?
+ */
+ private int streamCreationCount;
+
+ /**
+ * Current stream. Null if not opened yet.
+ * When {@link #newStream()} is called, this is set to the new value,
+ * Note: when the input stream itself is closed, this reference is not updated.
+ * Therefore this field not being null does not imply that the stream is open.
+ */
+ private T currentStream;
+
+ /**
+ * Constructor.
+ * @param size size of the data. Must be non-negative.
+ */
+ protected BaseContentProvider(int size) {
+ this(size, null);
+ }
+
+ /**
+ * Constructor.
+ * @param size size of the data. Must be non-negative.
+ * @param isOpen optional predicate to check if the stream is open.
+ */
+ protected BaseContentProvider(int size, @Nullable Supplier isOpen) {
+ checkArgument(size >= 0, "size is negative: %s", size);
+ this.size = size;
+ this.isOpen = isOpen;
+ }
+
+ /**
+ * Check if the stream is open.
+ * If the stream is not open, raise an exception
+ * @throws IllegalStateException if the stream is not open.
+ */
+ private void checkOpen() {
+ checkState(isOpen == null || isOpen.get(), "Stream is closed: %s", this);
+ }
+
+ /**
+ * Close the current stream.
+ */
+ @Override
+ public void close() {
+ cleanupWithLogger(LOG, getCurrentStream());
+ setCurrentStream(null);
+ }
+
+ /**
+ * Create a new stream.
+ *
+ * Calls {@link #close()} to ensure that any existing stream is closed,
+ * then {@link #checkOpen()} to verify that the data source is still open.
+ * Logs if this is a subsequent event as it implies a failure of the first attempt.
+ * @return the new stream
+ */
+ @Override
+ public final InputStream newStream() {
+ close();
+ checkOpen();
+ streamCreationCount++;
+ if (streamCreationCount > 1) {
+ LOG.info("Stream created more than once: {}", this);
+ }
+ return setCurrentStream(createNewStream());
+ }
+
+ /**
+ * Override point for subclasses to create their new streams.
+ * @return a stream
+ */
+ protected abstract T createNewStream();
+
+ /**
+ * How many times has a stream been created?
+ * @return stream creation count
+ */
+ public int getStreamCreationCount() {
+ return streamCreationCount;
+ }
+
+ /**
+ * Size as set by constructor parameter.
+ * @return size of the data
+ */
+ public int getSize() {
+ return size;
+ }
+
+ /**
+ * Current stream.
+ * When {@link #newStream()} is called, this is set to the new value,
+ * after closing the previous one.
+ *
+ * Why? The AWS SDK implementations do this, so there
+ * is an implication that it is needed to avoid keeping streams
+ * open on retries.
+ * @return the current stream, or null if none is open.
+ */
+ protected T getCurrentStream() {
+ return currentStream;
+ }
+
+ /**
+ * Set the current stream.
+ * @param stream the new stream
+ * @return the current stream.
+ */
+ protected T setCurrentStream(T stream) {
+ this.currentStream = stream;
+ return stream;
+ }
+
+ @Override
+ public String toString() {
+ return "BaseContentProvider{" +
+ "size=" + size +
+ ", streamCreationCount=" + streamCreationCount +
+ ", currentStream=" + currentStream +
+ '}';
+ }
+ }
+
+ /**
+ * Content provider for a file with an offset.
+ */
+ private static final class FileWithOffsetContentProvider
+ extends BaseContentProvider {
+
+ /**
+ * File to read.
+ */
+ private final File file;
+
+ /**
+ * Offset in file.
+ */
+ private final long offset;
+
+ /**
+ * Constructor.
+ * @param file file to read.
+ * @param offset offset in file.
+ * @param size of data.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @throws IllegalArgumentException if the offset is negative.
+ */
+ private FileWithOffsetContentProvider(
+ final File file,
+ final long offset,
+ final int size,
+ @Nullable final Supplier isOpen) {
+ super(size, isOpen);
+ this.file = requireNonNull(file);
+ checkArgument(offset >= 0, "Offset is negative: %s", offset);
+ this.offset = offset;
+ }
+
+ /**
+ * Constructor.
+ * @param file file to read.
+ * @param offset offset in file.
+ * @param size of data.
+ * @throws IllegalArgumentException if the offset is negative.
+ */
+ private FileWithOffsetContentProvider(final File file,
+ final long offset,
+ final int size) {
+ this(file, offset, size, null);
+ }
+
+ /**
+ * Create a new stream.
+ * @return a stream at the start of the offset in the file
+ * @throws UncheckedIOException on IO failure.
+ */
+ @Override
+ protected BufferedInputStream createNewStream() throws UncheckedIOException {
+ // create the stream, seek to the offset.
+ final FileInputStream fis = uncheckIOExceptions(() -> {
+ final FileInputStream f = new FileInputStream(file);
+ f.getChannel().position(offset);
+ return f;
+ });
+ return setCurrentStream(new BufferedInputStream(fis));
+ }
+
+ @Override
+ public String toString() {
+ return "FileWithOffsetContentProvider{" +
+ "file=" + file +
+ ", offset=" + offset +
+ "} " + super.toString();
+ }
+
+ }
+
+ /**
+ * Create a content provider for a byte buffer.
+ * Uses {@link ByteBufferInputStream} to read the data.
+ */
+ private static final class ByteBufferContentProvider
+ extends BaseContentProvider {
+
+ /**
+ * The buffer which will be read; on or off heap.
+ */
+ private final ByteBuffer blockBuffer;
+
+ /**
+ * The position in the buffer at the time the provider was created.
+ */
+ private final int initialPosition;
+
+ /**
+ * Constructor.
+ * @param blockBuffer buffer to read.
+ * @param size size of the data.
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null
+ */
+ private ByteBufferContentProvider(final ByteBuffer blockBuffer, int size) {
+ this(blockBuffer, size, null);
+ }
+
+ /**
+ * Constructor.
+ * @param blockBuffer buffer to read.
+ * @param size size of the data.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null
+ */
+ private ByteBufferContentProvider(
+ final ByteBuffer blockBuffer,
+ int size,
+ @Nullable final Supplier isOpen) {
+ super(size, isOpen);
+ this.blockBuffer = blockBuffer;
+ this.initialPosition = blockBuffer.position();
+ }
+
+ @Override
+ protected ByteBufferInputStream createNewStream() {
+ // set the buffer up from reading from the beginning
+ blockBuffer.limit(initialPosition);
+ blockBuffer.position(0);
+ return new ByteBufferInputStream(getSize(), blockBuffer);
+ }
+
+ @Override
+ public String toString() {
+ return "ByteBufferContentProvider{" +
+ "blockBuffer=" + blockBuffer +
+ ", initialPosition=" + initialPosition +
+ "} " + super.toString();
+ }
+ }
+
+ /**
+ * Simple byte array content provider.
+ *
+ * The array is not copied; if it is changed during the write the outcome
+ * of the upload is undefined.
+ */
+ private static final class ByteArrayContentProvider
+ extends BaseContentProvider {
+
+ /**
+ * The buffer where data is stored.
+ */
+ private final byte[] bytes;
+
+ /**
+ * Offset in the buffer.
+ */
+ private final int offset;
+
+ /**
+ * Constructor.
+ * @param bytes buffer to read.
+ * @param offset offset in buffer.
+ * @param size length of the data.
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null
+ */
+ private ByteArrayContentProvider(
+ final byte[] bytes,
+ final int offset,
+ final int size) {
+ this(bytes, offset, size, null);
+ }
+
+ /**
+ * Constructor.
+ * @param bytes buffer to read.
+ * @param offset offset in buffer.
+ * @param size length of the data.
+ * @param isOpen optional predicate to check if the stream is open.
+ * @throws IllegalArgumentException if the arguments are invalid.
+ * @throws NullPointerException if the buffer is null
+ */
+ private ByteArrayContentProvider(
+ final byte[] bytes,
+ final int offset,
+ final int size,
+ final Supplier isOpen) {
+
+ super(size, isOpen);
+ this.bytes = bytes;
+ this.offset = offset;
+ checkArgument(offset >= 0, "Offset is negative: %s", offset);
+ final int length = bytes.length;
+ checkArgument((offset + size) <= length,
+ "Data to read [%d-%d] is past end of array %s",
+ offset,
+ offset + size, length);
+ }
+
+ @Override
+ protected ByteArrayInputStream createNewStream() {
+ return new ByteArrayInputStream(bytes, offset, getSize());
+ }
+
+ @Override
+ public String toString() {
+ return "ByteArrayContentProvider{" +
+ "buffer with length=" + bytes.length +
+ ", offset=" + offset +
+ "} " + super.toString();
+ }
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java
index 554b628d003a4..6bf2354a83ede 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java
@@ -42,7 +42,8 @@ public interface BlockOutputStreamStatistics extends Closeable,
void blockUploadStarted(Duration timeInQueue, long blockSize);
/**
- * A block upload has completed. Duration excludes time in the queue.
+ * A block upload has completed, successfully or not.
+ * Duration excludes time in the queue.
* @param timeSinceUploadStarted time in since the transfer began.
* @param blockSize block size
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
index 48b0b2bf454d3..15c2ab9b2548f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/StatisticsFromAwsSdkImpl.java
@@ -27,6 +27,16 @@
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_429_TOO_MANY_REQUESTS_GCS;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_500_INTERNAL_SERVER_ERROR;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_503_SERVICE_UNAVAILABLE;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_400;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_4XX;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_500;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_503;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.HTTP_RESPONSE_5XX;
/**
* Hook up AWS SDK Statistics to the S3 counters.
@@ -85,4 +95,37 @@ public void noteRequestSigningTime(final Duration duration) {
public void noteResponseProcessingTime(final Duration duration) {
}
+
+ /**
+ * Map error status codes to statistic names, excluding 404.
+ * 429 (google throttle events) are mapped to the 503 statistic.
+ * @param sc status code.
+ * @return a statistic name or null.
+ */
+ public static String mapErrorStatusCodeToStatisticName(int sc) {
+ String stat = null;
+ switch (sc) {
+ case SC_400_BAD_REQUEST:
+ stat = HTTP_RESPONSE_400;
+ break;
+ case SC_404_NOT_FOUND:
+ /* do not map; not measured */
+ break;
+ case SC_500_INTERNAL_SERVER_ERROR:
+ stat = HTTP_RESPONSE_500;
+ break;
+ case SC_503_SERVICE_UNAVAILABLE:
+ case SC_429_TOO_MANY_REQUESTS_GCS:
+ stat = HTTP_RESPONSE_503;
+ break;
+
+ default:
+ if (sc > 500) {
+ stat = HTTP_RESPONSE_5XX;
+ } else if (sc > 400) {
+ stat = HTTP_RESPONSE_4XX;
+ }
+ }
+ return stat;
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md
index de3ea79f4fb1d..1018ec9e7d6c2 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md
@@ -213,7 +213,26 @@ as they keep trying to reconnect to ports which are never going to be available.
fs.s3a.bucket.nonexistent-bucket-example.connection.establish.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