From d94cec49615eb0438069108d457c85669aaa4242 Mon Sep 17 00:00:00 2001
From: Zoe Wang <33073555+zoewangg@users.noreply.github.com>
Date: Fri, 1 Aug 2025 18:29:53 -0700
Subject: [PATCH 1/5] Introduce a new split method in AsyncRequestBody that
returns an SdkPublisher of ClosableAsyncRequestBody and use it in s3
multipart client
---
.../awssdk/core/async/AsyncRequestBody.java | 41 ++++-
.../core/async/ClosableAsyncRequestBody.java | 26 +++
.../async/ByteBuffersAsyncRequestBody.java | 4 +-
.../ClosableAsyncRequestBodyAdaptor.java | 67 +++++++
.../internal/async/FileAsyncRequestBody.java | 28 +++
.../internal/async/SplittingPublisher.java | 130 +++++++-------
...yncRequestBodySplitConfigurationTest.java} | 2 +-
.../core/async/AsyncRequestBodyTest.java | 81 ++++++++-
.../ClosableAsyncRequestBodyAdaptorTest.java | 58 ++++++
.../FileAsyncRequestBodySplitHelperTest.java | 4 +-
.../async/SplittingPublisherTest.java | 44 ++---
.../async/SplittingPublisherTestUtils.java | 14 +-
...ntentLengthAsyncRequestBodySubscriber.java | 10 +-
.../UploadWithKnownContentLengthHelper.java | 2 +-
.../UploadWithUnknownContentLengthHelper.java | 37 ++--
...tLengthAsyncRequestBodySubscriberTest.java | 35 ++--
...3MultipartClientPutObjectWiremockTest.java | 166 ++++++++----------
...oadWithUnknownContentLengthHelperTest.java | 24 +--
.../s3/src/test/resources/log4j2.properties | 2 +-
19 files changed, 500 insertions(+), 275 deletions(-)
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java
rename core/sdk-core/src/test/java/software/amazon/awssdk/core/async/{AsyncRequestBodyConfigurationTest.java => AsyncRequestBodySplitConfigurationTest.java} (97%)
create mode 100644 core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
index 3fd8c3cc0165..22ac529aafa9 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
@@ -33,6 +33,7 @@
import software.amazon.awssdk.annotations.SdkPublicApi;
import software.amazon.awssdk.core.FileRequestBodyConfiguration;
import software.amazon.awssdk.core.internal.async.ByteBuffersAsyncRequestBody;
+import software.amazon.awssdk.core.internal.async.ClosableAsyncRequestBodyAdaptor;
import software.amazon.awssdk.core.internal.async.FileAsyncRequestBody;
import software.amazon.awssdk.core.internal.async.InputStreamWithExecutorAsyncRequestBody;
import software.amazon.awssdk.core.internal.async.SplittingPublisher;
@@ -507,17 +508,33 @@ static AsyncRequestBody empty() {
* is 2MB and the default buffer size is 8MB.
*
*
- * By default, if content length of this {@link AsyncRequestBody} is present, each divided {@link AsyncRequestBody} is
- * delivered to the subscriber right after it's initialized. On the other hand, if content length is null, it is sent after
- * the entire content for that chunk is buffered. In this case, the configured {@code maxMemoryUsageInBytes} must be larger
- * than or equal to {@code chunkSizeInBytes}. Note that this behavior may be different if a specific implementation of this
- * interface overrides this method.
+ * Each divided {@link AsyncRequestBody} is sent after the entire content for that chunk is buffered.
*
* @see AsyncRequestBodySplitConfiguration
+ * @deprecated Use {@link #splitV2(AsyncRequestBodySplitConfiguration)} instead.
*/
+ @Deprecated
default SdkPublisher split(AsyncRequestBodySplitConfiguration splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
+ return splitV2(splitConfiguration).map(body -> new ClosableAsyncRequestBodyAdaptor(body));
+ }
+ /**
+ * Converts this {@link AsyncRequestBody} to a publisher of {@link ClosableAsyncRequestBody}s, each of which publishes
+ * specific portion of the original data, based on the provided {@link AsyncRequestBodySplitConfiguration}. The default chunk
+ * size is 2MB and the default buffer size is 8MB.
+ *
+ *
+ * Each divided {@link ClosableAsyncRequestBody} is sent after the entire content for that chunk is buffered. This behavior
+ * may be different if a specific implementation of this interface overrides this method.
+ *
+ *
+ * Each {@link ClosableAsyncRequestBody} MUST be closed by the user when it is ready to be disposed.
+ *
+ * @see AsyncRequestBodySplitConfiguration
+ */
+ default SdkPublisher splitV2(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ Validate.notNull(splitConfiguration, "splitConfiguration");
return new SplittingPublisher(this, splitConfiguration);
}
@@ -526,12 +543,26 @@ default SdkPublisher split(AsyncRequestBodySplitConfiguration
* avoiding the need to create one manually via {@link AsyncRequestBodySplitConfiguration#builder()}.
*
* @see #split(AsyncRequestBodySplitConfiguration)
+ * @deprecated Use {@link #splitV2(Consumer)} instead.
*/
+ @Deprecated
default SdkPublisher split(Consumer splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
return split(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
}
+ /**
+ * This is a convenience method that passes an instance of the {@link AsyncRequestBodySplitConfiguration} builder,
+ * avoiding the need to create one manually via {@link AsyncRequestBodySplitConfiguration#builder()}.
+ *
+ * @see #splitV2(Consumer)
+ */
+ default SdkPublisher splitV2(
+ Consumer splitConfiguration) {
+ Validate.notNull(splitConfiguration, "splitConfiguration");
+ return splitV2(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
+ }
+
@SdkProtectedApi
enum BodyType {
FILE("File", "f"),
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java
new file mode 100644
index 000000000000..7f495883c477
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.async;
+
+import software.amazon.awssdk.annotations.SdkPublicApi;
+import software.amazon.awssdk.utils.SdkAutoCloseable;
+
+/**
+ * An extension of {@link AsyncRequestBody} that is closable.
+ */
+@SdkPublicApi
+public interface ClosableAsyncRequestBody extends AsyncRequestBody, SdkAutoCloseable {
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
index a304d75ccf94..1ae49d0dfdde 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
@@ -76,7 +76,9 @@ public final class ByteBuffersAsyncRequestBody implements AsyncRequestBody, SdkA
private final Object lock = new Object();
private boolean closed;
- private ByteBuffersAsyncRequestBody(String mimetype, Long length, List buffers) {
+ private ByteBuffersAsyncRequestBody(String mimetype,
+ Long length,
+ List buffers) {
this.mimetype = mimetype;
this.buffers = buffers;
this.length = length;
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java
new file mode 100644
index 000000000000..1c3d126981dd
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.reactivestreams.Subscriber;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+import software.amazon.awssdk.core.internal.util.NoopSubscription;
+
+/**
+ * Adaptor to convert a {@link ClosableAsyncRequestBody} to an {@link AsyncRequestBody}
+ *
+ *
+ * This is needed to maintain backwards compatibility for the deprecated
+ * {@link AsyncRequestBody#split(AsyncRequestBodySplitConfiguration)}
+ */
+@SdkInternalApi
+public final class ClosableAsyncRequestBodyAdaptor implements AsyncRequestBody {
+
+ private final AtomicBoolean subscribeCalled;
+ private final ClosableAsyncRequestBody delegate;
+
+ public ClosableAsyncRequestBodyAdaptor(ClosableAsyncRequestBody delegate) {
+ this.delegate = delegate;
+ subscribeCalled = new AtomicBoolean(false);
+ }
+
+ @Override
+ public Optional contentLength() {
+ return delegate.contentLength();
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> subscriber) {
+ if (subscribeCalled.compareAndSet(false, true)) {
+ delegate.doAfterOnComplete(() -> delegate.close())
+ .doAfterOnCancel(() -> delegate.close())
+ .doAfterOnError(t -> delegate.close())
+ .subscribe(subscriber);
+ } else {
+ subscriber.onSubscribe(new NoopSubscription(subscriber));
+ subscriber.onError(NonRetryableException.create(
+ "A retry was attempted, but AsyncRequestBody.split does not "
+ + "support retries."));
+ }
+ }
+
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
index f5dcc164f61c..656cd1a38e56 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
@@ -34,6 +34,7 @@
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.core.internal.util.Mimetype;
import software.amazon.awssdk.core.internal.util.NoopSubscription;
@@ -86,6 +87,11 @@ public SdkPublisher split(AsyncRequestBodySplitConfiguration s
return new FileAsyncRequestBodySplitHelper(this, splitConfiguration).split();
}
+ @Override
+ public SdkPublisher splitV2(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ return split(splitConfiguration).map(body -> new ClosableAsyncRequestBodyWrapper(body));
+ }
+
public Path path() {
return path;
}
@@ -436,4 +442,26 @@ private void signalOnError(Throwable t) {
private static AsynchronousFileChannel openInputChannel(Path path) throws IOException {
return AsynchronousFileChannel.open(path, StandardOpenOption.READ);
}
+
+ private static class ClosableAsyncRequestBodyWrapper implements ClosableAsyncRequestBody {
+ private final AsyncRequestBody body;
+
+ ClosableAsyncRequestBodyWrapper(AsyncRequestBody body) {
+ this.body = body;
+ }
+
+ @Override
+ public Optional contentLength() {
+ return body.contentLength();
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ body.subscribe(s);
+ }
+
+ @Override
+ public void close() {
+ }
+ }
}
\ No newline at end of file
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
index 12278cf84dca..1275d5e92fb3 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
@@ -16,6 +16,8 @@
package software.amazon.awssdk.core.internal.async;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@@ -25,9 +27,8 @@
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
-import software.amazon.awssdk.core.exception.NonRetryableException;
-import software.amazon.awssdk.core.internal.util.NoopSubscription;
import software.amazon.awssdk.utils.Logger;
import software.amazon.awssdk.utils.Validate;
import software.amazon.awssdk.utils.async.SimplePublisher;
@@ -36,17 +37,17 @@
* Splits an {@link AsyncRequestBody} to multiple smaller {@link AsyncRequestBody}s, each of which publishes a specific portion of
* the original data.
*
- * If content length is known, each {@link AsyncRequestBody} is sent to the subscriber right after it's initialized.
- * Otherwise, it is sent after the entire content for that chunk is buffered. This is required to get content length.
+ *
Each {@link AsyncRequestBody} is sent after the entire content for that chunk is buffered.
*/
@SdkInternalApi
-public class SplittingPublisher implements SdkPublisher {
+public class SplittingPublisher implements SdkPublisher {
private static final Logger log = Logger.loggerFor(SplittingPublisher.class);
private final AsyncRequestBody upstreamPublisher;
private final SplittingSubscriber splittingSubscriber;
- private final SimplePublisher downstreamPublisher = new SimplePublisher<>();
+ private final SimplePublisher downstreamPublisher = new SimplePublisher<>();
private final long chunkSizeInBytes;
private final long bufferSizeInBytes;
+ private final AtomicBoolean currentBodySent = new AtomicBoolean(false);
public SplittingPublisher(AsyncRequestBody asyncRequestBody,
AsyncRequestBodySplitConfiguration splitConfiguration) {
@@ -62,15 +63,13 @@ public SplittingPublisher(AsyncRequestBody asyncRequestBody,
this.splittingSubscriber = new SplittingSubscriber(upstreamPublisher.contentLength().orElse(null));
- if (!upstreamPublisher.contentLength().isPresent()) {
- Validate.isTrue(bufferSizeInBytes >= chunkSizeInBytes,
- "bufferSizeInBytes must be larger than or equal to " +
- "chunkSizeInBytes if the content length is unknown");
- }
+ Validate.isTrue(bufferSizeInBytes >= chunkSizeInBytes,
+ "bufferSizeInBytes must be larger than or equal to " +
+ "chunkSizeInBytes if the content length is unknown");
}
@Override
- public void subscribe(Subscriber super AsyncRequestBody> downstreamSubscriber) {
+ public void subscribe(Subscriber super ClosableAsyncRequestBody> downstreamSubscriber) {
downstreamPublisher.subscribe(downstreamSubscriber);
upstreamPublisher.subscribe(splittingSubscriber);
}
@@ -104,11 +103,9 @@ public void onSubscribe(Subscription s) {
}
private DownstreamBody initializeNextDownstreamBody(boolean contentLengthKnown, long chunkSize, int chunkNumber) {
- DownstreamBody body = new DownstreamBody(contentLengthKnown, chunkSize, chunkNumber);
- if (contentLengthKnown) {
- sendCurrentBody(body);
- }
- return body;
+ currentBodySent.set(false);
+ log.debug(() -> "initializing next downstream body " + chunkNumber);
+ return new DownstreamBody(contentLengthKnown, chunkSize, chunkNumber);
}
@Override
@@ -157,7 +154,7 @@ public void onNext(ByteBuffer byteBuffer) {
}
private void completeCurrentBodyAndCreateNewIfNeeded(ByteBuffer byteBuffer) {
- completeCurrentBody();
+ completeCurrentBodyAndDeliver();
int currentChunk = chunkNumber.incrementAndGet();
boolean shouldCreateNewDownstreamRequestBody;
Long dataRemaining = totalDataRemaining();
@@ -175,22 +172,42 @@ private void completeCurrentBodyAndCreateNewIfNeeded(ByteBuffer byteBuffer) {
}
private int amountRemainingInChunk() {
- return Math.toIntExact(currentBody.maxLength - currentBody.transferredLength);
+ return Math.toIntExact(currentBody.maxLength - currentBody.bufferedLength);
}
- private void completeCurrentBody() {
- log.debug(() -> "completeCurrentBody for chunk " + chunkNumber.get());
- currentBody.complete();
- if (upstreamSize == null) {
- sendCurrentBody(currentBody);
+ /**
+ * Current body could be completed in either onNext or onComplete, so we need to guard against sending the last body
+ * twice.
+ */
+ private void completeCurrentBodyAndDeliver() {
+ if (currentBodySent.compareAndSet(false, true)) {
+ log.debug(() -> "completeCurrentBody for chunk " + currentBody.chunkNumber);
+ // For unknown content length, we always create a new DownstreamBody because we don't know if there is data
+ // left or not, so we need to only send the body if there is actually data
+ long bufferedLength = currentBody.bufferedLength;
+ Long totalLength = currentBody.totalLength;
+ if (bufferedLength > 0) {
+ if (totalLength != null && totalLength != bufferedLength) {
+ upstreamSubscription.cancel();
+ downstreamPublisher.error(new IllegalStateException(
+ String.format("Content length of buffered data mismatches "
+ + "with the expected content length, buffered data content length: %d, "
+ + "expected length: %d", totalLength,
+ bufferedLength)));
+ return;
+ }
+
+ currentBody.complete();
+ sendCurrentBody(currentBody);
+ }
}
}
@Override
public void onComplete() {
upstreamComplete = true;
- log.trace(() -> "Received onComplete()");
- completeCurrentBody();
+ log.debug(() -> "Received onComplete() from upstream AsyncRequestBody");
+ completeCurrentBodyAndDeliver();
downstreamPublisher.complete();
}
@@ -200,7 +217,8 @@ public void onError(Throwable t) {
downstreamPublisher.error(t);
}
- private void sendCurrentBody(AsyncRequestBody body) {
+ private void sendCurrentBody(DownstreamBody body) {
+ log.debug(() -> "sendCurrentBody for chunk " + body.chunkNumber);
downstreamPublisher.send(body).exceptionally(t -> {
downstreamPublisher.error(t);
upstreamSubscription.cancel();
@@ -227,7 +245,7 @@ private void maybeRequestMoreUpstreamData() {
}
private boolean shouldRequestMoreData(long buffered) {
- return buffered == 0 || buffered + byteBufferSizeHint <= bufferSizeInBytes;
+ return buffered <= 0 || buffered + byteBufferSizeHint <= bufferSizeInBytes;
}
private Long totalDataRemaining() {
@@ -237,7 +255,11 @@ private Long totalDataRemaining() {
return upstreamSize - (chunkNumber.get() * chunkSizeInBytes);
}
- private final class DownstreamBody implements AsyncRequestBody {
+ /**
+ * AsyncRequestBody for individual part. The entire data is buffered in memory and can be subscribed multiple times
+ * for retry attempts. The buffered data is cleared upon close
+ */
+ private final class DownstreamBody implements ClosableAsyncRequestBody {
/**
* The maximum length of the content this AsyncRequestBody can hold. If the upstream content length is known, this is
@@ -245,10 +267,10 @@ private final class DownstreamBody implements AsyncRequestBody {
*/
private final long maxLength;
private final Long totalLength;
- private final SimplePublisher delegate = new SimplePublisher<>();
private final int chunkNumber;
- private final AtomicBoolean subscribeCalled = new AtomicBoolean(false);
- private volatile long transferredLength = 0;
+ private volatile long bufferedLength = 0;
+ private volatile ByteBuffersAsyncRequestBody delegate;
+ private final List buffers = new ArrayList<>();
private DownstreamBody(boolean contentLengthKnown, long maxLength, int chunkNumber) {
this.totalLength = contentLengthKnown ? maxLength : null;
@@ -258,53 +280,41 @@ private DownstreamBody(boolean contentLengthKnown, long maxLength, int chunkNumb
@Override
public Optional contentLength() {
- return totalLength != null ? Optional.of(totalLength) : Optional.of(transferredLength);
+ return totalLength != null ? Optional.of(totalLength) : Optional.of(bufferedLength);
}
public void send(ByteBuffer data) {
- log.trace(() -> String.format("Sending bytebuffer %s to chunk %d", data, chunkNumber));
+ log.debug(() -> String.format("Sending bytebuffer %s to chunk %d", data, chunkNumber));
int length = data.remaining();
- transferredLength += length;
+ bufferedLength += length;
addDataBuffered(length);
- delegate.send(data).whenComplete((r, t) -> {
- addDataBuffered(-length);
- if (t != null) {
- error(t);
- }
- });
+ buffers.add(data);
}
public void complete() {
- log.debug(() -> "Received complete() for chunk number: " + chunkNumber + " length " + transferredLength);
- delegate.complete().whenComplete((r, t) -> {
- if (t != null) {
- error(t);
- }
- });
- }
-
- public void error(Throwable error) {
- delegate.error(error);
+ log.debug(() -> "Received complete() for chunk number: " + chunkNumber + " length " + bufferedLength);
+ this.delegate = ByteBuffersAsyncRequestBody.of(buffers);
}
@Override
public void subscribe(Subscriber super ByteBuffer> s) {
- if (subscribeCalled.compareAndSet(false, true)) {
- delegate.subscribe(s);
- } else {
- s.onSubscribe(new NoopSubscription(s));
- s.onError(NonRetryableException.create(
- "A retry was attempted, but AsyncRequestBody.split does not "
- + "support retries."));
- }
+ log.debug(() -> "Subscribe for chunk number: " + chunkNumber + " length " + bufferedLength);
+ delegate.subscribe(s);
}
- private void addDataBuffered(int length) {
+ private void addDataBuffered(long length) {
dataBuffered.addAndGet(length);
if (length < 0) {
maybeRequestMoreUpstreamData();
}
}
+
+ @Override
+ public void close() {
+ log.debug(() -> "Closing current body " + chunkNumber);
+ delegate.close();
+ addDataBuffered(-bufferedLength);
+ }
}
}
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyConfigurationTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodySplitConfigurationTest.java
similarity index 97%
rename from core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyConfigurationTest.java
rename to core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodySplitConfigurationTest.java
index 8b8f78f2b5e9..e932da3bfa1c 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyConfigurationTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodySplitConfigurationTest.java
@@ -23,7 +23,7 @@
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
-public class AsyncRequestBodyConfigurationTest {
+public class AsyncRequestBodySplitConfigurationTest {
@Test
void equalsHashCode() {
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
index cdd87822e3d4..f0c72a624119 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
@@ -19,10 +19,12 @@
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static software.amazon.awssdk.core.internal.async.SplittingPublisherTestUtils.verifyIndividualAsyncRequestBody;
import com.google.common.jimfs.Configuration;
import com.google.common.jimfs.Jimfs;
import io.reactivex.Flowable;
+import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
@@ -31,13 +33,20 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.commons.lang3.RandomStringUtils;
import org.assertj.core.util.Lists;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import org.reactivestreams.Publisher;
@@ -49,16 +58,33 @@
public class AsyncRequestBodyTest {
private static final String testString = "Hello!";
- private static final Path path;
-
- static {
- FileSystem fs = Jimfs.newFileSystem(Configuration.unix());
+ private static Path path;
+ private static final int CONTENT_SIZE = 1024;
+ private static final byte[] CONTENT =
+ RandomStringUtils.randomAscii(CONTENT_SIZE).getBytes(Charset.defaultCharset());
+ private static File fileForSplit;
+ private static FileSystem fs;
+
+ @BeforeAll
+ public static void setup() throws IOException {
+ fs = Jimfs.newFileSystem(Configuration.unix());
path = fs.getPath("./test");
- try {
- Files.write(path, testString.getBytes());
- } catch (IOException e) {
- e.printStackTrace();
- }
+ Files.write(path, testString.getBytes());
+
+ fileForSplit = File.createTempFile("SplittingPublisherTest", UUID.randomUUID().toString());
+ Files.write(fileForSplit.toPath(), CONTENT);
+ }
+
+ @AfterAll
+ public static void teardown() throws IOException {
+ fileForSplit.delete();
+ fs.close();
+ }
+
+ public static Stream asyncRequestBodies() {
+ return Stream.of(Arguments.of(AsyncRequestBody.fromBytes(CONTENT)),
+ Arguments.of(AsyncRequestBody.fromFile(b -> b.chunkSizeInBytes(50)
+ .path(fileForSplit.toPath()))));
}
@ParameterizedTest
@@ -300,6 +326,34 @@ void rewindingByteBufferBuildersReadTheInputBufferFromTheBeginning(
assertEquals(bb, publishedBuffer.get());
}
+ @ParameterizedTest
+ @MethodSource("asyncRequestBodies")
+ void legacySplit_shouldWork(AsyncRequestBody delegate) throws Exception {
+ long chunkSize = 20l;
+ AsyncRequestBody asyncRequestBody = new AsyncRequestBody() {
+ @Override
+ public Optional contentLength() {
+ return delegate.contentLength();
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ delegate.subscribe(s);
+ }
+ };
+
+ AsyncRequestBodySplitConfiguration configuration = AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes(chunkSize)
+ .bufferSizeInBytes(chunkSize)
+ .build();
+
+ SdkPublisher split = asyncRequestBody.split(configuration);
+ verifyIndividualAsyncRequestBody(split, fileForSplit.toPath(), (int) chunkSize);
+ }
+
+
+
+
private static Function[] rewindingByteBufferBodyBuilders() {
Function fromByteBuffer = AsyncRequestBody::fromByteBuffer;
Function fromByteBufferUnsafe = AsyncRequestBody::fromByteBufferUnsafe;
@@ -356,4 +410,13 @@ void publisherConstructorHasCorrectContentType() {
AsyncRequestBody requestBody = AsyncRequestBody.fromPublisher(bodyPublisher);
assertEquals(Mimetype.MIMETYPE_OCTET_STREAM, requestBody.contentType());
}
+
+ @Test
+ void splitV2_nullConfig_shouldThrowException() {
+ AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world");
+ AsyncRequestBodySplitConfiguration config = null;
+ assertThatThrownBy(() -> requestBody.splitV2(config))
+ .isInstanceOf(NullPointerException.class)
+ .hasMessageContaining("splitConfig");
+ }
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
new file mode 100644
index 000000000000..7b633c1af1d6
--- /dev/null
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doNothing;
+
+import io.reactivex.Flowable;
+import io.reactivex.FlowableSubscriber;
+import java.util.function.Consumer;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.reactivestreams.Subscriber;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+
+public class ClosableAsyncRequestBodyAdaptorTest {
+
+ @Test
+ void resubscribe_shouldThrowException() {
+ ClosableAsyncRequestBody closableAsyncRequestBody = Mockito.mock(ClosableAsyncRequestBody.class);
+ Mockito.when(closableAsyncRequestBody.doAfterOnComplete(any(Runnable.class))).thenReturn(closableAsyncRequestBody);
+ Mockito.when(closableAsyncRequestBody.doAfterOnCancel(any(Runnable.class))).thenReturn(closableAsyncRequestBody);
+ Mockito.when(closableAsyncRequestBody.doAfterOnError(any(Consumer.class))).thenReturn(closableAsyncRequestBody);
+
+ ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(closableAsyncRequestBody);
+ Subscriber subscriber = Mockito.mock(Subscriber.class);
+ adaptor.subscribe(subscriber);
+
+ Subscriber anotherSubscriber = Mockito.mock(Subscriber.class);
+ ArgumentCaptor exceptionCaptor = ArgumentCaptor.forClass(Throwable.class);
+ doNothing().when(anotherSubscriber).onError(exceptionCaptor.capture());
+
+ adaptor.subscribe(anotherSubscriber);
+
+ assertThat(exceptionCaptor.getValue())
+ .isInstanceOf(NonRetryableException.class)
+ .hasMessageContaining("A retry was attempted");
+ }
+
+}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBodySplitHelperTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBodySplitHelperTest.java
index 4c5d0748d16d..1edea1a58b1b 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBodySplitHelperTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBodySplitHelperTest.java
@@ -77,7 +77,9 @@ public void split_differentChunkSize_shouldSplitCorrectly(int chunkSize) throws
ScheduledFuture> scheduledFuture = executor.scheduleWithFixedDelay(verifyConcurrentRequests(helper, maxConcurrency),
1, 50, TimeUnit.MICROSECONDS);
- verifyIndividualAsyncRequestBody(helper.split(), testFile, chunkSize);
+ verifyIndividualAsyncRequestBody(helper.split(),
+ testFile,
+ chunkSize);
scheduledFuture.cancel(true);
int expectedMaxConcurrency = (int) (bufferSize / chunkSizeInBytes);
assertThat(maxConcurrency.get()).isLessThanOrEqualTo(expectedMaxConcurrency);
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
index 6f116ca2667c..a1f22d2bf7af 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
@@ -16,7 +16,6 @@
package software.amazon.awssdk.core.internal.async;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.assertj.core.api.Assertions.fail;
import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
import static software.amazon.awssdk.core.internal.async.SplittingPublisherTestUtils.verifyIndividualAsyncRequestBody;
import static software.amazon.awssdk.utils.FunctionalUtils.invokeSafely;
@@ -24,7 +23,6 @@
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
-import java.io.FileInputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
@@ -42,7 +40,6 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
-import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
import org.reactivestreams.Subscription;
import software.amazon.awssdk.core.async.AsyncRequestBody;
@@ -72,13 +69,18 @@ public static void afterAll() throws Exception {
}
@Test
- public void split_contentUnknownMaxMemorySmallerThanChunkSize_shouldThrowException() {
+ public void split_MaxMemorySmallerThanChunkSize_shouldThrowException() {
AsyncRequestBody body = AsyncRequestBody.fromPublisher(s -> {
});
- assertThatThrownBy(() -> new SplittingPublisher(body, AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes(10L)
- .bufferSizeInBytes(5L)
- .build()))
+ AsyncRequestBodySplitConfiguration configuration = AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes(10L)
+ .bufferSizeInBytes(5L)
+ .build();
+ assertThatThrownBy(() -> new SplittingPublisher(body, configuration))
+ .hasMessageContaining("must be larger than or equal");
+
+ assertThatThrownBy(() -> new SplittingPublisher(AsyncRequestBody.fromString("test"),
+ configuration))
.hasMessageContaining("must be larger than or equal");
}
@@ -169,7 +171,7 @@ private static void verifySplitContent(AsyncRequestBody asyncRequestBody, int ch
.bufferSizeInBytes((long) chunkSize * 4)
.build());
- verifyIndividualAsyncRequestBody(splittingPublisher, testFile.toPath(), chunkSize);
+ verifyIndividualAsyncRequestBody(splittingPublisher.map(m -> m), testFile.toPath(), chunkSize);
}
private static class TestAsyncRequestBody implements AsyncRequestBody {
@@ -204,30 +206,6 @@ public void cancel() {
}
}
- private static final class OnlyRequestOnceSubscriber implements Subscriber {
- private List asyncRequestBodies = new ArrayList<>();
-
- @Override
- public void onSubscribe(Subscription s) {
- s.request(1);
- }
-
- @Override
- public void onNext(AsyncRequestBody requestBody) {
- asyncRequestBodies.add(requestBody);
- }
-
- @Override
- public void onError(Throwable t) {
-
- }
-
- @Override
- public void onComplete() {
-
- }
- }
-
private static final class BaosSubscriber implements Subscriber {
private final CompletableFuture resultFuture;
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java
index 04da97adbf42..145a1cecc0ef 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java
@@ -15,23 +15,16 @@
package software.amazon.awssdk.core.internal.async;
-import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
-
-import java.io.File;
import java.io.FileInputStream;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
import org.assertj.core.api.Assertions;
-import org.reactivestreams.Publisher;
import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
-import software.amazon.awssdk.core.internal.async.ByteArrayAsyncResponseTransformer;
-import software.amazon.awssdk.core.internal.async.SplittingPublisherTest;
public final class SplittingPublisherTestUtils {
@@ -45,6 +38,11 @@ public static void verifyIndividualAsyncRequestBody(SdkPublisher {
+ if (requestBody instanceof ClosableAsyncRequestBody) {
+ ((ClosableAsyncRequestBody) requestBody).close();
+ }
+ });
futures.add(baosFuture);
}).get(5, TimeUnit.SECONDS);
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
index 93bc0dfeb6f8..92ef401e830f 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
@@ -34,6 +34,7 @@
import org.reactivestreams.Subscription;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.listener.PublisherListener;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
@@ -47,7 +48,7 @@
import software.amazon.awssdk.utils.Pair;
@SdkInternalApi
-public class KnownContentLengthAsyncRequestBodySubscriber implements Subscriber {
+public class KnownContentLengthAsyncRequestBodySubscriber implements Subscriber {
private static final Logger log = Logger.loggerFor(KnownContentLengthAsyncRequestBodySubscriber.class);
@@ -144,7 +145,7 @@ public void onSubscribe(Subscription s) {
}
@Override
- public void onNext(AsyncRequestBody asyncRequestBody) {
+ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
if (isPaused || isDone) {
return;
}
@@ -152,8 +153,9 @@ public void onNext(AsyncRequestBody asyncRequestBody) {
int currentPartNum = partNumber.getAndIncrement();
if (existingParts.containsKey(currentPartNum)) {
asyncRequestBody.subscribe(new CancelledSubscriber<>());
- subscription.request(1);
asyncRequestBody.contentLength().ifPresent(progressListener::subscriberOnNext);
+ asyncRequestBody.close();
+ subscription.request(1);
return;
}
@@ -178,10 +180,12 @@ public void onNext(AsyncRequestBody asyncRequestBody) {
multipartUploadHelper.sendIndividualUploadPartRequest(uploadId, completedPartConsumer, futures,
Pair.of(uploadRequest, asyncRequestBody), progressListener)
.whenComplete((r, t) -> {
+ asyncRequestBody.close();
if (t != null) {
if (shouldFailRequest()) {
multipartUploadHelper.failRequestsElegantly(futures, t, uploadId, returnFuture,
putObjectRequest);
+ subscription.cancel();
}
} else {
completeMultipartUploadIfFinished(asyncRequestBodyInFlight.decrementAndGet());
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
index 04690677c92b..15f15767db54 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
@@ -186,7 +186,7 @@ private void splitAndSubscribe(MpuRequestContext mpuRequestContext, CompletableF
attachSubscriberToObservable(subscriber, mpuRequestContext.request().left());
mpuRequestContext.request().right()
- .split(b -> b.chunkSizeInBytes(mpuRequestContext.partSize())
+ .splitV2(b -> b.chunkSizeInBytes(mpuRequestContext.partSize())
.bufferSizeInBytes(maxMemoryUsageInBytes))
.subscribe(subscriber);
}
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
index 520625ad90b0..31a6e685db17 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
@@ -33,6 +33,7 @@
import org.reactivestreams.Subscription;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.core.async.listener.PublisherListener;
import software.amazon.awssdk.core.exception.SdkClientException;
@@ -53,12 +54,10 @@
public final class UploadWithUnknownContentLengthHelper {
private static final Logger log = Logger.loggerFor(UploadWithUnknownContentLengthHelper.class);
- private final S3AsyncClient s3AsyncClient;
private final long partSizeInBytes;
private final GenericMultipartHelper genericMultipartHelper;
private final long maxMemoryUsageInBytes;
- private final long multipartUploadThresholdInBytes;
private final MultipartUploadHelper multipartUploadHelper;
@@ -66,13 +65,11 @@ public UploadWithUnknownContentLengthHelper(S3AsyncClient s3AsyncClient,
long partSizeInBytes,
long multipartUploadThresholdInBytes,
long maxMemoryUsageInBytes) {
- this.s3AsyncClient = s3AsyncClient;
this.partSizeInBytes = partSizeInBytes;
this.genericMultipartHelper = new GenericMultipartHelper<>(s3AsyncClient,
SdkPojoConversionUtils::toAbortMultipartUploadRequest,
SdkPojoConversionUtils::toPutObjectResponse);
this.maxMemoryUsageInBytes = maxMemoryUsageInBytes;
- this.multipartUploadThresholdInBytes = multipartUploadThresholdInBytes;
this.multipartUploadHelper = new MultipartUploadHelper(s3AsyncClient, multipartUploadThresholdInBytes,
maxMemoryUsageInBytes);
}
@@ -81,8 +78,8 @@ public CompletableFuture uploadObject(PutObjectRequest putObj
AsyncRequestBody asyncRequestBody) {
CompletableFuture returnFuture = new CompletableFuture<>();
- SdkPublisher splitAsyncRequestBodyResponse =
- asyncRequestBody.split(b -> b.chunkSizeInBytes(partSizeInBytes)
+ SdkPublisher splitAsyncRequestBodyResponse =
+ asyncRequestBody.splitV2(b -> b.chunkSizeInBytes(partSizeInBytes)
.bufferSizeInBytes(maxMemoryUsageInBytes));
splitAsyncRequestBodyResponse.subscribe(new UnknownContentLengthAsyncRequestBodySubscriber(partSizeInBytes,
@@ -91,11 +88,7 @@ public CompletableFuture uploadObject(PutObjectRequest putObj
return returnFuture;
}
- private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscriber {
- /**
- * Indicates whether this is the first async request body or not.
- */
- private final AtomicBoolean isFirstAsyncRequestBody = new AtomicBoolean(true);
+ private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscriber {
/**
* Indicates whether CreateMultipartUpload has been initiated or not
@@ -161,12 +154,13 @@ public void onSubscribe(Subscription s) {
}
@Override
- public void onNext(AsyncRequestBody asyncRequestBody) {
+ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
if (isDone) {
return;
}
int currentPartNum = partNumber.incrementAndGet();
- log.trace(() -> "Received asyncRequestBody " + asyncRequestBody.contentLength());
+ log.debug(() -> String.format("Received asyncRequestBody for part number %d with length %d", currentPartNum,
+ asyncRequestBody.contentLength()));
asyncRequestBodyInFlight.incrementAndGet();
Optional sdkClientException = validatePart(asyncRequestBody, currentPartNum);
@@ -177,14 +171,6 @@ public void onNext(AsyncRequestBody asyncRequestBody) {
return;
}
- if (isFirstAsyncRequestBody.compareAndSet(true, false)) {
- log.trace(() -> "Received first async request body");
- // If this is the first AsyncRequestBody received, request another one because we don't know if there is more
- firstRequestBody = asyncRequestBody;
- subscription.request(1);
- return;
- }
-
// If there are more than 1 AsyncRequestBodies, then we know we need to upload this
// object using MPU
if (createMultipartUploadInitiated.compareAndSet(false, true)) {
@@ -201,8 +187,7 @@ public void onNext(AsyncRequestBody asyncRequestBody) {
uploadId = createMultipartUploadResponse.uploadId();
log.debug(() -> "Initiated a new multipart upload, uploadId: " + uploadId);
- sendUploadPartRequest(uploadId, firstRequestBody, 1);
- sendUploadPartRequest(uploadId, asyncRequestBody, 2);
+ sendUploadPartRequest(uploadId, asyncRequestBody, currentPartNum);
// We need to complete the uploadIdFuture *after* the first two requests have been sent
uploadIdFuture.complete(uploadId);
@@ -231,7 +216,7 @@ private Optional validatePart(AsyncRequestBody asyncRequestB
}
private void sendUploadPartRequest(String uploadId,
- AsyncRequestBody asyncRequestBody,
+ ClosableAsyncRequestBody asyncRequestBody,
int currentPartNum) {
Long contentLengthCurrentPart = asyncRequestBody.contentLength().get();
this.contentLength.getAndAdd(contentLengthCurrentPart);
@@ -240,15 +225,17 @@ private void sendUploadPartRequest(String uploadId,
.sendIndividualUploadPartRequest(uploadId, completedParts::add, futures,
uploadPart(asyncRequestBody, currentPartNum), progressListener)
.whenComplete((r, t) -> {
+ asyncRequestBody.close();
if (t != null) {
if (failureActionInitiated.compareAndSet(false, true)) {
+ subscription.cancel();
multipartUploadHelper.failRequestsElegantly(futures, t, uploadId, returnFuture, putObjectRequest);
}
} else {
completeMultipartUploadIfFinish(asyncRequestBodyInFlight.decrementAndGet());
}
});
- synchronized (this) {
+ synchronized (subscription) {
subscription.request(1);
};
}
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java
index 4faf9d4a04b0..d2bd9d55dfb7 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java
@@ -17,7 +17,6 @@
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyList;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
@@ -25,11 +24,9 @@
import java.io.IOException;
import java.util.Collection;
-import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.function.Function;
import java.util.stream.Collectors;
@@ -39,9 +36,9 @@
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.ArgumentCaptor;
-import org.reactivestreams.Subscriber;
import org.reactivestreams.Subscription;
import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
@@ -60,7 +57,7 @@ public class KnownContentLengthAsyncRequestBodySubscriberTest {
private static final int TOTAL_NUM_PARTS = 4;
private static final String UPLOAD_ID = "1234";
private static RandomTempFile testFile;
-
+
private AsyncRequestBody asyncRequestBody;
private PutObjectRequest putObjectRequest;
private S3AsyncClient s3AsyncClient;
@@ -114,7 +111,7 @@ void validatePart_withPartSizeExceedingLimit_shouldFailRequest() {
void validateLastPartSize_withIncorrectSize_shouldFailRequest() {
long expectedLastPartSize = MPU_CONTENT_SIZE % PART_SIZE;
long incorrectLastPartSize = expectedLastPartSize + 1;
-
+
KnownContentLengthAsyncRequestBodySubscriber lastPartSubscriber = createSubscriber(createDefaultMpuRequestContext());
lastPartSubscriber.onSubscribe(subscription);
@@ -130,12 +127,12 @@ void validateLastPartSize_withIncorrectSize_shouldFailRequest() {
@Test
void validateTotalPartNum_receivedMoreParts_shouldFail() {
long expectedLastPartSize = MPU_CONTENT_SIZE % PART_SIZE;
-
+
KnownContentLengthAsyncRequestBodySubscriber lastPartSubscriber = createSubscriber(createDefaultMpuRequestContext());
lastPartSubscriber.onSubscribe(subscription);
for (int i = 0; i < TOTAL_NUM_PARTS - 1; i++) {
- AsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
+ ClosableAsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
when(multipartUploadHelper.sendIndividualUploadPartRequest(eq(UPLOAD_ID), any(), any(), any(), any()))
.thenReturn(CompletableFuture.completedFuture(null));
lastPartSubscriber.onNext(regularPart);
@@ -157,7 +154,7 @@ void validateLastPartSize_withCorrectSize_shouldNotFail() {
subscriber.onSubscribe(subscription);
for (int i = 0; i < TOTAL_NUM_PARTS - 1; i++) {
- AsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
+ ClosableAsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
when(multipartUploadHelper.sendIndividualUploadPartRequest(eq(UPLOAD_ID), any(), any(), any(), any()))
.thenReturn(CompletableFuture.completedFuture(null));
subscriber.onNext(regularPart);
@@ -175,7 +172,7 @@ void validateLastPartSize_withCorrectSize_shouldNotFail() {
void pause_withOngoingCompleteMpuFuture_shouldReturnTokenAndCancelFuture() {
CompletableFuture completeMpuFuture = new CompletableFuture<>();
int numExistingParts = 2;
-
+
S3ResumeToken resumeToken = testPauseScenario(numExistingParts, completeMpuFuture);
verifyResumeToken(resumeToken, numExistingParts);
@@ -187,7 +184,7 @@ void pause_withCompletedCompleteMpuFuture_shouldReturnNullToken() {
CompletableFuture completeMpuFuture =
CompletableFuture.completedFuture(CompleteMultipartUploadResponse.builder().build());
int numExistingParts = 2;
-
+
S3ResumeToken resumeToken = testPauseScenario(numExistingParts, completeMpuFuture);
assertThat(resumeToken).isNull();
@@ -196,15 +193,15 @@ void pause_withCompletedCompleteMpuFuture_shouldReturnNullToken() {
@Test
void pause_withUninitiatedCompleteMpuFuture_shouldReturnToken() {
int numExistingParts = 2;
-
+
S3ResumeToken resumeToken = testPauseScenario(numExistingParts, null);
verifyResumeToken(resumeToken, numExistingParts);
}
-
- private S3ResumeToken testPauseScenario(int numExistingParts,
+
+ private S3ResumeToken testPauseScenario(int numExistingParts,
CompletableFuture completeMpuFuture) {
- KnownContentLengthAsyncRequestBodySubscriber subscriber =
+ KnownContentLengthAsyncRequestBodySubscriber subscriber =
createSubscriber(createMpuRequestContextWithExistingParts(numExistingParts));
when(multipartUploadHelper.completeMultipartUpload(any(CompletableFuture.class), any(String.class),
@@ -246,14 +243,14 @@ private KnownContentLengthAsyncRequestBodySubscriber createSubscriber(MpuRequest
return new KnownContentLengthAsyncRequestBodySubscriber(mpuRequestContext, returnFuture, multipartUploadHelper);
}
- private AsyncRequestBody createMockAsyncRequestBody(long contentLength) {
- AsyncRequestBody mockBody = mock(AsyncRequestBody.class);
+ private ClosableAsyncRequestBody createMockAsyncRequestBody(long contentLength) {
+ ClosableAsyncRequestBody mockBody = mock(ClosableAsyncRequestBody.class);
when(mockBody.contentLength()).thenReturn(Optional.of(contentLength));
return mockBody;
}
- private AsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLength() {
- AsyncRequestBody mockBody = mock(AsyncRequestBody.class);
+ private ClosableAsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLength() {
+ ClosableAsyncRequestBody mockBody = mock(ClosableAsyncRequestBody.class);
when(mockBody.contentLength()).thenReturn(Optional.empty());
return mockBody;
}
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
index 90e14dcff2dd..ee3bf01c3d54 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
@@ -18,14 +18,20 @@
import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
import static com.github.tomakehurst.wiremock.client.WireMock.anyUrl;
import static com.github.tomakehurst.wiremock.client.WireMock.delete;
+import static com.github.tomakehurst.wiremock.client.WireMock.matching;
import static com.github.tomakehurst.wiremock.client.WireMock.post;
import static com.github.tomakehurst.wiremock.client.WireMock.put;
+import static com.github.tomakehurst.wiremock.client.WireMock.putRequestedFor;
import static com.github.tomakehurst.wiremock.client.WireMock.stubFor;
import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo;
+import static com.github.tomakehurst.wiremock.client.WireMock.verify;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import com.github.tomakehurst.wiremock.client.ResponseDefinitionBuilder;
+import com.github.tomakehurst.wiremock.http.Fault;
import com.github.tomakehurst.wiremock.junit5.WireMockRuntimeInfo;
import com.github.tomakehurst.wiremock.junit5.WireMockTest;
+import com.github.tomakehurst.wiremock.stubbing.Scenario;
import io.reactivex.rxjava3.core.Flowable;
import java.io.InputStream;
import java.net.URI;
@@ -46,20 +52,17 @@
import org.reactivestreams.Subscriber;
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
import software.amazon.awssdk.core.async.BlockingInputStreamAsyncRequestBody;
-import software.amazon.awssdk.core.async.SdkPublisher;
-import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
import software.amazon.awssdk.services.s3.model.S3Exception;
-import software.amazon.awssdk.utils.async.SimplePublisher;
@WireMockTest
-@Timeout(100)
+@Timeout(120)
public class S3MultipartClientPutObjectWiremockTest {
private static final String BUCKET = "Example-Bucket";
@@ -71,25 +74,21 @@ public class S3MultipartClientPutObjectWiremockTest {
+ "";
private S3AsyncClient s3AsyncClient;
- public static Stream invalidAsyncRequestBodies() {
+ public static Stream retryableErrorTestCase() {
return Stream.of(
- Arguments.of("knownContentLength_nullPartSize", new TestPublisherWithIncorrectSplitImpl(20L, null),
- "Content length is missing on the AsyncRequestBody for part number"),
- Arguments.of("unknownContentLength_nullPartSize", new TestPublisherWithIncorrectSplitImpl(null, null),
- "Content length is missing on the AsyncRequestBody for part number"),
- Arguments.of("knownContentLength_partSizeIncorrect", new TestPublisherWithIncorrectSplitImpl(20L, 11L),
- "Content length must not be greater than part size"),
- Arguments.of("unknownContentLength_partSizeIncorrect", new TestPublisherWithIncorrectSplitImpl(null, 11L),
- "Content length must not be greater than part size"),
- Arguments.of("knownContentLength_sendMoreParts", new TestPublisherWithIncorrectSplitImpl(20L, 10L, 3),
- "The number of parts divided is not equal to the expected number of parts"),
- Arguments.of("knownContentLength_sendFewerParts", new TestPublisherWithIncorrectSplitImpl(20L, 10L, 1),
- "The number of parts divided is not equal to the expected number of parts"));
+ Arguments.of("unknownContentLength_failOfConnectionReset_shouldRetry", null,
+ aResponse().withFault(Fault.CONNECTION_RESET_BY_PEER)),
+ Arguments.of("unknownContentLength_failOf500_shouldRetry", null,
+ aResponse().withStatus(500)),
+ Arguments.of("knownContentLength_failOfConnectionReset_shouldRetry", 20L,
+ aResponse().withFault(Fault.CONNECTION_RESET_BY_PEER)),
+ Arguments.of("knownContentLength_failOf500_shouldRetry", 20L,
+ aResponse().withStatus(500))
+ );
}
@BeforeEach
public void setup(WireMockRuntimeInfo wiremock) {
- stubFailedPutObjectCalls();
s3AsyncClient = S3AsyncClient.builder()
.region(Region.US_EAST_1)
.endpointOverride(URI.create("http://localhost:" + wiremock.getHttpPort()))
@@ -101,22 +100,17 @@ public void setup(WireMockRuntimeInfo wiremock) {
.build();
}
- private void stubFailedPutObjectCalls() {
+ private void stubPutObject404Calls() {
stubFor(post(anyUrl()).willReturn(aResponse().withStatus(200).withBody(CREATE_MULTIPART_PAYLOAD)));
stubFor(put(anyUrl()).willReturn(aResponse().withStatus(404)));
stubFor(put(urlEqualTo("/Example-Bucket/Example-Object?partNumber=1&uploadId=string")).willReturn(aResponse().withStatus(200)));
stubFor(delete(anyUrl()).willReturn(aResponse().withStatus(200)));
}
- private void stubSuccessfulPutObjectCalls() {
- stubFor(post(anyUrl()).willReturn(aResponse().withStatus(200).withBody(CREATE_MULTIPART_PAYLOAD)));
- stubFor(put(anyUrl()).willReturn(aResponse().withStatus(200)));
- }
-
-
// https://github.com/aws/aws-sdk-java-v2/issues/4801
@Test
void uploadWithUnknownContentLength_onePartFails_shouldCancelUpstream() {
+ stubPutObject404Calls();
BlockingInputStreamAsyncRequestBody blockingInputStreamAsyncRequestBody = AsyncRequestBody.forBlockingInputStream(null);
CompletableFuture putObjectResponse = s3AsyncClient.putObject(
r -> r.bucket(BUCKET).key(KEY), blockingInputStreamAsyncRequestBody);
@@ -132,6 +126,7 @@ void uploadWithUnknownContentLength_onePartFails_shouldCancelUpstream() {
@Test
void uploadWithKnownContentLength_onePartFails_shouldCancelUpstream() {
+ stubPutObject404Calls();
BlockingInputStreamAsyncRequestBody blockingInputStreamAsyncRequestBody =
AsyncRequestBody.forBlockingInputStream(1024L * 20); // must be larger than the buffer used in
// InputStreamConsumingPublisher to trigger the error
@@ -147,19 +142,56 @@ void uploadWithKnownContentLength_onePartFails_shouldCancelUpstream() {
assertThatThrownBy(() -> putObjectResponse.join()).hasRootCauseInstanceOf(S3Exception.class);
}
- @ParameterizedTest(name = "{index} {0}")
- @MethodSource("invalidAsyncRequestBodies")
- void uploadWithIncorrectAsyncRequestBodySplit_contentLengthMismatch_shouldThrowException(String description,
- TestPublisherWithIncorrectSplitImpl asyncRequestBody,
- String errorMsg) {
- stubSuccessfulPutObjectCalls();
- CompletableFuture putObjectResponse = s3AsyncClient.putObject(
- r -> r.bucket(BUCKET).key(KEY), asyncRequestBody);
+ @ParameterizedTest
+ @MethodSource("retryableErrorTestCase")
+ void mpu_partsFailOfRetryableError_shouldRetry(String description,
+ Long contentLength,
+ ResponseDefinitionBuilder responseDefinitionBuilder) {
+ stubUploadPartFailsInitialAttemptCalls(responseDefinitionBuilder);
+ List buffers = new ArrayList<>();
+ buffers.add(SdkBytes.fromUtf8String(RandomStringUtils.randomAscii(10)).asByteBuffer());
+ buffers.add(SdkBytes.fromUtf8String(RandomStringUtils.randomAscii(10)).asByteBuffer());
+ AsyncRequestBody asyncRequestBody = new AsyncRequestBody() {
+ @Override
+ public Optional contentLength() {
+ return Optional.ofNullable(contentLength);
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ Flowable.fromIterable(buffers).subscribe(s);
+ }
+ };
+
+ s3AsyncClient.putObject(b -> b.bucket(BUCKET).key(KEY), asyncRequestBody).join();
+
+ verify(2, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(1))));
+ verify(2, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(2))));
+ }
+
- assertThatThrownBy(() -> putObjectResponse.join()).hasMessageContaining(errorMsg)
- .hasRootCauseInstanceOf(SdkClientException.class);
+ private void stubUploadPartFailsInitialAttemptCalls(ResponseDefinitionBuilder responseDefinitionBuilder) {
+ stubFor(post(anyUrl()).willReturn(aResponse().withStatus(200).withBody(CREATE_MULTIPART_PAYLOAD)));
+ stubUploadFailsInitialAttemptCalls(1, responseDefinitionBuilder);
+ stubUploadFailsInitialAttemptCalls(2, responseDefinitionBuilder);
}
+ private void stubUploadFailsInitialAttemptCalls(int partNumber, ResponseDefinitionBuilder responseDefinitionBuilder) {
+ stubFor(put(anyUrl())
+ .withQueryParam("partNumber", matching(String.valueOf(partNumber)))
+ .inScenario(String.valueOf(partNumber))
+ .whenScenarioStateIs(Scenario.STARTED)
+ .willReturn(responseDefinitionBuilder)
+ .willSetStateTo("SecondAttempt" + partNumber));
+
+ stubFor(put(anyUrl())
+ .withQueryParam("partNumber", matching(String.valueOf(partNumber)))
+ .inScenario(String.valueOf(partNumber))
+ .whenScenarioStateIs("SecondAttempt" + partNumber)
+ .willReturn(aResponse().withStatus(200)));
+ }
+
+
private InputStream createUnlimitedInputStream() {
return new InputStream() {
@Override
@@ -168,65 +200,5 @@ public int read() {
}
};
}
-
- private static class TestPublisherWithIncorrectSplitImpl implements AsyncRequestBody {
- private SimplePublisher simplePublisher = new SimplePublisher<>();
- private Long totalSize;
- private Long partSize;
- private Integer numParts;
-
- private TestPublisherWithIncorrectSplitImpl(Long totalSize, Long partSize) {
- this.totalSize = totalSize;
- this.partSize = partSize;
- }
-
- private TestPublisherWithIncorrectSplitImpl(Long totalSize, long partSize, int numParts) {
- this.totalSize = totalSize;
- this.partSize = partSize;
- this.numParts = numParts;
- }
-
- @Override
- public Optional contentLength() {
- return Optional.ofNullable(totalSize);
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- simplePublisher.subscribe(s);
- }
-
- @Override
- public SdkPublisher split(AsyncRequestBodySplitConfiguration splitConfiguration) {
- List requestBodies = new ArrayList<>();
- int numAsyncRequestBodies = numParts == null ? 1 : numParts;
- for (int i = 0; i < numAsyncRequestBodies; i++) {
- requestBodies.add(new TestAsyncRequestBody(partSize));
- }
-
- return SdkPublisher.adapt(Flowable.fromArray(requestBodies.toArray(new AsyncRequestBody[requestBodies.size()])));
- }
- }
-
- private static class TestAsyncRequestBody implements AsyncRequestBody {
- private Long partSize;
- private SimplePublisher simplePublisher = new SimplePublisher<>();
-
- public TestAsyncRequestBody(Long partSize) {
- this.partSize = partSize;
- }
-
- @Override
- public Optional contentLength() {
- return Optional.ofNullable(partSize);
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- simplePublisher.subscribe(s);
- simplePublisher.send(ByteBuffer.wrap(
- RandomStringUtils.randomAscii(Math.toIntExact(partSize)).getBytes()));
- simplePublisher.complete();
- }
- }
}
+
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
index 972f0b86241a..e5c96bc6abed 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
@@ -31,6 +31,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
+import java.nio.ByteBuffer;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
@@ -46,6 +47,7 @@
import org.reactivestreams.Subscriber;
import org.reactivestreams.Subscription;
import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.BlockingInputStreamAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.core.exception.SdkClientException;
@@ -114,14 +116,14 @@ void upload_blockingInputStream_shouldInOrder() throws FileNotFoundException {
@Test
void uploadObject_withMissingContentLength_shouldFailRequest() {
- AsyncRequestBody asyncRequestBody = createMockAsyncRequestBodyWithEmptyContentLength();
+ ClosableAsyncRequestBody asyncRequestBody = createMockAsyncRequestBodyWithEmptyContentLength();
CompletableFuture future = setupAndTriggerUploadFailure(asyncRequestBody);
verifyFailureWithMessage(future, "Content length is missing on the AsyncRequestBody for part number");
}
@Test
void uploadObject_withPartSizeExceedingLimit_shouldFailRequest() {
- AsyncRequestBody asyncRequestBody = createMockAsyncRequestBody(PART_SIZE + 1);
+ ClosableAsyncRequestBody asyncRequestBody = createMockAsyncRequestBody(PART_SIZE + 1);
CompletableFuture future = setupAndTriggerUploadFailure(asyncRequestBody);
verifyFailureWithMessage(future, "Content length must not be greater than part size");
}
@@ -139,27 +141,27 @@ private List createCompletedParts(int totalNumParts) {
.collect(Collectors.toList());
}
- private AsyncRequestBody createMockAsyncRequestBody(long contentLength) {
- AsyncRequestBody mockBody = mock(AsyncRequestBody.class);
+ private ClosableAsyncRequestBody createMockAsyncRequestBody(long contentLength) {
+ ClosableAsyncRequestBody mockBody = mock(ClosableAsyncRequestBody.class);
when(mockBody.contentLength()).thenReturn(Optional.of(contentLength));
return mockBody;
}
- private AsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLength() {
- AsyncRequestBody mockBody = mock(AsyncRequestBody.class);
+ private ClosableAsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLength() {
+ ClosableAsyncRequestBody mockBody = mock(ClosableAsyncRequestBody.class);
when(mockBody.contentLength()).thenReturn(Optional.empty());
return mockBody;
}
- private CompletableFuture setupAndTriggerUploadFailure(AsyncRequestBody asyncRequestBody) {
- SdkPublisher mockPublisher = mock(SdkPublisher.class);
- when(asyncRequestBody.split(any(Consumer.class))).thenReturn(mockPublisher);
+ private CompletableFuture setupAndTriggerUploadFailure(ClosableAsyncRequestBody asyncRequestBody) {
+ SdkPublisher mockPublisher = mock(SdkPublisher.class);
+ when(asyncRequestBody.splitV2(any(Consumer.class))).thenReturn(mockPublisher);
- ArgumentCaptor> subscriberCaptor = ArgumentCaptor.forClass(Subscriber.class);
+ ArgumentCaptor> subscriberCaptor = ArgumentCaptor.forClass(Subscriber.class);
CompletableFuture future = helper.uploadObject(createPutObjectRequest(), asyncRequestBody);
verify(mockPublisher).subscribe(subscriberCaptor.capture());
- Subscriber subscriber = subscriberCaptor.getValue();
+ Subscriber subscriber = subscriberCaptor.getValue();
Subscription subscription = mock(Subscription.class);
subscriber.onSubscribe(subscription);
diff --git a/services/s3/src/test/resources/log4j2.properties b/services/s3/src/test/resources/log4j2.properties
index ad5cb8e79a64..8f3afbf09abe 100644
--- a/services/s3/src/test/resources/log4j2.properties
+++ b/services/s3/src/test/resources/log4j2.properties
@@ -35,4 +35,4 @@ rootLogger.appenderRef.stdout.ref = ConsoleAppender
#logger.apache.level = debug
#
#logger.netty.name = io.netty.handler.logging
-#logger.netty.level = debug
+#logger.netty.level = debug
\ No newline at end of file
From 3c91cc9bcd3f179270be629267ab05ff76f6d327 Mon Sep 17 00:00:00 2001
From: Zoe Wang <33073555+zoewangg@users.noreply.github.com>
Date: Fri, 15 Aug 2025 10:28:21 -0700
Subject: [PATCH 2/5] Fix build and add more tests
---
.../listener/AsyncRequestBodyListener.java | 12 ++
.../internal/async/SplittingPublisher.java | 39 +++---
.../ClosableAsyncRequestBodyAdaptorTest.java | 118 +++++++++++++++++-
...ntentLengthAsyncRequestBodySubscriber.java | 6 +-
.../multipart/MultipartUploadHelper.java | 7 +-
.../UploadWithUnknownContentLengthHelper.java | 4 +-
6 files changed, 159 insertions(+), 27 deletions(-)
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
index a37b226d4bc3..e0228b896d26 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
@@ -23,6 +23,7 @@
import software.amazon.awssdk.annotations.SdkProtectedApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.utils.Logger;
import software.amazon.awssdk.utils.Validate;
@@ -76,6 +77,17 @@ public SdkPublisher split(Consumer splitV2(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ return delegate.splitV2(splitConfiguration);
+ }
+
+ @Override
+ public SdkPublisher splitV2(
+ Consumer splitConfiguration) {
+ return delegate.splitV2(splitConfiguration);
+ }
+
@Override
public void subscribe(Subscriber super ByteBuffer> s) {
invoke(() -> listener.publisherSubscribe(s), "publisherSubscribe");
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
index 1275d5e92fb3..c44c728746a3 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
@@ -65,7 +65,7 @@ public SplittingPublisher(AsyncRequestBody asyncRequestBody,
Validate.isTrue(bufferSizeInBytes >= chunkSizeInBytes,
"bufferSizeInBytes must be larger than or equal to " +
- "chunkSizeInBytes if the content length is unknown");
+ "chunkSizeInBytes");
}
@Override
@@ -77,7 +77,10 @@ public void subscribe(Subscriber super ClosableAsyncRequestBody> downstreamSub
private class SplittingSubscriber implements Subscriber {
private Subscription upstreamSubscription;
private final Long upstreamSize;
- private final AtomicInteger chunkNumber = new AtomicInteger(0);
+ /**
+ * 1 based index number for each part/chunk
+ */
+ private final AtomicInteger partNumber = new AtomicInteger(1);
private volatile DownstreamBody currentBody;
private final AtomicBoolean hasOpenUpstreamDemand = new AtomicBoolean(false);
private final AtomicLong dataBuffered = new AtomicLong(0);
@@ -97,15 +100,15 @@ public void onSubscribe(Subscription s) {
this.upstreamSubscription = s;
this.currentBody =
initializeNextDownstreamBody(upstreamSize != null, calculateChunkSize(upstreamSize),
- chunkNumber.get());
+ partNumber.get());
// We need to request subscription *after* we set currentBody because onNext could be invoked right away.
upstreamSubscription.request(1);
}
- private DownstreamBody initializeNextDownstreamBody(boolean contentLengthKnown, long chunkSize, int chunkNumber) {
+ private DownstreamBody initializeNextDownstreamBody(boolean contentLengthKnown, long chunkSize, int partNumber) {
currentBodySent.set(false);
- log.debug(() -> "initializing next downstream body " + chunkNumber);
- return new DownstreamBody(contentLengthKnown, chunkSize, chunkNumber);
+ log.debug(() -> "initializing next downstream body " + partNumber);
+ return new DownstreamBody(contentLengthKnown, chunkSize, partNumber);
}
@Override
@@ -155,7 +158,7 @@ public void onNext(ByteBuffer byteBuffer) {
private void completeCurrentBodyAndCreateNewIfNeeded(ByteBuffer byteBuffer) {
completeCurrentBodyAndDeliver();
- int currentChunk = chunkNumber.incrementAndGet();
+ int nextChunk = partNumber.incrementAndGet();
boolean shouldCreateNewDownstreamRequestBody;
Long dataRemaining = totalDataRemaining();
@@ -167,7 +170,7 @@ private void completeCurrentBodyAndCreateNewIfNeeded(ByteBuffer byteBuffer) {
if (shouldCreateNewDownstreamRequestBody) {
long chunkSize = calculateChunkSize(dataRemaining);
- currentBody = initializeNextDownstreamBody(upstreamSize != null, chunkSize, currentChunk);
+ currentBody = initializeNextDownstreamBody(upstreamSize != null, chunkSize, nextChunk);
}
}
@@ -181,7 +184,7 @@ private int amountRemainingInChunk() {
*/
private void completeCurrentBodyAndDeliver() {
if (currentBodySent.compareAndSet(false, true)) {
- log.debug(() -> "completeCurrentBody for chunk " + currentBody.chunkNumber);
+ log.debug(() -> "completeCurrentBody for chunk " + currentBody.partNumber);
// For unknown content length, we always create a new DownstreamBody because we don't know if there is data
// left or not, so we need to only send the body if there is actually data
long bufferedLength = currentBody.bufferedLength;
@@ -218,7 +221,7 @@ public void onError(Throwable t) {
}
private void sendCurrentBody(DownstreamBody body) {
- log.debug(() -> "sendCurrentBody for chunk " + body.chunkNumber);
+ log.debug(() -> "sendCurrentBody for chunk " + body.partNumber);
downstreamPublisher.send(body).exceptionally(t -> {
downstreamPublisher.error(t);
upstreamSubscription.cancel();
@@ -252,7 +255,7 @@ private Long totalDataRemaining() {
if (upstreamSize == null) {
return null;
}
- return upstreamSize - (chunkNumber.get() * chunkSizeInBytes);
+ return upstreamSize - ((partNumber.get() - 1) * chunkSizeInBytes);
}
/**
@@ -267,15 +270,15 @@ private final class DownstreamBody implements ClosableAsyncRequestBody {
*/
private final long maxLength;
private final Long totalLength;
- private final int chunkNumber;
+ private final int partNumber;
private volatile long bufferedLength = 0;
private volatile ByteBuffersAsyncRequestBody delegate;
private final List buffers = new ArrayList<>();
- private DownstreamBody(boolean contentLengthKnown, long maxLength, int chunkNumber) {
+ private DownstreamBody(boolean contentLengthKnown, long maxLength, int partNumber) {
this.totalLength = contentLengthKnown ? maxLength : null;
this.maxLength = maxLength;
- this.chunkNumber = chunkNumber;
+ this.partNumber = partNumber;
}
@Override
@@ -284,7 +287,7 @@ public Optional contentLength() {
}
public void send(ByteBuffer data) {
- log.debug(() -> String.format("Sending bytebuffer %s to chunk %d", data, chunkNumber));
+ log.debug(() -> String.format("Sending bytebuffer %s to chunk %d", data, partNumber));
int length = data.remaining();
bufferedLength += length;
addDataBuffered(length);
@@ -292,13 +295,13 @@ public void send(ByteBuffer data) {
}
public void complete() {
- log.debug(() -> "Received complete() for chunk number: " + chunkNumber + " length " + bufferedLength);
+ log.debug(() -> "Received complete() for chunk number: " + partNumber + " length " + bufferedLength);
this.delegate = ByteBuffersAsyncRequestBody.of(buffers);
}
@Override
public void subscribe(Subscriber super ByteBuffer> s) {
- log.debug(() -> "Subscribe for chunk number: " + chunkNumber + " length " + bufferedLength);
+ log.debug(() -> "Subscribe for chunk number: " + partNumber + " length " + bufferedLength);
delegate.subscribe(s);
}
@@ -311,7 +314,7 @@ private void addDataBuffered(long length) {
@Override
public void close() {
- log.debug(() -> "Closing current body " + chunkNumber);
+ log.debug(() -> "Closing current body " + partNumber);
delegate.close();
addDataBuffered(-bufferedLength);
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
index 7b633c1af1d6..1a9a3ffc833b 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
@@ -20,26 +20,43 @@
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
import io.reactivex.Flowable;
import io.reactivex.FlowableSubscriber;
+import io.reactivex.internal.observers.BiConsumerSingleObserver;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Observable;
+import java.util.Observer;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;
+import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
import software.amazon.awssdk.core.exception.NonRetryableException;
public class ClosableAsyncRequestBodyAdaptorTest {
+ private ClosableAsyncRequestBody closableAsyncRequestBody;
- @Test
- void resubscribe_shouldThrowException() {
- ClosableAsyncRequestBody closableAsyncRequestBody = Mockito.mock(ClosableAsyncRequestBody.class);
+ @BeforeEach
+ public void setup() {
+ closableAsyncRequestBody =Mockito.mock(ClosableAsyncRequestBody.class);
Mockito.when(closableAsyncRequestBody.doAfterOnComplete(any(Runnable.class))).thenReturn(closableAsyncRequestBody);
Mockito.when(closableAsyncRequestBody.doAfterOnCancel(any(Runnable.class))).thenReturn(closableAsyncRequestBody);
Mockito.when(closableAsyncRequestBody.doAfterOnError(any(Consumer.class))).thenReturn(closableAsyncRequestBody);
+ }
+ @Test
+ void resubscribe_shouldThrowException() {
ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(closableAsyncRequestBody);
Subscriber subscriber = Mockito.mock(Subscriber.class);
adaptor.subscribe(subscriber);
@@ -55,4 +72,99 @@ void resubscribe_shouldThrowException() {
.hasMessageContaining("A retry was attempted");
}
+ @Test
+ void onComplete_shouldCloseAsyncRequestBody() {
+ TestClosableAsyncRequestBody asyncRequestBody = new TestClosableAsyncRequestBody();
+ ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(asyncRequestBody);
+ CompletableFuture future = new CompletableFuture<>();
+ Subscriber subscriber = new ByteArrayAsyncResponseTransformer.BaosSubscriber(future);
+ adaptor.subscribe(subscriber);
+ assertThat(asyncRequestBody.closeInvoked).isTrue();
+ }
+
+ @Test
+ void cancel_shouldCloseAsyncRequestBody() {
+ TestClosableAsyncRequestBody asyncRequestBody = new TestClosableAsyncRequestBody();
+ ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(asyncRequestBody);
+ Subscriber subscriber = new Subscriber() {
+ @Override
+ public void onSubscribe(Subscription s) {
+ s.cancel();
+ }
+
+ @Override
+ public void onNext(ByteBuffer byteBuffer) {
+ }
+
+ @Override
+ public void onError(Throwable t) {
+ }
+
+ @Override
+ public void onComplete() {
+ }
+ };
+ adaptor.subscribe(subscriber);
+ assertThat(asyncRequestBody.closeInvoked).isTrue();
+ }
+
+ @Test
+ void onError_shouldCloseAsyncRequestBody() {
+ OnErrorClosableAsyncRequestBody asyncRequestBody = new OnErrorClosableAsyncRequestBody();
+ ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(asyncRequestBody);
+ CompletableFuture future = new CompletableFuture<>();
+ Subscriber subscriber = new ByteArrayAsyncResponseTransformer.BaosSubscriber(future);
+ adaptor.subscribe(subscriber);
+ assertThat(asyncRequestBody.closeInvoked).isTrue();
+ }
+
+
+ private static class TestClosableAsyncRequestBody implements ClosableAsyncRequestBody {
+ private boolean closeInvoked;
+
+ @Override
+ public Optional contentLength() {
+ return Optional.empty();
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ Flowable.just(ByteBuffer.wrap("foo bar".getBytes(StandardCharsets.UTF_8)))
+ .subscribe(s);
+ }
+
+ @Override
+ public void close() {
+ closeInvoked = true;
+ }
+ }
+
+ private static class OnErrorClosableAsyncRequestBody implements ClosableAsyncRequestBody {
+ private boolean closeInvoked;
+
+ @Override
+ public Optional contentLength() {
+ return Optional.empty();
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ s.onSubscribe(new Subscription() {
+ @Override
+ public void request(long n) {
+ s.onError(new IllegalStateException("foobar"));
+ }
+
+ @Override
+ public void cancel() {
+
+ }
+ });
+ }
+
+ @Override
+ public void close() {
+ closeInvoked = true;
+ }
+ }
}
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
index 92ef401e830f..527119415ea6 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
@@ -151,6 +151,10 @@ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
}
int currentPartNum = partNumber.getAndIncrement();
+
+ log.debug(() -> String.format("Received asyncRequestBody for part number %d with length %s", currentPartNum,
+ asyncRequestBody.contentLength()));
+
if (existingParts.containsKey(currentPartNum)) {
asyncRequestBody.subscribe(new CancelledSubscriber<>());
asyncRequestBody.contentLength().ifPresent(progressListener::subscriberOnNext);
@@ -210,7 +214,7 @@ private Optional validatePart(AsyncRequestBody asyncRequestB
}
if (currentPartSize != partSize) {
- return Optional.of(contentLengthMismatchForPart(partSize, currentPartSize));
+ return Optional.of(contentLengthMismatchForPart(partSize, currentPartSize, currentPartNum));
}
return Optional.empty();
}
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/MultipartUploadHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/MultipartUploadHelper.java
index d25d5b6fa7fa..d7c988c16e55 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/MultipartUploadHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/MultipartUploadHelper.java
@@ -162,11 +162,12 @@ static SdkClientException contentLengthMissingForPart(int currentPartNum) {
return SdkClientException.create("Content length is missing on the AsyncRequestBody for part number " + currentPartNum);
}
- static SdkClientException contentLengthMismatchForPart(long expected, long actual) {
+ static SdkClientException contentLengthMismatchForPart(long expected, long actual, int partNum) {
return SdkClientException.create(String.format("Content length must not be greater than "
- + "part size. Expected: %d, Actual: %d",
+ + "part size. Expected: %d, Actual: %d, partNum: %d",
expected,
- actual));
+ actual,
+ partNum));
}
static SdkClientException partNumMismatch(int expectedNumParts, int actualNumParts) {
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
index 31a6e685db17..73a0c0d39cfc 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
@@ -159,7 +159,7 @@ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
return;
}
int currentPartNum = partNumber.incrementAndGet();
- log.debug(() -> String.format("Received asyncRequestBody for part number %d with length %d", currentPartNum,
+ log.debug(() -> String.format("Received asyncRequestBody for part number %d with length %s", currentPartNum,
asyncRequestBody.contentLength()));
asyncRequestBodyInFlight.incrementAndGet();
@@ -209,7 +209,7 @@ private Optional validatePart(AsyncRequestBody asyncRequestB
Long contentLengthCurrentPart = contentLength.get();
if (contentLengthCurrentPart > partSizeInBytes) {
- return Optional.of(contentLengthMismatchForPart(partSizeInBytes, contentLengthCurrentPart));
+ return Optional.of(contentLengthMismatchForPart(partSizeInBytes, contentLengthCurrentPart, currentPartNum));
}
return Optional.empty();
From 004733bf21a91dd216cb2fb1353879a679f88b17 Mon Sep 17 00:00:00 2001
From: Zoe Wang <33073555+zoewangg@users.noreply.github.com>
Date: Wed, 20 Aug 2025 11:08:53 -0700
Subject: [PATCH 3/5] Refactoring based on API surface are review
---
.../awssdk/core/async/AsyncRequestBody.java | 31 ++-
.../BufferedSplittableAsyncRequestBody.java | 59 ++++++
.../listener/AsyncRequestBodyListener.java | 8 +-
.../async/ByteBuffersAsyncRequestBody.java | 10 +-
.../ClosableAsyncRequestBodyAdaptor.java | 67 -------
.../internal/async/FileAsyncRequestBody.java | 16 +-
.../NonRetryableSubAsyncRequestBody.java | 125 ++++++++++++
.../async/RetryableSubAsyncRequestBody.java | 165 ++++++++++++++++
.../internal/async/SplittingPublisher.java | 182 ++++++++----------
.../internal/async/SubAsyncRequestBody.java | 65 +++++++
.../SubAsyncRequestBodyConfiguration.java | 140 ++++++++++++++
.../core/async/AsyncRequestBodyTest.java | 2 +-
.../ClosableAsyncRequestBodyAdaptorTest.java | 170 ----------------
.../async/SplittingPublisherTest.java | 96 +++++++--
.../src/test/resources/log4j2.properties | 4 +-
.../UploadWithKnownContentLengthHelper.java | 4 +-
.../UploadWithUnknownContentLengthHelper.java | 33 +++-
...3MultipartClientPutObjectWiremockTest.java | 55 ++++--
...oadWithUnknownContentLengthHelperTest.java | 7 +-
.../s3/src/test/resources/log4j2.properties | 5 +-
20 files changed, 828 insertions(+), 416 deletions(-)
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java
delete mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java
create mode 100644 core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBodyConfiguration.java
delete mode 100644 core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
index 22ac529aafa9..f5282d71a736 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
@@ -33,7 +33,6 @@
import software.amazon.awssdk.annotations.SdkPublicApi;
import software.amazon.awssdk.core.FileRequestBodyConfiguration;
import software.amazon.awssdk.core.internal.async.ByteBuffersAsyncRequestBody;
-import software.amazon.awssdk.core.internal.async.ClosableAsyncRequestBodyAdaptor;
import software.amazon.awssdk.core.internal.async.FileAsyncRequestBody;
import software.amazon.awssdk.core.internal.async.InputStreamWithExecutorAsyncRequestBody;
import software.amazon.awssdk.core.internal.async.SplittingPublisher;
@@ -501,22 +500,23 @@ static AsyncRequestBody empty() {
return fromBytes(new byte[0]);
}
-
/**
* Converts this {@link AsyncRequestBody} to a publisher of {@link AsyncRequestBody}s, each of which publishes a specific
* portion of the original data, based on the provided {@link AsyncRequestBodySplitConfiguration}. The default chunk size
* is 2MB and the default buffer size is 8MB.
*
*
- * Each divided {@link AsyncRequestBody} is sent after the entire content for that chunk is buffered.
+ * By default, if content length of this {@link AsyncRequestBody} is present, each divided {@link AsyncRequestBody} is
+ * delivered to the subscriber right after it's initialized. On the other hand, if content length is null, it is sent after
+ * the entire content for that chunk is buffered. In this case, the configured {@code maxMemoryUsageInBytes} must be larger
+ * than or equal to {@code chunkSizeInBytes}. Note that this behavior may be different if a specific implementation of this
+ * interface overrides this method.
*
* @see AsyncRequestBodySplitConfiguration
- * @deprecated Use {@link #splitV2(AsyncRequestBodySplitConfiguration)} instead.
*/
- @Deprecated
default SdkPublisher split(AsyncRequestBodySplitConfiguration splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
- return splitV2(splitConfiguration).map(body -> new ClosableAsyncRequestBodyAdaptor(body));
+ return new SplittingPublisher(this, splitConfiguration, false).map(r -> r);
}
/**
@@ -525,17 +525,18 @@ default SdkPublisher split(AsyncRequestBodySplitConfiguration
* size is 2MB and the default buffer size is 8MB.
*
*
- * Each divided {@link ClosableAsyncRequestBody} is sent after the entire content for that chunk is buffered. This behavior
- * may be different if a specific implementation of this interface overrides this method.
+ * The default implementation behaves the same as {@link #split(AsyncRequestBodySplitConfiguration)}. This behavior may
+ * vary in different implementations.
*
*
- * Each {@link ClosableAsyncRequestBody} MUST be closed by the user when it is ready to be disposed.
+ * Caller is responsible for closing {@link ClosableAsyncRequestBody} when it is ready to be disposed to release any
+ * resources.
*
* @see AsyncRequestBodySplitConfiguration
*/
- default SdkPublisher splitV2(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ default SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
- return new SplittingPublisher(this, splitConfiguration);
+ return new SplittingPublisher(this, splitConfiguration, false);
}
/**
@@ -543,9 +544,7 @@ default SdkPublisher splitV2(AsyncRequestBodySplitConf
* avoiding the need to create one manually via {@link AsyncRequestBodySplitConfiguration#builder()}.
*
* @see #split(AsyncRequestBodySplitConfiguration)
- * @deprecated Use {@link #splitV2(Consumer)} instead.
*/
- @Deprecated
default SdkPublisher split(Consumer splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
return split(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
@@ -555,12 +554,12 @@ default SdkPublisher split(Consumer splitV2(
+ default SdkPublisher splitClosable(
Consumer splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
- return splitV2(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
+ return splitClosable(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
}
@SdkProtectedApi
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java
new file mode 100644
index 000000000000..21d701109d2c
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.async;
+
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import org.reactivestreams.Subscriber;
+import software.amazon.awssdk.annotations.SdkPublicApi;
+import software.amazon.awssdk.core.internal.async.SplittingPublisher;
+
+/**
+ * An {@link AsyncRequestBody} decorator that can be split into buffered sub {@link AsyncRequestBody}s. Each sub
+ * {@link AsyncRequestBody} can be retried/resubscribed if all data has been successfully been published to first subscriber.
+ */
+@SdkPublicApi
+public final class BufferedSplittableAsyncRequestBody implements AsyncRequestBody {
+ private final AsyncRequestBody delegate;
+
+ private BufferedSplittableAsyncRequestBody(AsyncRequestBody delegate) {
+ this.delegate = delegate;
+ }
+
+ public static BufferedSplittableAsyncRequestBody create(AsyncRequestBody delegate) {
+ return new BufferedSplittableAsyncRequestBody(delegate);
+ }
+
+ @Override
+ public Optional contentLength() {
+ return delegate.contentLength();
+ }
+
+ @Override
+ public SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ return new SplittingPublisher(this, splitConfiguration, true);
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ delegate.subscribe(s);
+ }
+
+ @Override
+ public String body() {
+ return delegate.body();
+ }
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
index e0228b896d26..71a6f87a4288 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
@@ -78,14 +78,14 @@ public SdkPublisher split(Consumer splitV2(AsyncRequestBodySplitConfiguration splitConfiguration) {
- return delegate.splitV2(splitConfiguration);
+ public SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ return delegate.splitClosable(splitConfiguration);
}
@Override
- public SdkPublisher splitV2(
+ public SdkPublisher splitClosable(
Consumer splitConfiguration) {
- return delegate.splitV2(splitConfiguration);
+ return delegate.splitClosable(splitConfiguration);
}
@Override
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
index 1ae49d0dfdde..e1f6d9e8cb05 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
@@ -123,6 +123,10 @@ public String body() {
return BodyType.BYTES.getName();
}
+ public static ByteBuffersAsyncRequestBody of(List buffers, long length) {
+ return new ByteBuffersAsyncRequestBody(Mimetype.MIMETYPE_OCTET_STREAM, length, buffers);
+ }
+
public static ByteBuffersAsyncRequestBody of(List buffers) {
long length = buffers.stream()
.mapToLong(ByteBuffer::remaining)
@@ -131,7 +135,11 @@ public static ByteBuffersAsyncRequestBody of(List buffers) {
}
public static ByteBuffersAsyncRequestBody of(ByteBuffer... buffers) {
- return of(Arrays.asList(buffers));
+ List byteBuffers = Arrays.asList(buffers);
+ long length = byteBuffers.stream()
+ .mapToLong(ByteBuffer::remaining)
+ .sum();
+ return of(byteBuffers, length);
}
public static ByteBuffersAsyncRequestBody of(Long length, ByteBuffer... buffers) {
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java
deleted file mode 100644
index 1c3d126981dd..000000000000
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptor.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Licensed under the Apache License, Version 2.0 (the "License").
- * You may not use this file except in compliance with the License.
- * A copy of the License is located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
-
-import java.nio.ByteBuffer;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.reactivestreams.Subscriber;
-import software.amazon.awssdk.annotations.SdkInternalApi;
-import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
-import software.amazon.awssdk.core.exception.NonRetryableException;
-import software.amazon.awssdk.core.internal.util.NoopSubscription;
-
-/**
- * Adaptor to convert a {@link ClosableAsyncRequestBody} to an {@link AsyncRequestBody}
- *
- *
- * This is needed to maintain backwards compatibility for the deprecated
- * {@link AsyncRequestBody#split(AsyncRequestBodySplitConfiguration)}
- */
-@SdkInternalApi
-public final class ClosableAsyncRequestBodyAdaptor implements AsyncRequestBody {
-
- private final AtomicBoolean subscribeCalled;
- private final ClosableAsyncRequestBody delegate;
-
- public ClosableAsyncRequestBodyAdaptor(ClosableAsyncRequestBody delegate) {
- this.delegate = delegate;
- subscribeCalled = new AtomicBoolean(false);
- }
-
- @Override
- public Optional contentLength() {
- return delegate.contentLength();
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> subscriber) {
- if (subscribeCalled.compareAndSet(false, true)) {
- delegate.doAfterOnComplete(() -> delegate.close())
- .doAfterOnCancel(() -> delegate.close())
- .doAfterOnError(t -> delegate.close())
- .subscribe(subscriber);
- } else {
- subscriber.onSubscribe(new NoopSubscription(subscriber));
- subscriber.onError(NonRetryableException.create(
- "A retry was attempted, but AsyncRequestBody.split does not "
- + "support retries."));
- }
- }
-
-}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
index 656cd1a38e56..0ffb08dc15d1 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
@@ -88,7 +88,7 @@ public SdkPublisher split(AsyncRequestBodySplitConfiguration s
}
@Override
- public SdkPublisher splitV2(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ public SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
return split(splitConfiguration).map(body -> new ClosableAsyncRequestBodyWrapper(body));
}
@@ -444,24 +444,30 @@ private static AsynchronousFileChannel openInputChannel(Path path) throws IOExce
}
private static class ClosableAsyncRequestBodyWrapper implements ClosableAsyncRequestBody {
- private final AsyncRequestBody body;
+ private final AsyncRequestBody delegate;
ClosableAsyncRequestBodyWrapper(AsyncRequestBody body) {
- this.body = body;
+ this.delegate = body;
}
@Override
public Optional contentLength() {
- return body.contentLength();
+ return delegate.contentLength();
}
@Override
public void subscribe(Subscriber super ByteBuffer> s) {
- body.subscribe(s);
+ delegate.subscribe(s);
}
@Override
public void close() {
+ // no op
+ }
+
+ @Override
+ public String body() {
+ return delegate.body();
}
}
}
\ No newline at end of file
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java
new file mode 100644
index 000000000000..eb30b502d8c4
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+import java.nio.ByteBuffer;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import org.reactivestreams.Subscriber;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+import software.amazon.awssdk.core.internal.util.NoopSubscription;
+import software.amazon.awssdk.utils.Logger;
+import software.amazon.awssdk.utils.Validate;
+import software.amazon.awssdk.utils.async.SimplePublisher;
+
+/**
+ * A {@link SubAsyncRequestBody} implementation that doesn't support resubscribe/retry
+ */
+@SdkInternalApi
+public final class NonRetryableSubAsyncRequestBody implements SubAsyncRequestBody {
+ private static final Logger log = Logger.loggerFor(NonRetryableSubAsyncRequestBody.class);
+ private final SubAsyncRequestBodyConfiguration configuration;
+ private final int partNumber;
+ private final boolean contentLengthKnown;
+ private final String sourceBodyName;
+ private final SimplePublisher delegate = new SimplePublisher<>();
+ private final AtomicBoolean subscribeCalled = new AtomicBoolean(false);
+ private volatile long bufferedLength = 0;
+ private final Consumer onNumBytesReceived;
+ private final Consumer onNumBytesConsumed;
+
+ /**
+ * Creates a new NonRetryableSubAsyncRequestBody with the given configuration.
+ */
+ public NonRetryableSubAsyncRequestBody(SubAsyncRequestBodyConfiguration configuration) {
+ this.configuration = Validate.paramNotNull(configuration, "configuration");
+ this.partNumber = configuration.partNumber();
+ this.contentLengthKnown = configuration.contentLengthKnown();
+ this.sourceBodyName = configuration.sourceBodyName();
+ this.onNumBytesReceived = configuration.onNumBytesReceived();
+ this.onNumBytesConsumed = configuration.onNumBytesConsumed();
+ }
+
+ @Override
+ public Optional contentLength() {
+ return contentLengthKnown ? Optional.of(configuration.maxLength()) : Optional.of(bufferedLength);
+ }
+
+ public void send(ByteBuffer data) {
+ log.debug(() -> String.format("Sending bytebuffer %s to part %d", data, partNumber));
+ int length = data.remaining();
+ bufferedLength += length;
+ onNumBytesReceived.accept((long) length);
+ delegate.send(data).whenComplete((r, t) -> {
+ onNumBytesConsumed.accept((long) length);
+ if (t != null) {
+ error(t);
+ }
+ });
+ }
+
+ public void complete() {
+ log.debug(() -> "Received complete() for part number: " + partNumber);
+ delegate.complete().whenComplete((r, t) -> {
+ if (t != null) {
+ error(t);
+ }
+ });
+ }
+
+ @Override
+ public long maxLength() {
+ return configuration.maxLength();
+ }
+
+ @Override
+ public long receivedBytesLength() {
+ return bufferedLength;
+ }
+
+ @Override
+ public boolean contentLengthKnown() {
+ return contentLengthKnown;
+ }
+
+ @Override
+ public int partNumber() {
+ return partNumber;
+ }
+
+ public void error(Throwable error) {
+ delegate.error(error);
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ if (subscribeCalled.compareAndSet(false, true)) {
+ delegate.subscribe(s);
+ } else {
+ s.onSubscribe(new NoopSubscription(s));
+ s.onError(NonRetryableException.create(
+ "A retry was attempted, but the provided source AsyncRequestBody does not "
+ + "support splitting to retryable AsyncRequestBody. Consider using BufferedSplittableAsyncRequestBody."));
+ }
+ }
+
+ @Override
+ public String body() {
+ return sourceBodyName;
+ }
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java
new file mode 100644
index 000000000000..00f4127ca061
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import org.reactivestreams.Subscriber;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+import software.amazon.awssdk.core.internal.util.NoopSubscription;
+import software.amazon.awssdk.utils.Logger;
+import software.amazon.awssdk.utils.Validate;
+import software.amazon.awssdk.utils.async.SimplePublisher;
+
+/**
+ * A {@link SubAsyncRequestBody} implementation that supports resubscribe/retry once all data has been published to the first
+ * subscriber
+ */
+@SdkInternalApi
+public final class RetryableSubAsyncRequestBody implements SubAsyncRequestBody {
+ private static final Logger log = Logger.loggerFor(RetryableSubAsyncRequestBody.class);
+ /**
+ * The maximum length of the content this AsyncRequestBody can hold. If the upstream content length is known, this is
+ * the same as totalLength
+ */
+ private final SubAsyncRequestBodyConfiguration configuration;
+ private final int partNumber;
+ private final boolean contentLengthKnown;
+ private final String sourceBodyName;
+
+ private volatile long bufferedLength = 0;
+ private volatile ByteBuffersAsyncRequestBody bufferedAsyncRequestBody;
+ private List buffers = new ArrayList<>();
+ private final AtomicBoolean subscribeCalled = new AtomicBoolean(false);
+ private final SimplePublisher delegate = new SimplePublisher<>();
+ private final Consumer onNumBytesReceived;
+ private final Consumer onNumBytesConsumed;
+ private final Object buffersLock = new Object();
+
+ /**
+ * Creates a new RetryableSubAsyncRequestBody with the given configuration.
+ */
+ public RetryableSubAsyncRequestBody(SubAsyncRequestBodyConfiguration configuration) {
+ this.configuration = Validate.paramNotNull(configuration, "configuration");
+ this.partNumber = configuration.partNumber();
+ this.contentLengthKnown = configuration.contentLengthKnown();
+ this.sourceBodyName = configuration.sourceBodyName();
+ this.onNumBytesReceived = configuration.onNumBytesReceived();
+ this.onNumBytesConsumed = configuration.onNumBytesConsumed();
+ }
+
+ @Override
+ public Optional contentLength() {
+ return contentLengthKnown ? Optional.of(configuration.maxLength()) : Optional.of(bufferedLength);
+ }
+
+ @Override
+ public void send(ByteBuffer data) {
+ log.trace(() -> String.format("Sending bytebuffer %s to part number %d", data, partNumber));
+ int length = data.remaining();
+ bufferedLength += length;
+
+ onNumBytesReceived.accept((long) length);
+ delegate.send(data.asReadOnlyBuffer()).whenComplete((r, t) -> {
+ if (t != null) {
+ delegate.error(t);
+ }
+ });
+ synchronized (buffersLock) {
+ buffers.add(data.asReadOnlyBuffer());
+ }
+ }
+
+ @Override
+ public void complete() {
+ log.debug(() -> "Received complete() for part number: " + partNumber);
+ // ByteBuffersAsyncRequestBody MUST be created before we complete the current
+ // request because retry may happen right after
+ bufferedAsyncRequestBody = ByteBuffersAsyncRequestBody.of(buffers, bufferedLength);
+ delegate.complete().exceptionally(e -> {
+ delegate.error(e);
+ return null;
+ });
+ }
+
+ @Override
+ public long maxLength() {
+ return configuration.maxLength();
+ }
+
+ @Override
+ public long receivedBytesLength() {
+ return bufferedLength;
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ log.debug(() -> "Subscribe for part number: " + partNumber);
+ if (subscribeCalled.compareAndSet(false, true)) {
+ delegate.subscribe(s);
+ } else {
+ log.debug(() -> "Resubscribe for part number " + partNumber);
+ if (bufferedAsyncRequestBody == null) {
+ s.onSubscribe(new NoopSubscription(s));
+ s.onError(NonRetryableException.create(
+ "A retry was attempted, but data is not buffered successfully for retry, partNumber " + partNumber));
+ return;
+ }
+ bufferedAsyncRequestBody.subscribe(s);
+ }
+ }
+
+ @Override
+ public void close() {
+ try {
+ log.debug(() -> "Closing current body " + partNumber);
+ onNumBytesConsumed.accept(bufferedLength);
+ if (bufferedAsyncRequestBody != null) {
+ synchronized (buffersLock) {
+ buffers.clear();
+ buffers = null;
+ }
+ bufferedAsyncRequestBody.close();
+ log.debug(() -> "requesting data after closing" + partNumber);
+ }
+ } catch (Throwable e) {
+ log.warn(() -> String.format("Unexpected error thrown from cleaning up AsyncRequestBody for part number %d, "
+ + "resource may be leaked", partNumber));
+ }
+ }
+
+ @Override
+ public boolean contentLengthKnown() {
+ return contentLengthKnown;
+ }
+
+ @Override
+ public int partNumber() {
+ return partNumber;
+ }
+
+ @Override
+ public String body() {
+ return sourceBodyName;
+ }
+
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
index c44c728746a3..d7e16ca5445c 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
@@ -16,9 +16,6 @@
package software.amazon.awssdk.core.internal.async;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -37,7 +34,8 @@
* Splits an {@link AsyncRequestBody} to multiple smaller {@link AsyncRequestBody}s, each of which publishes a specific portion of
* the original data.
*
- * Each {@link AsyncRequestBody} is sent after the entire content for that chunk is buffered.
+ *
If content length is known, each {@link AsyncRequestBody} is sent to the subscriber right after it's initialized.
+ * Otherwise, it is sent after the entire content for that chunk is buffered. This is required to get content length.
*/
@SdkInternalApi
public class SplittingPublisher implements SdkPublisher {
@@ -47,10 +45,13 @@ public class SplittingPublisher implements SdkPublisher downstreamPublisher = new SimplePublisher<>();
private final long chunkSizeInBytes;
private final long bufferSizeInBytes;
+ private final boolean enableRetryableSubAsyncRequestBody;
private final AtomicBoolean currentBodySent = new AtomicBoolean(false);
+ private final String sourceBodyName;
public SplittingPublisher(AsyncRequestBody asyncRequestBody,
- AsyncRequestBodySplitConfiguration splitConfiguration) {
+ AsyncRequestBodySplitConfiguration splitConfiguration,
+ boolean enableRetryableSubAsyncRequestBody) {
this.upstreamPublisher = Validate.paramNotNull(asyncRequestBody, "asyncRequestBody");
Validate.notNull(splitConfiguration, "splitConfiguration");
this.chunkSizeInBytes = splitConfiguration.chunkSizeInBytes() == null ?
@@ -63,9 +64,13 @@ public SplittingPublisher(AsyncRequestBody asyncRequestBody,
this.splittingSubscriber = new SplittingSubscriber(upstreamPublisher.contentLength().orElse(null));
- Validate.isTrue(bufferSizeInBytes >= chunkSizeInBytes,
- "bufferSizeInBytes must be larger than or equal to " +
- "chunkSizeInBytes");
+ this.enableRetryableSubAsyncRequestBody = enableRetryableSubAsyncRequestBody;
+ this.sourceBodyName = asyncRequestBody.body();
+ if (!upstreamPublisher.contentLength().isPresent()) {
+ Validate.isTrue(bufferSizeInBytes >= chunkSizeInBytes,
+ "bufferSizeInBytes must be larger than or equal to " +
+ "chunkSizeInBytes if the content length is unknown");
+ }
}
@Override
@@ -81,7 +86,7 @@ private class SplittingSubscriber implements Subscriber {
* 1 based index number for each part/chunk
*/
private final AtomicInteger partNumber = new AtomicInteger(1);
- private volatile DownstreamBody currentBody;
+ private volatile SubAsyncRequestBody currentBody;
private final AtomicBoolean hasOpenUpstreamDemand = new AtomicBoolean(false);
private final AtomicLong dataBuffered = new AtomicLong(0);
@@ -105,10 +110,30 @@ public void onSubscribe(Subscription s) {
upstreamSubscription.request(1);
}
- private DownstreamBody initializeNextDownstreamBody(boolean contentLengthKnown, long chunkSize, int partNumber) {
- currentBodySent.set(false);
+ private SubAsyncRequestBody initializeNextDownstreamBody(boolean contentLengthKnown, long chunkSize, int chunkNumber) {
+ SubAsyncRequestBody body;
log.debug(() -> "initializing next downstream body " + partNumber);
- return new DownstreamBody(contentLengthKnown, chunkSize, partNumber);
+
+ SubAsyncRequestBodyConfiguration config = SubAsyncRequestBodyConfiguration.builder()
+ .contentLengthKnown(contentLengthKnown)
+ .maxLength(chunkSize)
+ .partNumber(chunkNumber)
+ .onNumBytesReceived(data -> addDataBuffered(data))
+ .onNumBytesConsumed(data -> addDataBuffered(-data))
+ .sourceBodyName(sourceBodyName)
+ .build();
+
+ if (enableRetryableSubAsyncRequestBody) {
+ body = new RetryableSubAsyncRequestBody(config);
+ } else {
+ body = new NonRetryableSubAsyncRequestBody(config);
+ }
+
+ currentBodySent.set(false);
+ if (contentLengthKnown) {
+ sendCurrentBody(body);
+ }
+ return body;
}
@Override
@@ -157,7 +182,7 @@ public void onNext(ByteBuffer byteBuffer) {
}
private void completeCurrentBodyAndCreateNewIfNeeded(ByteBuffer byteBuffer) {
- completeCurrentBodyAndDeliver();
+ completeCurrentBody();
int nextChunk = partNumber.incrementAndGet();
boolean shouldCreateNewDownstreamRequestBody;
Long dataRemaining = totalDataRemaining();
@@ -175,53 +200,54 @@ private void completeCurrentBodyAndCreateNewIfNeeded(ByteBuffer byteBuffer) {
}
private int amountRemainingInChunk() {
- return Math.toIntExact(currentBody.maxLength - currentBody.bufferedLength);
+ return Math.toIntExact(currentBody.maxLength() - currentBody.receivedBytesLength());
}
- /**
- * Current body could be completed in either onNext or onComplete, so we need to guard against sending the last body
- * twice.
- */
- private void completeCurrentBodyAndDeliver() {
- if (currentBodySent.compareAndSet(false, true)) {
- log.debug(() -> "completeCurrentBody for chunk " + currentBody.partNumber);
- // For unknown content length, we always create a new DownstreamBody because we don't know if there is data
- // left or not, so we need to only send the body if there is actually data
- long bufferedLength = currentBody.bufferedLength;
- Long totalLength = currentBody.totalLength;
- if (bufferedLength > 0) {
- if (totalLength != null && totalLength != bufferedLength) {
- upstreamSubscription.cancel();
- downstreamPublisher.error(new IllegalStateException(
- String.format("Content length of buffered data mismatches "
- + "with the expected content length, buffered data content length: %d, "
- + "expected length: %d", totalLength,
- bufferedLength)));
- return;
- }
-
- currentBody.complete();
- sendCurrentBody(currentBody);
- }
+
+ private void completeCurrentBody() {
+ log.debug(() -> "completeCurrentBody for part " + currentBody.partNumber());
+ // For unknown content length, we always create a new DownstreamBody because we don't know if there is data
+ // left or not, so we need to only send the body if there is actually data
+ long bufferedLength = currentBody.receivedBytesLength();
+ if (bufferedLength == 0) {
+ return;
+ }
+
+ Long totalLength = currentBody.maxLength();
+ if (currentBody.contentLengthKnown() && totalLength != bufferedLength) {
+ upstreamSubscription.cancel();
+ downstreamPublisher.error(new IllegalStateException(
+ String.format("Content length of buffered data mismatches "
+ + "with the expected content length, buffered data content length: %d, "
+ + "expected length: %d", totalLength,
+ bufferedLength)));
+ return;
+ }
+ currentBody.complete();
+
+ // Current body could be completed in either onNext or onComplete, so we need to guard against sending the last body
+ // twice.
+ if (upstreamSize == null && currentBodySent.compareAndSet(false, true)) {
+ sendCurrentBody(currentBody);
}
}
@Override
public void onComplete() {
upstreamComplete = true;
- log.debug(() -> "Received onComplete() from upstream AsyncRequestBody");
- completeCurrentBodyAndDeliver();
+ log.debug(() -> "Received onComplete()");
+ completeCurrentBody();
downstreamPublisher.complete();
}
@Override
public void onError(Throwable t) {
- log.trace(() -> "Received onError()", t);
+ log.debug(() -> "Received onError()", t);
downstreamPublisher.error(t);
}
- private void sendCurrentBody(DownstreamBody body) {
- log.debug(() -> "sendCurrentBody for chunk " + body.partNumber);
+ private void sendCurrentBody(SubAsyncRequestBody body) {
+ log.debug(() -> "sendCurrentBody for part " + body.partNumber());
downstreamPublisher.send(body).exceptionally(t -> {
downstreamPublisher.error(t);
upstreamSubscription.cancel();
@@ -244,6 +270,8 @@ private void maybeRequestMoreUpstreamData() {
hasOpenUpstreamDemand.compareAndSet(false, true)) {
log.trace(() -> "Requesting more data, current data buffered: " + buffered);
upstreamSubscription.request(1);
+ } else {
+ log.trace(() -> "Should not request more data, current data buffered: " + buffered);
}
}
@@ -258,66 +286,12 @@ private Long totalDataRemaining() {
return upstreamSize - ((partNumber.get() - 1) * chunkSizeInBytes);
}
- /**
- * AsyncRequestBody for individual part. The entire data is buffered in memory and can be subscribed multiple times
- * for retry attempts. The buffered data is cleared upon close
- */
- private final class DownstreamBody implements ClosableAsyncRequestBody {
-
- /**
- * The maximum length of the content this AsyncRequestBody can hold. If the upstream content length is known, this is
- * the same as totalLength
- */
- private final long maxLength;
- private final Long totalLength;
- private final int partNumber;
- private volatile long bufferedLength = 0;
- private volatile ByteBuffersAsyncRequestBody delegate;
- private final List buffers = new ArrayList<>();
-
- private DownstreamBody(boolean contentLengthKnown, long maxLength, int partNumber) {
- this.totalLength = contentLengthKnown ? maxLength : null;
- this.maxLength = maxLength;
- this.partNumber = partNumber;
- }
-
- @Override
- public Optional contentLength() {
- return totalLength != null ? Optional.of(totalLength) : Optional.of(bufferedLength);
- }
-
- public void send(ByteBuffer data) {
- log.debug(() -> String.format("Sending bytebuffer %s to chunk %d", data, partNumber));
- int length = data.remaining();
- bufferedLength += length;
- addDataBuffered(length);
- buffers.add(data);
- }
-
- public void complete() {
- log.debug(() -> "Received complete() for chunk number: " + partNumber + " length " + bufferedLength);
- this.delegate = ByteBuffersAsyncRequestBody.of(buffers);
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- log.debug(() -> "Subscribe for chunk number: " + partNumber + " length " + bufferedLength);
- delegate.subscribe(s);
- }
-
- private void addDataBuffered(long length) {
- dataBuffered.addAndGet(length);
- if (length < 0) {
- maybeRequestMoreUpstreamData();
- }
- }
-
- @Override
- public void close() {
- log.debug(() -> "Closing current body " + partNumber);
- delegate.close();
- addDataBuffered(-bufferedLength);
+ private void addDataBuffered(long length) {
+ log.trace(() -> "Adding data buffered " + length);
+ dataBuffered.addAndGet(length);
+ if (length < 0) {
+ maybeRequestMoreUpstreamData();
}
}
}
-}
+}
\ No newline at end of file
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java
new file mode 100644
index 000000000000..c29446eb4929
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+import java.nio.ByteBuffer;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+
+/**
+ * Represent a sub {@link AsyncRequestBody} that publishes a portion of the source {@link AsyncRequestBody}
+ */
+@SdkInternalApi
+public interface SubAsyncRequestBody extends ClosableAsyncRequestBody {
+
+ /**
+ * Send a byte buffer.
+ *
+ * This method must not be invoked concurrently.
+ */
+ void send(ByteBuffer byteBuffer);
+
+ /**
+ * Indicate that no more {@link #send(ByteBuffer)} )} calls will be made,
+ * and that stream of messages is completed successfully.
+ */
+ void complete();
+
+ /**
+ * The maximum length of the content this AsyncRequestBody can hold. If the upstream content length is known, this should be
+ * the same as receivedBytesLength
+ */
+ long maxLength();
+
+ /**
+ * The length of the bytes received
+ */
+ long receivedBytesLength();
+
+ @Override
+ default void close() {
+ // no op
+ }
+
+ boolean contentLengthKnown();
+
+ /**
+ * The part number associated with this SubAsyncRequestBody
+ * @return
+ */
+ int partNumber();
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBodyConfiguration.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBodyConfiguration.java
new file mode 100644
index 000000000000..cca1abe166d1
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBodyConfiguration.java
@@ -0,0 +1,140 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+import java.util.function.Consumer;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.utils.Validate;
+
+/**
+ * Configuration class containing shared properties for SubAsyncRequestBody implementations.
+ */
+@SdkInternalApi
+public final class SubAsyncRequestBodyConfiguration {
+ private final boolean contentLengthKnown;
+ private final long maxLength;
+ private final int partNumber;
+ private final Consumer onNumBytesReceived;
+ private final Consumer onNumBytesConsumed;
+ private final String sourceBodyName;
+
+ private SubAsyncRequestBodyConfiguration(Builder builder) {
+ this.contentLengthKnown = Validate.paramNotNull(builder.contentLengthKnown, "contentLengthKnown");
+ this.maxLength = Validate.paramNotNull(builder.maxLength, "maxLength");
+ this.partNumber = Validate.paramNotNull(builder.partNumber, "partNumber");
+ this.onNumBytesReceived = Validate.paramNotNull(builder.onNumBytesReceived, "onNumBytesReceived");
+ this.onNumBytesConsumed = Validate.paramNotNull(builder.onNumBytesConsumed, "onNumBytesConsumed");
+ this.sourceBodyName = Validate.paramNotNull(builder.sourceBodyName, "sourceBodyName");
+ }
+
+ /**
+ * Returns a newly initialized builder object for a {@link SubAsyncRequestBodyConfiguration}
+ */
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ public boolean contentLengthKnown() {
+ return contentLengthKnown;
+ }
+
+ public long maxLength() {
+ return maxLength;
+ }
+
+ public int partNumber() {
+ return partNumber;
+ }
+
+ public Consumer onNumBytesReceived() {
+ return onNumBytesReceived;
+ }
+
+ public Consumer onNumBytesConsumed() {
+ return onNumBytesConsumed;
+ }
+
+ public String sourceBodyName() {
+ return sourceBodyName;
+ }
+
+ public static final class Builder {
+ private Boolean contentLengthKnown;
+ private Long maxLength;
+ private Integer partNumber;
+ private Consumer onNumBytesReceived;
+ private Consumer onNumBytesConsumed;
+ private String sourceBodyName;
+
+ private Builder() {
+ }
+
+ /**
+ * Sets whether the content length is known.
+ */
+ public Builder contentLengthKnown(Boolean contentLengthKnown) {
+ this.contentLengthKnown = contentLengthKnown;
+ return this;
+ }
+
+ /**
+ * Sets the maximum length of the content this AsyncRequestBody can hold.
+ */
+ public Builder maxLength(Long maxLength) {
+ this.maxLength = maxLength;
+ return this;
+ }
+
+ /**
+ * Sets the part number for this request body.
+ */
+ public Builder partNumber(Integer partNumber) {
+ this.partNumber = partNumber;
+ return this;
+ }
+
+ /**
+ * Sets the callback to be invoked when bytes are received.
+ */
+ public Builder onNumBytesReceived(Consumer onNumBytesReceived) {
+ this.onNumBytesReceived = onNumBytesReceived;
+ return this;
+ }
+
+ /**
+ * Sets the callback to be invoked when bytes are consumed.
+ */
+ public Builder onNumBytesConsumed(Consumer onNumBytesConsumed) {
+ this.onNumBytesConsumed = onNumBytesConsumed;
+ return this;
+ }
+
+ /**
+ * Sets the source body name for identification.
+ */
+ public Builder sourceBodyName(String sourceBodyName) {
+ this.sourceBodyName = sourceBodyName;
+ return this;
+ }
+
+ /**
+ * Builds a {@link SubAsyncRequestBodyConfiguration} object based on the values held by this builder.
+ */
+ public SubAsyncRequestBodyConfiguration build() {
+ return new SubAsyncRequestBodyConfiguration(this);
+ }
+ }
+}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
index f0c72a624119..1c363eb90fe8 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
@@ -415,7 +415,7 @@ void publisherConstructorHasCorrectContentType() {
void splitV2_nullConfig_shouldThrowException() {
AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world");
AsyncRequestBodySplitConfiguration config = null;
- assertThatThrownBy(() -> requestBody.splitV2(config))
+ assertThatThrownBy(() -> requestBody.splitClosable(config))
.isInstanceOf(NullPointerException.class)
.hasMessageContaining("splitConfig");
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
deleted file mode 100644
index 1a9a3ffc833b..000000000000
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ClosableAsyncRequestBodyAdaptorTest.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Licensed under the Apache License, Version 2.0 (the "License").
- * You may not use this file except in compliance with the License.
- * A copy of the License is located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
-
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import io.reactivex.Flowable;
-import io.reactivex.FlowableSubscriber;
-import io.reactivex.internal.observers.BiConsumerSingleObserver;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.Observable;
-import java.util.Observer;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.reactivestreams.Subscriber;
-import org.reactivestreams.Subscription;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
-import software.amazon.awssdk.core.exception.NonRetryableException;
-
-public class ClosableAsyncRequestBodyAdaptorTest {
- private ClosableAsyncRequestBody closableAsyncRequestBody;
-
- @BeforeEach
- public void setup() {
- closableAsyncRequestBody =Mockito.mock(ClosableAsyncRequestBody.class);
- Mockito.when(closableAsyncRequestBody.doAfterOnComplete(any(Runnable.class))).thenReturn(closableAsyncRequestBody);
- Mockito.when(closableAsyncRequestBody.doAfterOnCancel(any(Runnable.class))).thenReturn(closableAsyncRequestBody);
- Mockito.when(closableAsyncRequestBody.doAfterOnError(any(Consumer.class))).thenReturn(closableAsyncRequestBody);
- }
-
- @Test
- void resubscribe_shouldThrowException() {
- ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(closableAsyncRequestBody);
- Subscriber subscriber = Mockito.mock(Subscriber.class);
- adaptor.subscribe(subscriber);
-
- Subscriber anotherSubscriber = Mockito.mock(Subscriber.class);
- ArgumentCaptor exceptionCaptor = ArgumentCaptor.forClass(Throwable.class);
- doNothing().when(anotherSubscriber).onError(exceptionCaptor.capture());
-
- adaptor.subscribe(anotherSubscriber);
-
- assertThat(exceptionCaptor.getValue())
- .isInstanceOf(NonRetryableException.class)
- .hasMessageContaining("A retry was attempted");
- }
-
- @Test
- void onComplete_shouldCloseAsyncRequestBody() {
- TestClosableAsyncRequestBody asyncRequestBody = new TestClosableAsyncRequestBody();
- ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(asyncRequestBody);
- CompletableFuture future = new CompletableFuture<>();
- Subscriber subscriber = new ByteArrayAsyncResponseTransformer.BaosSubscriber(future);
- adaptor.subscribe(subscriber);
- assertThat(asyncRequestBody.closeInvoked).isTrue();
- }
-
- @Test
- void cancel_shouldCloseAsyncRequestBody() {
- TestClosableAsyncRequestBody asyncRequestBody = new TestClosableAsyncRequestBody();
- ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(asyncRequestBody);
- Subscriber subscriber = new Subscriber() {
- @Override
- public void onSubscribe(Subscription s) {
- s.cancel();
- }
-
- @Override
- public void onNext(ByteBuffer byteBuffer) {
- }
-
- @Override
- public void onError(Throwable t) {
- }
-
- @Override
- public void onComplete() {
- }
- };
- adaptor.subscribe(subscriber);
- assertThat(asyncRequestBody.closeInvoked).isTrue();
- }
-
- @Test
- void onError_shouldCloseAsyncRequestBody() {
- OnErrorClosableAsyncRequestBody asyncRequestBody = new OnErrorClosableAsyncRequestBody();
- ClosableAsyncRequestBodyAdaptor adaptor = new ClosableAsyncRequestBodyAdaptor(asyncRequestBody);
- CompletableFuture future = new CompletableFuture<>();
- Subscriber subscriber = new ByteArrayAsyncResponseTransformer.BaosSubscriber(future);
- adaptor.subscribe(subscriber);
- assertThat(asyncRequestBody.closeInvoked).isTrue();
- }
-
-
- private static class TestClosableAsyncRequestBody implements ClosableAsyncRequestBody {
- private boolean closeInvoked;
-
- @Override
- public Optional contentLength() {
- return Optional.empty();
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- Flowable.just(ByteBuffer.wrap("foo bar".getBytes(StandardCharsets.UTF_8)))
- .subscribe(s);
- }
-
- @Override
- public void close() {
- closeInvoked = true;
- }
- }
-
- private static class OnErrorClosableAsyncRequestBody implements ClosableAsyncRequestBody {
- private boolean closeInvoked;
-
- @Override
- public Optional contentLength() {
- return Optional.empty();
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- s.onSubscribe(new Subscription() {
- @Override
- public void request(long n) {
- s.onError(new IllegalStateException("foobar"));
- }
-
- @Override
- public void cancel() {
-
- }
- });
- }
-
- @Override
- public void close() {
- closeInvoked = true;
- }
- }
-}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
index a1f22d2bf7af..057d6c165eb8 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
@@ -28,11 +28,15 @@
import java.nio.charset.Charset;
import java.nio.file.Files;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.lang3.RandomStringUtils;
import org.junit.jupiter.api.AfterAll;
@@ -45,6 +49,7 @@
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
import software.amazon.awssdk.utils.BinaryUtils;
+import software.amazon.awssdk.utils.Pair;
public class SplittingPublisherTest {
private static final int CHUNK_SIZE = 5;
@@ -69,18 +74,14 @@ public static void afterAll() throws Exception {
}
@Test
- public void split_MaxMemorySmallerThanChunkSize_shouldThrowException() {
+ public void split_contentUnknownMaxMemorySmallerThanChunkSize_shouldThrowException() {
AsyncRequestBody body = AsyncRequestBody.fromPublisher(s -> {
});
AsyncRequestBodySplitConfiguration configuration = AsyncRequestBodySplitConfiguration.builder()
.chunkSizeInBytes(10L)
.bufferSizeInBytes(5L)
.build();
- assertThatThrownBy(() -> new SplittingPublisher(body, configuration))
- .hasMessageContaining("must be larger than or equal");
-
- assertThatThrownBy(() -> new SplittingPublisher(AsyncRequestBody.fromString("test"),
- configuration))
+ assertThatThrownBy(() -> new SplittingPublisher(body, configuration, false))
.hasMessageContaining("must be larger than or equal");
}
@@ -93,16 +94,24 @@ void differentChunkSize_shouldSplitAsyncRequestBodyCorrectly(int chunkSize) thro
.chunkSizeInBytes(chunkSize)
.build();
verifySplitContent(fileAsyncRequestBody, chunkSize);
+ fileAsyncRequestBody = FileAsyncRequestBody.builder()
+ .path(testFile.toPath())
+ .chunkSizeInBytes(chunkSize)
+ .build();
+ verifyRetryableSplitContent(fileAsyncRequestBody, chunkSize);
}
@ParameterizedTest
@ValueSource(ints = {CHUNK_SIZE, CHUNK_SIZE * 2 - 1, CHUNK_SIZE * 2})
void differentChunkSize_byteArrayShouldSplitAsyncRequestBodyCorrectly(int chunkSize) throws Exception {
verifySplitContent(AsyncRequestBody.fromBytes(CONTENT), chunkSize);
+ verifyRetryableSplitContent(AsyncRequestBody.fromBytes(CONTENT), chunkSize);
}
- @Test
- void contentLengthNotPresent_shouldHandle() throws Exception {
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ void contentLengthNotPresent_shouldHandle(boolean enableRetryableSubAsyncRequestBody) throws Exception {
CompletableFuture future = new CompletableFuture<>();
TestAsyncRequestBody asyncRequestBody = new TestAsyncRequestBody() {
@Override
@@ -110,10 +119,11 @@ public Optional contentLength() {
return Optional.empty();
}
};
- SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody, AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes((long) CHUNK_SIZE)
- .bufferSizeInBytes(10L)
- .build());
+ SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
+ AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) CHUNK_SIZE)
+ .bufferSizeInBytes(10L)
+ .build(), enableRetryableSubAsyncRequestBody);
List> futures = new ArrayList<>();
@@ -149,27 +159,79 @@ public Optional contentLength() {
}
- @Test
- void downStreamFailed_shouldPropagateCancellation() {
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ void downStreamFailed_shouldPropagateCancellation(boolean enableRetryableSubAsyncRequestBody) throws Exception {
CompletableFuture future = new CompletableFuture<>();
TestAsyncRequestBody asyncRequestBody = new TestAsyncRequestBody();
SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody, AsyncRequestBodySplitConfiguration.builder()
.chunkSizeInBytes((long) CHUNK_SIZE)
.bufferSizeInBytes(10L)
- .build());
-
+ .build(), enableRetryableSubAsyncRequestBody);
assertThatThrownBy(() -> splittingPublisher.subscribe(requestBody -> {
throw new RuntimeException("foobar");
}).get(5, TimeUnit.SECONDS)).hasMessageContaining("foobar");
assertThat(asyncRequestBody.cancelled).isTrue();
}
+ @Test
+ void retryableSubAsyncRequestBodyEnabled_shouldBeAbleToResubscribe() throws ExecutionException, InterruptedException, TimeoutException {
+ int chunkSize = 5;
+ AsyncRequestBody asyncRequestBody = FileAsyncRequestBody.builder()
+ .path(testFile.toPath())
+ .chunkSizeInBytes(chunkSize)
+ .build();
+
+ SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
+ AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) chunkSize)
+ .bufferSizeInBytes((long) chunkSize * 4)
+ .build(),
+ true);
+
+
+
+ Map, CompletableFuture>> futures = new HashMap<>();
+ AtomicInteger index = new AtomicInteger();
+ splittingPublisher.subscribe(requestBody -> {
+ int i = index.getAndIncrement();
+ CompletableFuture future = new CompletableFuture<>();
+ BaosSubscriber subscriber = new BaosSubscriber(future);
+ requestBody.subscribe(subscriber);
+
+ future.whenComplete((r, t) -> {
+ CompletableFuture future2 = new CompletableFuture<>();
+ BaosSubscriber anotherSubscriber = new BaosSubscriber(future2);
+ requestBody.subscribe(anotherSubscriber);
+ futures.put(i, Pair.of(future, future2));
+
+ future2.whenComplete((res, throwable) -> {
+ requestBody.close();
+ });
+ });
+ }).get(5, TimeUnit.SECONDS);
+
+ for (int i = 0; i < futures.size(); i++) {
+ assertThat(futures.get(i).left().join()).containsExactly( futures.get(i).right().join());
+ }
+ }
+
private static void verifySplitContent(AsyncRequestBody asyncRequestBody, int chunkSize) throws Exception {
SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
AsyncRequestBodySplitConfiguration.builder()
.chunkSizeInBytes((long) chunkSize)
.bufferSizeInBytes((long) chunkSize * 4)
- .build());
+ .build(), false);
+
+ verifyIndividualAsyncRequestBody(splittingPublisher.map(m -> m), testFile.toPath(), chunkSize);
+ }
+
+ private static void verifyRetryableSplitContent(AsyncRequestBody asyncRequestBody, int chunkSize) throws Exception {
+ SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
+ AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) chunkSize)
+ .bufferSizeInBytes((long) chunkSize * 4)
+ .build(), false);
verifyIndividualAsyncRequestBody(splittingPublisher.map(m -> m), testFile.toPath(), chunkSize);
}
diff --git a/core/sdk-core/src/test/resources/log4j2.properties b/core/sdk-core/src/test/resources/log4j2.properties
index e5e68dd2faa4..e96677c00925 100644
--- a/core/sdk-core/src/test/resources/log4j2.properties
+++ b/core/sdk-core/src/test/resources/log4j2.properties
@@ -25,8 +25,8 @@ rootLogger.appenderRef.stdout.ref = ConsoleAppender
# Uncomment below to enable more specific logging
#
-#logger.sdk.name = software.amazon.awssdk
-#logger.sdk.level = debug
+logger.sdk.name = software.amazon.awssdk
+logger.sdk.level = debug
#
#logger.request.name = software.amazon.awssdk.request
#logger.request.level = debug
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
index 15f15767db54..3808cb864de1 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
@@ -186,8 +186,8 @@ private void splitAndSubscribe(MpuRequestContext mpuRequestContext, CompletableF
attachSubscriberToObservable(subscriber, mpuRequestContext.request().left());
mpuRequestContext.request().right()
- .splitV2(b -> b.chunkSizeInBytes(mpuRequestContext.partSize())
- .bufferSizeInBytes(maxMemoryUsageInBytes))
+ .splitClosable(b -> b.chunkSizeInBytes(mpuRequestContext.partSize())
+ .bufferSizeInBytes(maxMemoryUsageInBytes))
.subscribe(subscriber);
}
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
index 73a0c0d39cfc..a02f643c70c8 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
@@ -54,10 +54,12 @@
public final class UploadWithUnknownContentLengthHelper {
private static final Logger log = Logger.loggerFor(UploadWithUnknownContentLengthHelper.class);
+ private final S3AsyncClient s3AsyncClient;
private final long partSizeInBytes;
private final GenericMultipartHelper genericMultipartHelper;
private final long maxMemoryUsageInBytes;
+ private final long multipartUploadThresholdInBytes;
private final MultipartUploadHelper multipartUploadHelper;
@@ -65,11 +67,13 @@ public UploadWithUnknownContentLengthHelper(S3AsyncClient s3AsyncClient,
long partSizeInBytes,
long multipartUploadThresholdInBytes,
long maxMemoryUsageInBytes) {
+ this.s3AsyncClient = s3AsyncClient;
this.partSizeInBytes = partSizeInBytes;
this.genericMultipartHelper = new GenericMultipartHelper<>(s3AsyncClient,
SdkPojoConversionUtils::toAbortMultipartUploadRequest,
SdkPojoConversionUtils::toPutObjectResponse);
this.maxMemoryUsageInBytes = maxMemoryUsageInBytes;
+ this.multipartUploadThresholdInBytes = multipartUploadThresholdInBytes;
this.multipartUploadHelper = new MultipartUploadHelper(s3AsyncClient, multipartUploadThresholdInBytes,
maxMemoryUsageInBytes);
}
@@ -79,8 +83,8 @@ public CompletableFuture uploadObject(PutObjectRequest putObj
CompletableFuture returnFuture = new CompletableFuture<>();
SdkPublisher splitAsyncRequestBodyResponse =
- asyncRequestBody.splitV2(b -> b.chunkSizeInBytes(partSizeInBytes)
- .bufferSizeInBytes(maxMemoryUsageInBytes));
+ asyncRequestBody.splitClosable(b -> b.chunkSizeInBytes(partSizeInBytes)
+ .bufferSizeInBytes(maxMemoryUsageInBytes));
splitAsyncRequestBodyResponse.subscribe(new UnknownContentLengthAsyncRequestBodySubscriber(partSizeInBytes,
putObjectRequest,
@@ -89,6 +93,10 @@ public CompletableFuture uploadObject(PutObjectRequest putObj
}
private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscriber {
+ /**
+ * Indicates whether this is the first async request body or not.
+ */
+ private final AtomicBoolean isFirstAsyncRequestBody = new AtomicBoolean(true);
/**
* Indicates whether CreateMultipartUpload has been initiated or not
@@ -120,7 +128,7 @@ private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscrib
private final CompletableFuture returnFuture;
private final PublisherListener progressListener;
private Subscription subscription;
- private AsyncRequestBody firstRequestBody;
+ private ClosableAsyncRequestBody firstRequestBody;
private String uploadId;
private volatile boolean isDone;
@@ -159,8 +167,7 @@ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
return;
}
int currentPartNum = partNumber.incrementAndGet();
- log.debug(() -> String.format("Received asyncRequestBody for part number %d with length %s", currentPartNum,
- asyncRequestBody.contentLength()));
+ log.trace(() -> "Received asyncRequestBody " + asyncRequestBody.contentLength());
asyncRequestBodyInFlight.incrementAndGet();
Optional sdkClientException = validatePart(asyncRequestBody, currentPartNum);
@@ -171,6 +178,14 @@ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
return;
}
+ if (isFirstAsyncRequestBody.compareAndSet(true, false)) {
+ log.trace(() -> "Received first async request body");
+ // If this is the first AsyncRequestBody received, request another one because we don't know if there is more
+ firstRequestBody = asyncRequestBody;
+ subscription.request(1);
+ return;
+ }
+
// If there are more than 1 AsyncRequestBodies, then we know we need to upload this
// object using MPU
if (createMultipartUploadInitiated.compareAndSet(false, true)) {
@@ -187,7 +202,8 @@ public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
uploadId = createMultipartUploadResponse.uploadId();
log.debug(() -> "Initiated a new multipart upload, uploadId: " + uploadId);
- sendUploadPartRequest(uploadId, asyncRequestBody, currentPartNum);
+ sendUploadPartRequest(uploadId, firstRequestBody, 1);
+ sendUploadPartRequest(uploadId, asyncRequestBody, 2);
// We need to complete the uploadIdFuture *after* the first two requests have been sent
uploadIdFuture.complete(uploadId);
@@ -228,14 +244,13 @@ private void sendUploadPartRequest(String uploadId,
asyncRequestBody.close();
if (t != null) {
if (failureActionInitiated.compareAndSet(false, true)) {
- subscription.cancel();
multipartUploadHelper.failRequestsElegantly(futures, t, uploadId, returnFuture, putObjectRequest);
}
} else {
completeMultipartUploadIfFinish(asyncRequestBodyInFlight.decrementAndGet());
}
});
- synchronized (subscription) {
+ synchronized (this) {
subscription.request(1);
};
}
@@ -292,4 +307,4 @@ private void completeMultipartUploadIfFinish(int requestsInFlight) {
}
}
}
-}
+}
\ No newline at end of file
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
index ee3bf01c3d54..63ae1d39875a 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
@@ -55,7 +55,9 @@
import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.BlockingInputStreamAsyncRequestBody;
-import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient;
+import software.amazon.awssdk.core.async.BufferedSplittableAsyncRequestBody;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
@@ -76,13 +78,13 @@ public class S3MultipartClientPutObjectWiremockTest {
public static Stream retryableErrorTestCase() {
return Stream.of(
- Arguments.of("unknownContentLength_failOfConnectionReset_shouldRetry", null,
+ Arguments.of("unknownContentLength_failOfConnectionReset", null,
aResponse().withFault(Fault.CONNECTION_RESET_BY_PEER)),
- Arguments.of("unknownContentLength_failOf500_shouldRetry", null,
+ Arguments.of("unknownContentLength_failOf500", null,
aResponse().withStatus(500)),
- Arguments.of("knownContentLength_failOfConnectionReset_shouldRetry", 20L,
+ Arguments.of("knownContentLength_failOfConnectionReset", 20L,
aResponse().withFault(Fault.CONNECTION_RESET_BY_PEER)),
- Arguments.of("knownContentLength_failOf500_shouldRetry", 20L,
+ Arguments.of("knownContentLength_failOf500", 20L,
aResponse().withStatus(500))
);
}
@@ -95,8 +97,8 @@ public void setup(WireMockRuntimeInfo wiremock) {
.credentialsProvider(
StaticCredentialsProvider.create(AwsBasicCredentials.create("key", "secret")))
.multipartEnabled(true)
- .multipartConfiguration(b -> b.minimumPartSizeInBytes(10L).apiCallBufferSizeInBytes(10L))
- .httpClientBuilder(AwsCrtAsyncHttpClient.builder())
+ .multipartConfiguration(b -> b.minimumPartSizeInBytes(10L).apiCallBufferSizeInBytes(20L))
+ .httpClientBuilder(NettyNioAsyncHttpClient.builder())
.build();
}
@@ -144,10 +146,10 @@ void uploadWithKnownContentLength_onePartFails_shouldCancelUpstream() {
@ParameterizedTest
@MethodSource("retryableErrorTestCase")
- void mpu_partsFailOfRetryableError_shouldRetry(String description,
+ void mpuWithBufferedSplittableAsyncRequestBody_partsFailOfRetryableError_shouldRetry(String description,
Long contentLength,
ResponseDefinitionBuilder responseDefinitionBuilder) {
- stubUploadPartFailsInitialAttemptCalls(responseDefinitionBuilder);
+ stubUploadPartFailsInitialAttemptSucceedsUponRetryCalls(responseDefinitionBuilder);
List buffers = new ArrayList<>();
buffers.add(SdkBytes.fromUtf8String(RandomStringUtils.randomAscii(10)).asByteBuffer());
buffers.add(SdkBytes.fromUtf8String(RandomStringUtils.randomAscii(10)).asByteBuffer());
@@ -163,14 +165,45 @@ public void subscribe(Subscriber super ByteBuffer> s) {
}
};
- s3AsyncClient.putObject(b -> b.bucket(BUCKET).key(KEY), asyncRequestBody).join();
+ s3AsyncClient.putObject(b -> b.bucket(BUCKET).key(KEY), BufferedSplittableAsyncRequestBody.create(asyncRequestBody))
+ .join();
verify(2, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(1))));
verify(2, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(2))));
}
+ @ParameterizedTest
+ @MethodSource("retryableErrorTestCase")
+ void mpuDefaultSplitImpl_partsFailOfRetryableError_shouldFail(String description,
+ Long contentLength,
+ ResponseDefinitionBuilder responseDefinitionBuilder) {
+ stubUploadPartFailsInitialAttemptSucceedsUponRetryCalls(responseDefinitionBuilder);
+ List buffers = new ArrayList<>();
+ buffers.add(SdkBytes.fromUtf8String(RandomStringUtils.randomAscii(10)).asByteBuffer());
+ buffers.add(SdkBytes.fromUtf8String(RandomStringUtils.randomAscii(10)).asByteBuffer());
+ AsyncRequestBody asyncRequestBody = new AsyncRequestBody() {
+ @Override
+ public Optional contentLength() {
+ return Optional.ofNullable(contentLength);
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ Flowable.fromIterable(buffers).subscribe(s);
+ }
+ };
+
+ assertThatThrownBy(() -> s3AsyncClient.putObject(b -> b.bucket(BUCKET).key(KEY), asyncRequestBody)
+ .join())
+ .hasCauseInstanceOf(NonRetryableException.class)
+ .hasMessageContaining("A retry was attempted, but");
+
+ verify(1, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(1))));
+ verify(1, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(1))));
+ }
+
- private void stubUploadPartFailsInitialAttemptCalls(ResponseDefinitionBuilder responseDefinitionBuilder) {
+ private void stubUploadPartFailsInitialAttemptSucceedsUponRetryCalls(ResponseDefinitionBuilder responseDefinitionBuilder) {
stubFor(post(anyUrl()).willReturn(aResponse().withStatus(200).withBody(CREATE_MULTIPART_PAYLOAD)));
stubUploadFailsInitialAttemptCalls(1, responseDefinitionBuilder);
stubUploadFailsInitialAttemptCalls(2, responseDefinitionBuilder);
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
index e5c96bc6abed..82de9fa606d2 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
@@ -26,12 +26,9 @@
import static software.amazon.awssdk.services.s3.internal.multipart.MpuTestUtils.stubSuccessfulCreateMultipartCall;
import static software.amazon.awssdk.services.s3.internal.multipart.MpuTestUtils.stubSuccessfulUploadPartCalls;
-import java.io.ByteArrayInputStream;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
@@ -54,12 +51,10 @@
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompletedPart;
-import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
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.testutils.RandomTempFile;
-import software.amazon.awssdk.utils.StringInputStream;
public class UploadWithUnknownContentLengthHelperTest {
private static final String BUCKET = "bucket";
@@ -155,7 +150,7 @@ private ClosableAsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLengt
private CompletableFuture setupAndTriggerUploadFailure(ClosableAsyncRequestBody asyncRequestBody) {
SdkPublisher mockPublisher = mock(SdkPublisher.class);
- when(asyncRequestBody.splitV2(any(Consumer.class))).thenReturn(mockPublisher);
+ when(asyncRequestBody.splitClosable(any(Consumer.class))).thenReturn(mockPublisher);
ArgumentCaptor> subscriberCaptor = ArgumentCaptor.forClass(Subscriber.class);
CompletableFuture future = helper.uploadObject(createPutObjectRequest(), asyncRequestBody);
diff --git a/services/s3/src/test/resources/log4j2.properties b/services/s3/src/test/resources/log4j2.properties
index 8f3afbf09abe..fc101dc6e0d7 100644
--- a/services/s3/src/test/resources/log4j2.properties
+++ b/services/s3/src/test/resources/log4j2.properties
@@ -35,4 +35,7 @@ rootLogger.appenderRef.stdout.ref = ConsoleAppender
#logger.apache.level = debug
#
#logger.netty.name = io.netty.handler.logging
-#logger.netty.level = debug
\ No newline at end of file
+#logger.netty.level = debug
+
+#logger.multipart.name = software.amazon.awssdk.services.s3.internal.multipart
+#logger.multipart.level = debug
\ No newline at end of file
From 0c2000e75b49755bf9bb17f51d79f1d2c3efd61b Mon Sep 17 00:00:00 2001
From: Zoe Wang <33073555+zoewangg@users.noreply.github.com>
Date: Thu, 21 Aug 2025 15:45:30 -0700
Subject: [PATCH 4/5] Add more tests and address comments
---
.../awssdk/core/async/AsyncRequestBody.java | 25 ++-
.../BufferedSplittableAsyncRequestBody.java | 69 +++++-
...dy.java => CloseableAsyncRequestBody.java} | 2 +-
.../listener/AsyncRequestBodyListener.java | 10 +-
.../internal/async/FileAsyncRequestBody.java | 6 +-
.../NonRetryableSubAsyncRequestBody.java | 17 +-
.../async/RetryableSubAsyncRequestBody.java | 17 +-
.../internal/async/SplittingPublisher.java | 87 ++++++--
.../internal/async/SubAsyncRequestBody.java | 6 +-
.../core/async/AsyncRequestBodyTest.java | 2 +-
.../NonRetryableSubAsyncRequestBodyTest.java | 152 +++++++++++++
.../RetryableSubAsyncRequestBodyTest.java | 203 ++++++++++++++++++
.../async/SplittingPublisherTest.java | 71 +++---
.../async/SplittingPublisherTestUtils.java | 6 +-
.../src/test/resources/log4j2.properties | 4 +-
...ltipartClientPutObjectIntegrationTest.java | 98 +++------
...ntentLengthAsyncRequestBodySubscriber.java | 6 +-
.../UploadWithKnownContentLengthHelper.java | 4 +-
.../UploadWithUnknownContentLengthHelper.java | 16 +-
...tLengthAsyncRequestBodySubscriberTest.java | 14 +-
...3MultipartClientPutObjectWiremockTest.java | 2 +-
...oadWithUnknownContentLengthHelperTest.java | 24 +--
.../CodingConventionWithSuppressionTest.java | 4 +-
.../UploadStreamingRegressionTesting.java | 23 +-
24 files changed, 673 insertions(+), 195 deletions(-)
rename core/sdk-core/src/main/java/software/amazon/awssdk/core/async/{ClosableAsyncRequestBody.java => CloseableAsyncRequestBody.java} (90%)
create mode 100644 core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBodyTest.java
create mode 100644 core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBodyTest.java
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
index f5282d71a736..c00e39382807 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/AsyncRequestBody.java
@@ -516,11 +516,16 @@ static AsyncRequestBody empty() {
*/
default SdkPublisher split(AsyncRequestBodySplitConfiguration splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
- return new SplittingPublisher(this, splitConfiguration, false).map(r -> r);
+ return SplittingPublisher.builder()
+ .asyncRequestBody(this)
+ .splitConfiguration(splitConfiguration)
+ .retryableSubAsyncRequestBodyEnabled(false)
+ .build()
+ .map(r -> r);
}
/**
- * Converts this {@link AsyncRequestBody} to a publisher of {@link ClosableAsyncRequestBody}s, each of which publishes
+ * Converts this {@link AsyncRequestBody} to a publisher of {@link CloseableAsyncRequestBody}s, each of which publishes
* specific portion of the original data, based on the provided {@link AsyncRequestBodySplitConfiguration}. The default chunk
* size is 2MB and the default buffer size is 8MB.
*
@@ -529,14 +534,18 @@ default SdkPublisher split(AsyncRequestBodySplitConfiguration
* vary in different implementations.
*
*
- * Caller is responsible for closing {@link ClosableAsyncRequestBody} when it is ready to be disposed to release any
+ * Caller is responsible for closing {@link CloseableAsyncRequestBody} when it is ready to be disposed to release any
* resources.
*
* @see AsyncRequestBodySplitConfiguration
*/
- default SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ default SdkPublisher splitCloseable(AsyncRequestBodySplitConfiguration splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
- return new SplittingPublisher(this, splitConfiguration, false);
+ return SplittingPublisher.builder()
+ .asyncRequestBody(this)
+ .splitConfiguration(splitConfiguration)
+ .retryableSubAsyncRequestBodyEnabled(false)
+ .build();
}
/**
@@ -554,12 +563,12 @@ default SdkPublisher split(Consumer splitClosable(
+ default SdkPublisher splitCloseable(
Consumer splitConfiguration) {
Validate.notNull(splitConfiguration, "splitConfiguration");
- return splitClosable(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
+ return splitCloseable(AsyncRequestBodySplitConfiguration.builder().applyMutation(splitConfiguration).build());
}
@SdkProtectedApi
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java
index 21d701109d2c..27d4df56ecff 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/BufferedSplittableAsyncRequestBody.java
@@ -20,10 +20,43 @@
import org.reactivestreams.Subscriber;
import software.amazon.awssdk.annotations.SdkPublicApi;
import software.amazon.awssdk.core.internal.async.SplittingPublisher;
+import software.amazon.awssdk.utils.Validate;
/**
- * An {@link AsyncRequestBody} decorator that can be split into buffered sub {@link AsyncRequestBody}s. Each sub
- * {@link AsyncRequestBody} can be retried/resubscribed if all data has been successfully been published to first subscriber.
+ * An {@link AsyncRequestBody} decorator that enables splitting into retryable sub-request bodies.
+ *
+ * This wrapper allows any {@link AsyncRequestBody} to be split into multiple parts where each part
+ * can be retried independently. When split, each sub-body buffers its portion of data, enabling
+ * resubscription if a retry is needed (e.g., due to network failures or service errors).
+ *
+ * Retry Requirements:
+ * Retry is only possible if all the data has been successfully buffered during the first subscription.
+ * If the first subscriber fails to consume all the data (e.g., due to early cancellation or errors),
+ * subsequent retry attempts will fail since the complete data set is not available for resubscription.
+ *
+ * Usage Example:
+ * {@code
+ * AsyncRequestBody originalBody = AsyncRequestBody.fromString("Hello World");
+ * BufferedSplittableAsyncRequestBody retryableBody =
+ * BufferedSplittableAsyncRequestBody.create(originalBody);
+ *
+ * AsyncRequestBodySplitConfiguration config = AsyncRequestBodySplitConfiguration.builder()
+ * .chunkSizeInBytes(1024)
+ * .bufferSizeInBytes(2048)
+ * .build();
+ *
+ * SdkPublisher parts = retryableBody.splitClosable(config);
+ * }
+ *
+ * Performance Considerations:
+ * This implementation buffers data in memory to enable retries, but memory usage is controlled by
+ * the {@code bufferSizeInBytes} configuration. However, this buffering limits the ability to request
+ * more data from the original AsyncRequestBody until buffered data is consumed (i.e., when subscribers
+ * closes sub-body), which may increase latency compared to non-buffered implementations.
+ *
+ * @see AsyncRequestBody
+ * @see AsyncRequestBodySplitConfiguration
+ * @see CloseableAsyncRequestBody
*/
@SdkPublicApi
public final class BufferedSplittableAsyncRequestBody implements AsyncRequestBody {
@@ -33,7 +66,15 @@ private BufferedSplittableAsyncRequestBody(AsyncRequestBody delegate) {
this.delegate = delegate;
}
+ /**
+ * Creates a new {@link BufferedSplittableAsyncRequestBody} that wraps the provided {@link AsyncRequestBody}.
+ *
+ * @param delegate the {@link AsyncRequestBody} to wrap and make retryable. Must not be null.
+ * @return a new {@link BufferedSplittableAsyncRequestBody} instance
+ * @throws NullPointerException if delegate is null
+ */
public static BufferedSplittableAsyncRequestBody create(AsyncRequestBody delegate) {
+ Validate.paramNotNull(delegate, "delegate");
return new BufferedSplittableAsyncRequestBody(delegate);
}
@@ -42,9 +83,29 @@ public Optional contentLength() {
return delegate.contentLength();
}
+ /**
+ * Splits this request body into multiple retryable parts based on the provided configuration.
+ *
+ * Each part returned by the publisher will be a {@link CloseableAsyncRequestBody} that buffers
+ * its portion of data, enabling resubscription for retry scenarios. This is the key difference from non-buffered splitting -
+ * each part can be safely retried without data loss.
+ *
+ *
The splitting process respects the chunk size and buffer size specified in the configuration
+ * to optimize memory usage.
+ *
+ *
The subscriber MUST close each {@link CloseableAsyncRequestBody} to ensure resource is released
+ *
+ * @param splitConfiguration configuration specifying how to split the request body
+ * @return a publisher that emits retryable closable request body parts
+ * @see AsyncRequestBodySplitConfiguration
+ */
@Override
- public SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
- return new SplittingPublisher(this, splitConfiguration, true);
+ public SdkPublisher splitCloseable(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ return SplittingPublisher.builder()
+ .asyncRequestBody(this)
+ .splitConfiguration(splitConfiguration)
+ .retryableSubAsyncRequestBodyEnabled(true)
+ .build();
}
@Override
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/CloseableAsyncRequestBody.java
similarity index 90%
rename from core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java
rename to core/sdk-core/src/main/java/software/amazon/awssdk/core/async/CloseableAsyncRequestBody.java
index 7f495883c477..18c656c3018b 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/ClosableAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/CloseableAsyncRequestBody.java
@@ -22,5 +22,5 @@
* An extension of {@link AsyncRequestBody} that is closable.
*/
@SdkPublicApi
-public interface ClosableAsyncRequestBody extends AsyncRequestBody, SdkAutoCloseable {
+public interface CloseableAsyncRequestBody extends AsyncRequestBody, SdkAutoCloseable {
}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
index 71a6f87a4288..8835d1a007ae 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/async/listener/AsyncRequestBodyListener.java
@@ -23,7 +23,7 @@
import software.amazon.awssdk.annotations.SdkProtectedApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.utils.Logger;
import software.amazon.awssdk.utils.Validate;
@@ -78,14 +78,14 @@ public SdkPublisher split(Consumer splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
- return delegate.splitClosable(splitConfiguration);
+ public SdkPublisher splitCloseable(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ return delegate.splitCloseable(splitConfiguration);
}
@Override
- public SdkPublisher splitClosable(
+ public SdkPublisher splitCloseable(
Consumer splitConfiguration) {
- return delegate.splitClosable(splitConfiguration);
+ return delegate.splitCloseable(splitConfiguration);
}
@Override
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
index 0ffb08dc15d1..2af70796f4e1 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/FileAsyncRequestBody.java
@@ -34,7 +34,7 @@
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.core.internal.util.Mimetype;
import software.amazon.awssdk.core.internal.util.NoopSubscription;
@@ -88,7 +88,7 @@ public SdkPublisher split(AsyncRequestBodySplitConfiguration s
}
@Override
- public SdkPublisher splitClosable(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ public SdkPublisher splitCloseable(AsyncRequestBodySplitConfiguration splitConfiguration) {
return split(splitConfiguration).map(body -> new ClosableAsyncRequestBodyWrapper(body));
}
@@ -443,7 +443,7 @@ private static AsynchronousFileChannel openInputChannel(Path path) throws IOExce
return AsynchronousFileChannel.open(path, StandardOpenOption.READ);
}
- private static class ClosableAsyncRequestBodyWrapper implements ClosableAsyncRequestBody {
+ private static class ClosableAsyncRequestBodyWrapper implements CloseableAsyncRequestBody {
private final AsyncRequestBody delegate;
ClosableAsyncRequestBodyWrapper(AsyncRequestBody body) {
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java
index eb30b502d8c4..221f9246a5e6 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBody.java
@@ -62,11 +62,11 @@ public Optional contentLength() {
public void send(ByteBuffer data) {
log.debug(() -> String.format("Sending bytebuffer %s to part %d", data, partNumber));
- int length = data.remaining();
+ long length = data.remaining();
bufferedLength += length;
- onNumBytesReceived.accept((long) length);
+ onNumBytesReceived.accept(length);
delegate.send(data).whenComplete((r, t) -> {
- onNumBytesConsumed.accept((long) length);
+ onNumBytesConsumed.accept(length);
if (t != null) {
error(t);
}
@@ -92,11 +92,6 @@ public long receivedBytesLength() {
return bufferedLength;
}
- @Override
- public boolean contentLengthKnown() {
- return contentLengthKnown;
- }
-
@Override
public int partNumber() {
return partNumber;
@@ -113,8 +108,10 @@ public void subscribe(Subscriber super ByteBuffer> s) {
} else {
s.onSubscribe(new NoopSubscription(s));
s.onError(NonRetryableException.create(
- "A retry was attempted, but the provided source AsyncRequestBody does not "
- + "support splitting to retryable AsyncRequestBody. Consider using BufferedSplittableAsyncRequestBody."));
+ "Multiple subscribers detected. This could happen due to a retry attempt. The AsyncRequestBody implementation"
+ + " provided does not support splitting to retryable/resubscribable AsyncRequestBody. If you need retry "
+ + "capability or multiple subscriptions, consider using BufferedSplittableAsyncRequestBody to wrap your "
+ + "AsyncRequestBody."));
}
}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java
index 00f4127ca061..15f8b0199107 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBody.java
@@ -75,10 +75,10 @@ public Optional contentLength() {
@Override
public void send(ByteBuffer data) {
log.trace(() -> String.format("Sending bytebuffer %s to part number %d", data, partNumber));
- int length = data.remaining();
+ long length = data.remaining();
bufferedLength += length;
- onNumBytesReceived.accept((long) length);
+ onNumBytesReceived.accept(length);
delegate.send(data.asReadOnlyBuffer()).whenComplete((r, t) -> {
if (t != null) {
delegate.error(t);
@@ -94,7 +94,9 @@ public void complete() {
log.debug(() -> "Received complete() for part number: " + partNumber);
// ByteBuffersAsyncRequestBody MUST be created before we complete the current
// request because retry may happen right after
- bufferedAsyncRequestBody = ByteBuffersAsyncRequestBody.of(buffers, bufferedLength);
+ synchronized (buffersLock) {
+ bufferedAsyncRequestBody = ByteBuffersAsyncRequestBody.of(buffers, bufferedLength);
+ }
delegate.complete().exceptionally(e -> {
delegate.error(e);
return null;
@@ -121,7 +123,7 @@ public void subscribe(Subscriber super ByteBuffer> s) {
if (bufferedAsyncRequestBody == null) {
s.onSubscribe(new NoopSubscription(s));
s.onError(NonRetryableException.create(
- "A retry was attempted, but data is not buffered successfully for retry, partNumber " + partNumber));
+ "A retry was attempted, but data is not buffered successfully for retry for partNumber: " + partNumber));
return;
}
bufferedAsyncRequestBody.subscribe(s);
@@ -139,7 +141,7 @@ public void close() {
buffers = null;
}
bufferedAsyncRequestBody.close();
- log.debug(() -> "requesting data after closing" + partNumber);
+ bufferedAsyncRequestBody = null;
}
} catch (Throwable e) {
log.warn(() -> String.format("Unexpected error thrown from cleaning up AsyncRequestBody for part number %d, "
@@ -147,11 +149,6 @@ public void close() {
}
}
- @Override
- public boolean contentLengthKnown() {
- return contentLengthKnown;
- }
-
@Override
public int partNumber() {
return partNumber;
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
index d7e16ca5445c..d4b58b0285e3 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SplittingPublisher.java
@@ -24,7 +24,7 @@
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncRequestBodySplitConfiguration;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.utils.Logger;
import software.amazon.awssdk.utils.Validate;
@@ -38,34 +38,33 @@
* Otherwise, it is sent after the entire content for that chunk is buffered. This is required to get content length.
*/
@SdkInternalApi
-public class SplittingPublisher implements SdkPublisher {
+public class SplittingPublisher implements SdkPublisher {
private static final Logger log = Logger.loggerFor(SplittingPublisher.class);
private final AsyncRequestBody upstreamPublisher;
private final SplittingSubscriber splittingSubscriber;
- private final SimplePublisher downstreamPublisher = new SimplePublisher<>();
+ private final SimplePublisher downstreamPublisher = new SimplePublisher<>();
private final long chunkSizeInBytes;
private final long bufferSizeInBytes;
- private final boolean enableRetryableSubAsyncRequestBody;
+ private final boolean retryableSubAsyncRequestBodyEnabled;
private final AtomicBoolean currentBodySent = new AtomicBoolean(false);
private final String sourceBodyName;
- public SplittingPublisher(AsyncRequestBody asyncRequestBody,
- AsyncRequestBodySplitConfiguration splitConfiguration,
- boolean enableRetryableSubAsyncRequestBody) {
- this.upstreamPublisher = Validate.paramNotNull(asyncRequestBody, "asyncRequestBody");
- Validate.notNull(splitConfiguration, "splitConfiguration");
- this.chunkSizeInBytes = splitConfiguration.chunkSizeInBytes() == null ?
+ private SplittingPublisher(Builder builder) {
+ this.upstreamPublisher = Validate.paramNotNull(builder.asyncRequestBody, "asyncRequestBody");
+ Validate.notNull(builder.splitConfiguration, "splitConfiguration");
+ this.chunkSizeInBytes = builder.splitConfiguration.chunkSizeInBytes() == null ?
AsyncRequestBodySplitConfiguration.defaultConfiguration().chunkSizeInBytes() :
- splitConfiguration.chunkSizeInBytes();
+ builder.splitConfiguration.chunkSizeInBytes();
- this.bufferSizeInBytes = splitConfiguration.bufferSizeInBytes() == null ?
+ this.bufferSizeInBytes = builder.splitConfiguration.bufferSizeInBytes() == null ?
AsyncRequestBodySplitConfiguration.defaultConfiguration().bufferSizeInBytes() :
- splitConfiguration.bufferSizeInBytes();
+ builder.splitConfiguration.bufferSizeInBytes();
this.splittingSubscriber = new SplittingSubscriber(upstreamPublisher.contentLength().orElse(null));
- this.enableRetryableSubAsyncRequestBody = enableRetryableSubAsyncRequestBody;
- this.sourceBodyName = asyncRequestBody.body();
+ this.retryableSubAsyncRequestBodyEnabled = Validate.paramNotNull(builder.retryableSubAsyncRequestBodyEnabled,
+ "retryableSubAsyncRequestBodyEnabled");
+ this.sourceBodyName = builder.asyncRequestBody.body();
if (!upstreamPublisher.contentLength().isPresent()) {
Validate.isTrue(bufferSizeInBytes >= chunkSizeInBytes,
"bufferSizeInBytes must be larger than or equal to " +
@@ -73,8 +72,15 @@ public SplittingPublisher(AsyncRequestBody asyncRequestBody,
}
}
+ /**
+ * Returns a newly initialized builder object for a {@link SplittingPublisher}
+ */
+ public static Builder builder() {
+ return new Builder();
+ }
+
@Override
- public void subscribe(Subscriber super ClosableAsyncRequestBody> downstreamSubscriber) {
+ public void subscribe(Subscriber super CloseableAsyncRequestBody> downstreamSubscriber) {
downstreamPublisher.subscribe(downstreamSubscriber);
upstreamPublisher.subscribe(splittingSubscriber);
}
@@ -123,7 +129,7 @@ private SubAsyncRequestBody initializeNextDownstreamBody(boolean contentLengthKn
.sourceBodyName(sourceBodyName)
.build();
- if (enableRetryableSubAsyncRequestBody) {
+ if (retryableSubAsyncRequestBodyEnabled) {
body = new RetryableSubAsyncRequestBody(config);
} else {
body = new NonRetryableSubAsyncRequestBody(config);
@@ -206,15 +212,16 @@ private int amountRemainingInChunk() {
private void completeCurrentBody() {
log.debug(() -> "completeCurrentBody for part " + currentBody.partNumber());
- // For unknown content length, we always create a new DownstreamBody because we don't know if there is data
- // left or not, so we need to only send the body if there is actually data
long bufferedLength = currentBody.receivedBytesLength();
+ // For unknown content length, we always create a new DownstreamBody once the current one is sent
+ // because we don't know if there is data
+ // left or not, so we need to check the length and only send the body if there is actually data
if (bufferedLength == 0) {
return;
}
Long totalLength = currentBody.maxLength();
- if (currentBody.contentLengthKnown() && totalLength != bufferedLength) {
+ if (upstreamSize != null && totalLength != bufferedLength) {
upstreamSubscription.cancel();
downstreamPublisher.error(new IllegalStateException(
String.format("Content length of buffered data mismatches "
@@ -294,4 +301,44 @@ private void addDataBuffered(long length) {
}
}
}
+
+ public static final class Builder {
+ private AsyncRequestBody asyncRequestBody;
+ private AsyncRequestBodySplitConfiguration splitConfiguration;
+ private Boolean retryableSubAsyncRequestBodyEnabled;
+
+ private Builder() {
+ }
+
+ /**
+ * Sets the AsyncRequestBody to be split.
+ */
+ public Builder asyncRequestBody(AsyncRequestBody asyncRequestBody) {
+ this.asyncRequestBody = asyncRequestBody;
+ return this;
+ }
+
+ /**
+ * Sets the split configuration.
+ */
+ public Builder splitConfiguration(AsyncRequestBodySplitConfiguration splitConfiguration) {
+ this.splitConfiguration = splitConfiguration;
+ return this;
+ }
+
+ /**
+ * Sets whether to enable retryable sub async request bodies.
+ */
+ public Builder retryableSubAsyncRequestBodyEnabled(Boolean retryableSubAsyncRequestBodyEnabled) {
+ this.retryableSubAsyncRequestBodyEnabled = retryableSubAsyncRequestBodyEnabled;
+ return this;
+ }
+
+ /**
+ * Builds a {@link SplittingPublisher} object based on the values held by this builder.
+ */
+ public SplittingPublisher build() {
+ return new SplittingPublisher(this);
+ }
+ }
}
\ No newline at end of file
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java
index c29446eb4929..6d3ec1b979a6 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/SubAsyncRequestBody.java
@@ -18,13 +18,13 @@
import java.nio.ByteBuffer;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
/**
* Represent a sub {@link AsyncRequestBody} that publishes a portion of the source {@link AsyncRequestBody}
*/
@SdkInternalApi
-public interface SubAsyncRequestBody extends ClosableAsyncRequestBody {
+public interface SubAsyncRequestBody extends CloseableAsyncRequestBody {
/**
* Send a byte buffer.
@@ -55,8 +55,6 @@ default void close() {
// no op
}
- boolean contentLengthKnown();
-
/**
* The part number associated with this SubAsyncRequestBody
* @return
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
index 1c363eb90fe8..f0c58c37e9a4 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/async/AsyncRequestBodyTest.java
@@ -415,7 +415,7 @@ void publisherConstructorHasCorrectContentType() {
void splitV2_nullConfig_shouldThrowException() {
AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world");
AsyncRequestBodySplitConfiguration config = null;
- assertThatThrownBy(() -> requestBody.splitClosable(config))
+ assertThatThrownBy(() -> requestBody.splitCloseable(config))
.isInstanceOf(NullPointerException.class)
.hasMessageContaining("splitConfig");
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBodyTest.java
new file mode 100644
index 000000000000..583aa077f21c
--- /dev/null
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/NonRetryableSubAsyncRequestBodyTest.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import io.reactivex.Flowable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Consumer;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+
+class NonRetryableSubAsyncRequestBodyTest {
+
+ private SubAsyncRequestBodyConfiguration configuration;
+ private Consumer onNumBytesReceived;
+ private Consumer onNumBytesConsumed;
+ private NonRetryableSubAsyncRequestBody requestBody;
+
+ @BeforeEach
+ void setUp() {
+ onNumBytesReceived = mock(Consumer.class);
+ onNumBytesConsumed = mock(Consumer.class);
+
+ configuration = SubAsyncRequestBodyConfiguration.builder()
+ .contentLengthKnown(true)
+ .maxLength(1024L)
+ .partNumber(1)
+ .onNumBytesReceived(onNumBytesReceived)
+ .onNumBytesConsumed(onNumBytesConsumed)
+ .sourceBodyName("test-body")
+ .build();
+
+ requestBody = new NonRetryableSubAsyncRequestBody(configuration);
+ }
+
+ @Test
+ void getters_shouldReturnConfigurationValues() {
+ assertThat(requestBody.maxLength()).isEqualTo(1024L);
+ assertThat(requestBody.partNumber()).isEqualTo(1);
+ assertThat(requestBody.body()).isEqualTo("test-body");
+ assertThat(requestBody.contentLength()).isEqualTo(Optional.of(1024L));
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(0L);
+ }
+
+ @Test
+ void constructor_withNullConfiguration_shouldThrowException() {
+ assertThatThrownBy(() -> new NonRetryableSubAsyncRequestBody(null))
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ @Test
+ void contentLength_whenContentLengthUnknown_shouldReturnBufferedLength() {
+ SubAsyncRequestBodyConfiguration unknownLengthConfig = SubAsyncRequestBodyConfiguration.builder()
+ .contentLengthKnown(false)
+ .maxLength(1024L)
+ .partNumber(1)
+ .onNumBytesReceived(onNumBytesReceived)
+ .onNumBytesConsumed(onNumBytesConsumed)
+ .sourceBodyName("test-body")
+ .build();
+
+ NonRetryableSubAsyncRequestBody unknownLengthBody = new NonRetryableSubAsyncRequestBody(unknownLengthConfig);
+
+ assertThat(unknownLengthBody.contentLength()).isEqualTo(Optional.of(0L));
+
+ // Send some data
+ ByteBuffer data = ByteBuffer.wrap("test".getBytes());
+ unknownLengthBody.send(data);
+
+ assertThat(unknownLengthBody.contentLength()).isEqualTo(Optional.of(4L));
+ }
+
+ @Test
+ void subscribe_shouldReceiveAllData() {
+ byte[] part1 = RandomStringUtils.randomAscii(1024).getBytes(StandardCharsets.UTF_8);
+ byte[] part2 = RandomStringUtils.randomAscii(512).getBytes(StandardCharsets.UTF_8);
+ requestBody.send(ByteBuffer.wrap(part1));
+ requestBody.send(ByteBuffer.wrap(part2));
+ requestBody.complete();
+ List receivedBuffers = new ArrayList<>();
+ Flowable.fromPublisher(requestBody).forEach(buffer -> receivedBuffers.add(buffer));
+
+ verify(onNumBytesReceived).accept(1024L);
+ verify(onNumBytesConsumed).accept(1024L);
+ verify(onNumBytesReceived).accept(512L);
+ verify(onNumBytesConsumed).accept(512L);
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(1536L);
+ assertThat(receivedBuffers).containsExactly(ByteBuffer.wrap(part1), ByteBuffer.wrap(part2));
+ }
+
+ @Test
+ void subscribe_secondTime_shouldSendError() {
+ Subscriber subscriber1 = mock(Subscriber.class);
+ Subscriber subscriber2 = mock(Subscriber.class);
+
+ // First subscription
+ requestBody.subscribe(subscriber1);
+
+ // Second subscription should fail
+ requestBody.subscribe(subscriber2);
+
+ ArgumentCaptor subscriptionCaptor = ArgumentCaptor.forClass(Subscription.class);
+ verify(subscriber2).onSubscribe(subscriptionCaptor.capture());
+
+ ArgumentCaptor errorCaptor = ArgumentCaptor.forClass(Throwable.class);
+ verify(subscriber2).onError(errorCaptor.capture());
+
+ Throwable error = errorCaptor.getValue();
+ assertThat(error).isInstanceOf(NonRetryableException.class);
+ assertThat(error.getMessage()).contains("This could happen due to a retry attempt");
+ }
+
+ @Test
+ void receivedBytesLength_shouldTrackSentData() {
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(0L);
+
+ ByteBuffer data1 = ByteBuffer.wrap("hello".getBytes());
+ requestBody.send(data1);
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(5L);
+
+ ByteBuffer data2 = ByteBuffer.wrap(" world".getBytes());
+ requestBody.send(data2);
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(11L);
+ }
+}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBodyTest.java
new file mode 100644
index 000000000000..969bdb99cb4a
--- /dev/null
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/RetryableSubAsyncRequestBodyTest.java
@@ -0,0 +1,203 @@
+/*
+ * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License").
+ * You may not use this file except in compliance with the License.
+ * A copy of the License is located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file 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 software.amazon.awssdk.core.internal.async;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import io.reactivex.Flowable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import software.amazon.awssdk.core.exception.NonRetryableException;
+
+class RetryableSubAsyncRequestBodyTest {
+
+ private SubAsyncRequestBodyConfiguration configuration;
+ private Consumer onNumBytesReceived;
+ private Consumer onNumBytesConsumed;
+ private RetryableSubAsyncRequestBody requestBody;
+
+ @BeforeEach
+ void setUp() {
+ onNumBytesReceived = mock(Consumer.class);
+ onNumBytesConsumed = mock(Consumer.class);
+
+ configuration = SubAsyncRequestBodyConfiguration.builder()
+ .contentLengthKnown(true)
+ .maxLength(1024L)
+ .partNumber(1)
+ .onNumBytesReceived(onNumBytesReceived)
+ .onNumBytesConsumed(onNumBytesConsumed)
+ .sourceBodyName("test-body")
+ .build();
+
+ requestBody = new RetryableSubAsyncRequestBody(configuration);
+ }
+
+ @Test
+ void getters_shouldReturnConfigurationValues() {
+ assertThat(requestBody.maxLength()).isEqualTo(1024L);
+ assertThat(requestBody.partNumber()).isEqualTo(1);
+ assertThat(requestBody.body()).isEqualTo("test-body");
+ assertThat(requestBody.contentLength()).isEqualTo(Optional.of(1024L));
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(0L);
+ }
+
+ @Test
+ void constructor_withNullConfiguration_shouldThrowException() {
+ assertThatThrownBy(() -> new RetryableSubAsyncRequestBody(null))
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ @Test
+ void contentLength_whenContentLengthUnknown_shouldReturnBufferedLength() {
+ SubAsyncRequestBodyConfiguration unknownLengthConfig = SubAsyncRequestBodyConfiguration.builder()
+ .contentLengthKnown(false)
+ .maxLength(1024L)
+ .partNumber(1)
+ .onNumBytesReceived(onNumBytesReceived)
+ .onNumBytesConsumed(onNumBytesConsumed)
+ .sourceBodyName("test-body")
+ .build();
+
+ RetryableSubAsyncRequestBody unknownLengthBody = new RetryableSubAsyncRequestBody(unknownLengthConfig);
+
+ assertThat(unknownLengthBody.contentLength()).isEqualTo(Optional.of(0L));
+
+ // Send some data
+ ByteBuffer data = ByteBuffer.wrap("test".getBytes());
+ unknownLengthBody.send(data);
+
+ assertThat(unknownLengthBody.contentLength()).isEqualTo(Optional.of(4L));
+ }
+
+ @Test
+ void subscribe_shouldReceiveAllData() {
+ byte[] part1 = RandomStringUtils.randomAscii(1024).getBytes(StandardCharsets.UTF_8);
+ byte[] part2 = RandomStringUtils.randomAscii(512).getBytes(StandardCharsets.UTF_8);
+ requestBody.send(ByteBuffer.wrap(part1));
+ requestBody.send(ByteBuffer.wrap(part2));
+ requestBody.complete();
+ List receivedBuffers = new ArrayList<>();
+ Flowable.fromPublisher(requestBody).forEach(buffer -> receivedBuffers.add(buffer));
+
+ verify(onNumBytesReceived).accept(1024L);
+ verify(onNumBytesReceived).accept(512L);
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(1536L);
+ assertThat(receivedBuffers).containsExactly(ByteBuffer.wrap(part1), ByteBuffer.wrap(part2));
+ }
+
+ @Test
+ void subscribe_secondTime_shouldUseBufferedBody() {
+ byte[] part1 = RandomStringUtils.randomAscii(1024).getBytes(StandardCharsets.UTF_8);
+ byte[] part2 = RandomStringUtils.randomAscii(512).getBytes(StandardCharsets.UTF_8);
+ requestBody.send(ByteBuffer.wrap(part1));
+ requestBody.send(ByteBuffer.wrap(part2));
+ requestBody.complete();
+
+ List buffer1 = new ArrayList<>();
+ Flowable.fromPublisher(requestBody).forEach(buffer -> buffer1.add(buffer));
+
+ List buffer2 = new ArrayList<>();
+ Flowable.fromPublisher(requestBody).forEach(buffer -> buffer2.add(buffer));
+
+ assertThat(buffer1).containsExactly(ByteBuffer.wrap(part1), ByteBuffer.wrap(part2));
+ assertThat(buffer2).containsExactly(ByteBuffer.wrap(part1), ByteBuffer.wrap(part2));
+ }
+
+ @Test
+ void subscribe_retryWithoutFirstSubscriberDone_shouldSendError() {
+ Subscriber subscriber1 = mock(Subscriber.class);
+ Subscriber subscriber2 = mock(Subscriber.class);
+
+ // First subscription
+ requestBody.subscribe(subscriber1);
+ // Second subscription without completing first (no buffered data)
+ requestBody.subscribe(subscriber2);
+
+ ArgumentCaptor subscriptionCaptor = ArgumentCaptor.forClass(Subscription.class);
+ verify(subscriber2).onSubscribe(subscriptionCaptor.capture());
+
+ ArgumentCaptor errorCaptor = ArgumentCaptor.forClass(Throwable.class);
+ verify(subscriber2).onError(errorCaptor.capture());
+
+ Throwable error = errorCaptor.getValue();
+ assertThat(error).isInstanceOf(NonRetryableException.class);
+ assertThat(error.getMessage()).contains("data is not buffered successfully for retry");
+ }
+
+ @Test
+ void subscribe_resubscribeAfterClose_shouldSendError() {
+ byte[] data = RandomStringUtils.randomAscii(1024).getBytes(StandardCharsets.UTF_8);
+ requestBody.send(ByteBuffer.wrap(data));
+ requestBody.complete();
+
+ Flowable.fromPublisher(requestBody).forEach(buffer -> {});
+
+ requestBody.close();
+ Subscriber secondSubscriber = mock(Subscriber.class);
+ requestBody.subscribe(secondSubscriber);
+
+ ArgumentCaptor subscriptionCaptor = ArgumentCaptor.forClass(Subscription.class);
+ verify(secondSubscriber).onSubscribe(subscriptionCaptor.capture());
+
+ ArgumentCaptor errorCaptor = ArgumentCaptor.forClass(Throwable.class);
+ verify(secondSubscriber).onError(errorCaptor.capture());
+
+ Throwable error = errorCaptor.getValue();
+ assertThat(error).isInstanceOf(NonRetryableException.class);
+ assertThat(error.getMessage()).contains("data is not buffered successfully for retry");
+ }
+
+ @Test
+ void close_shouldInvokeOnNumBytesConsumed() {
+ ByteBuffer data = ByteBuffer.wrap("test data".getBytes());
+ requestBody.send(data);
+
+ requestBody.close();
+
+ verify(onNumBytesConsumed).accept(9L);
+ }
+
+ @Test
+ void receivedBytesLength_shouldTrackSentData() {
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(0L);
+
+ ByteBuffer data1 = ByteBuffer.wrap("hello".getBytes());
+ requestBody.send(data1);
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(5L);
+
+ ByteBuffer data2 = ByteBuffer.wrap(" world".getBytes());
+ requestBody.send(data2);
+ assertThat(requestBody.receivedBytesLength()).isEqualTo(11L);
+ }
+}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
index 057d6c165eb8..87ac0b4726f4 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTest.java
@@ -81,7 +81,11 @@ public void split_contentUnknownMaxMemorySmallerThanChunkSize_shouldThrowExcepti
.chunkSizeInBytes(10L)
.bufferSizeInBytes(5L)
.build();
- assertThatThrownBy(() -> new SplittingPublisher(body, configuration, false))
+ assertThatThrownBy(() -> SplittingPublisher.builder()
+ .asyncRequestBody(body)
+ .splitConfiguration(configuration)
+ .retryableSubAsyncRequestBodyEnabled(false)
+ .build())
.hasMessageContaining("must be larger than or equal");
}
@@ -119,11 +123,14 @@ public Optional contentLength() {
return Optional.empty();
}
};
- SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
- AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes((long) CHUNK_SIZE)
- .bufferSizeInBytes(10L)
- .build(), enableRetryableSubAsyncRequestBody);
+ SplittingPublisher splittingPublisher = SplittingPublisher.builder()
+ .asyncRequestBody(asyncRequestBody)
+ .splitConfiguration(AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) CHUNK_SIZE)
+ .bufferSizeInBytes(10L)
+ .build())
+ .retryableSubAsyncRequestBodyEnabled(enableRetryableSubAsyncRequestBody)
+ .build();
List> futures = new ArrayList<>();
@@ -164,10 +171,14 @@ public Optional contentLength() {
void downStreamFailed_shouldPropagateCancellation(boolean enableRetryableSubAsyncRequestBody) throws Exception {
CompletableFuture future = new CompletableFuture<>();
TestAsyncRequestBody asyncRequestBody = new TestAsyncRequestBody();
- SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody, AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes((long) CHUNK_SIZE)
- .bufferSizeInBytes(10L)
- .build(), enableRetryableSubAsyncRequestBody);
+ SplittingPublisher splittingPublisher = SplittingPublisher.builder()
+ .asyncRequestBody(asyncRequestBody)
+ .splitConfiguration(AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) CHUNK_SIZE)
+ .bufferSizeInBytes(10L)
+ .build())
+ .retryableSubAsyncRequestBodyEnabled(enableRetryableSubAsyncRequestBody)
+ .build();
assertThatThrownBy(() -> splittingPublisher.subscribe(requestBody -> {
throw new RuntimeException("foobar");
}).get(5, TimeUnit.SECONDS)).hasMessageContaining("foobar");
@@ -182,12 +193,14 @@ void retryableSubAsyncRequestBodyEnabled_shouldBeAbleToResubscribe() throws Exec
.chunkSizeInBytes(chunkSize)
.build();
- SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
- AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes((long) chunkSize)
- .bufferSizeInBytes((long) chunkSize * 4)
- .build(),
- true);
+ SplittingPublisher splittingPublisher = SplittingPublisher.builder()
+ .asyncRequestBody(asyncRequestBody)
+ .splitConfiguration(AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) chunkSize)
+ .bufferSizeInBytes((long) chunkSize * 4)
+ .build())
+ .retryableSubAsyncRequestBodyEnabled(true)
+ .build();
@@ -217,21 +230,27 @@ void retryableSubAsyncRequestBodyEnabled_shouldBeAbleToResubscribe() throws Exec
}
private static void verifySplitContent(AsyncRequestBody asyncRequestBody, int chunkSize) throws Exception {
- SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
- AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes((long) chunkSize)
- .bufferSizeInBytes((long) chunkSize * 4)
- .build(), false);
+ SplittingPublisher splittingPublisher = SplittingPublisher.builder()
+ .asyncRequestBody(asyncRequestBody)
+ .splitConfiguration(AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) chunkSize)
+ .bufferSizeInBytes((long) chunkSize * 4)
+ .build())
+ .retryableSubAsyncRequestBodyEnabled(false)
+ .build();
verifyIndividualAsyncRequestBody(splittingPublisher.map(m -> m), testFile.toPath(), chunkSize);
}
private static void verifyRetryableSplitContent(AsyncRequestBody asyncRequestBody, int chunkSize) throws Exception {
- SplittingPublisher splittingPublisher = new SplittingPublisher(asyncRequestBody,
- AsyncRequestBodySplitConfiguration.builder()
- .chunkSizeInBytes((long) chunkSize)
- .bufferSizeInBytes((long) chunkSize * 4)
- .build(), false);
+ SplittingPublisher splittingPublisher = SplittingPublisher.builder()
+ .asyncRequestBody(asyncRequestBody)
+ .splitConfiguration(AsyncRequestBodySplitConfiguration.builder()
+ .chunkSizeInBytes((long) chunkSize)
+ .bufferSizeInBytes((long) chunkSize * 4)
+ .build())
+ .retryableSubAsyncRequestBodyEnabled(false)
+ .build();
verifyIndividualAsyncRequestBody(splittingPublisher.map(m -> m), testFile.toPath(), chunkSize);
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java
index 145a1cecc0ef..095877d9f2b5 100644
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/SplittingPublisherTestUtils.java
@@ -23,7 +23,7 @@
import java.util.concurrent.TimeUnit;
import org.assertj.core.api.Assertions;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
public final class SplittingPublisherTestUtils {
@@ -39,8 +39,8 @@ public static void verifyIndividualAsyncRequestBody(SdkPublisher {
- if (requestBody instanceof ClosableAsyncRequestBody) {
- ((ClosableAsyncRequestBody) requestBody).close();
+ if (requestBody instanceof CloseableAsyncRequestBody) {
+ ((CloseableAsyncRequestBody) requestBody).close();
}
});
futures.add(baosFuture);
diff --git a/core/sdk-core/src/test/resources/log4j2.properties b/core/sdk-core/src/test/resources/log4j2.properties
index e96677c00925..e5e68dd2faa4 100644
--- a/core/sdk-core/src/test/resources/log4j2.properties
+++ b/core/sdk-core/src/test/resources/log4j2.properties
@@ -25,8 +25,8 @@ rootLogger.appenderRef.stdout.ref = ConsoleAppender
# Uncomment below to enable more specific logging
#
-logger.sdk.name = software.amazon.awssdk
-logger.sdk.level = debug
+#logger.sdk.name = software.amazon.awssdk
+#logger.sdk.level = debug
#
#logger.request.name = software.amazon.awssdk.request
#logger.request.level = debug
diff --git a/services/s3/src/it/java/software/amazon/awssdk/services/s3/multipart/S3MultipartClientPutObjectIntegrationTest.java b/services/s3/src/it/java/software/amazon/awssdk/services/s3/multipart/S3MultipartClientPutObjectIntegrationTest.java
index e1f68ce2e234..de20ce7e6331 100644
--- a/services/s3/src/it/java/software/amazon/awssdk/services/s3/multipart/S3MultipartClientPutObjectIntegrationTest.java
+++ b/services/s3/src/it/java/software/amazon/awssdk/services/s3/multipart/S3MultipartClientPutObjectIntegrationTest.java
@@ -43,6 +43,7 @@
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
+import java.util.stream.Stream;
import java.util.zip.CRC32;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -54,6 +55,9 @@
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
import org.reactivestreams.Subscriber;
import software.amazon.awssdk.core.ClientType;
import software.amazon.awssdk.core.ResponseBytes;
@@ -61,6 +65,7 @@
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.async.AsyncResponseTransformer;
import software.amazon.awssdk.core.async.BlockingInputStreamAsyncRequestBody;
+import software.amazon.awssdk.core.async.BufferedSplittableAsyncRequestBody;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
@@ -75,6 +80,7 @@
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
+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.utils.ChecksumUtils;
@@ -93,7 +99,9 @@ public class S3MultipartClientPutObjectIntegrationTest extends S3IntegrationTest
private static final CapturingInterceptor CAPTURING_INTERCEPTOR = new CapturingInterceptor();
private static File testFile;
private static S3AsyncClient mpuS3Client;
- private static ExecutorService executorService = Executors.newFixedThreadPool(2);
+ private static ExecutorService executorService = Executors.newFixedThreadPool(5);
+ private static byte[] bytes;
+ private static byte[] expectedChecksum;
@BeforeAll
public static void setup() throws Exception {
@@ -101,6 +109,8 @@ public static void setup() throws Exception {
createBucket(TEST_BUCKET);
testFile = new RandomTempFile(OBJ_SIZE);
+ bytes = Files.readAllBytes(testFile.toPath());
+ expectedChecksum = ChecksumUtils.computeCheckSum(Files.newInputStream(testFile.toPath()));
mpuS3Client = S3AsyncClient
.builder()
.region(DEFAULT_REGION)
@@ -119,6 +129,18 @@ public static void teardown() throws Exception {
executorService.shutdown();
}
+ public static Stream asyncRequestBodies() {
+ return Stream.of(Arguments.of("file", AsyncRequestBody.fromFile(testFile)),
+ Arguments.of("bytes", AsyncRequestBody.fromBytes(bytes)),
+ Arguments.of("inputStream_knownLength",
+ AsyncRequestBody.fromInputStream(new ByteArrayInputStream(bytes), (long) bytes.length,
+ executorService)),
+ Arguments.of("inputStream_unknownLength",
+ AsyncRequestBody.fromInputStream(new ByteArrayInputStream(bytes), null,
+ executorService))
+ );
+ }
+
@BeforeEach
public void reset() {
CAPTURING_INTERCEPTOR.reset();
@@ -144,76 +166,28 @@ public void upload_blockingInputStream_shouldSucceed() throws IOException {
assertEquals(expectedMd5, actualMd5);
}
- @Test
- void putObject_fileRequestBody_objectSentCorrectly() throws Exception {
- AsyncRequestBody body = AsyncRequestBody.fromFile(testFile.toPath());
+ @ParameterizedTest
+ @MethodSource("asyncRequestBodies")
+ void putObject_variousRequestBody_objectSentCorrectly(String description, AsyncRequestBody body) throws Exception {
mpuS3Client.putObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY), body).join();
- assertThat(CAPTURING_INTERCEPTOR.createMpuChecksumAlgorithm).isEqualTo("CRC32");
- assertThat(CAPTURING_INTERCEPTOR.uploadPartChecksumAlgorithm).isEqualTo("CRC32");
-
ResponseInputStream objContent = s3.getObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY),
ResponseTransformer.toInputStream());
- assertThat(objContent.response().contentLength()).isEqualTo(testFile.length());
- byte[] expectedSum = ChecksumUtils.computeCheckSum(Files.newInputStream(testFile.toPath()));
- assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedSum);
- }
-
- @Test
- void putObject_inputStreamAsyncRequestBody_objectSentCorrectly() throws Exception {
- AsyncRequestBody body = AsyncRequestBody.fromInputStream(
- new FileInputStream(testFile),
- Long.valueOf(OBJ_SIZE),
- executorService);
- mpuS3Client.putObject(r -> r.bucket(TEST_BUCKET)
- .key(TEST_KEY)
- .contentLength(Long.valueOf(OBJ_SIZE)), body).join();
-
assertThat(CAPTURING_INTERCEPTOR.createMpuChecksumAlgorithm).isEqualTo("CRC32");
assertThat(CAPTURING_INTERCEPTOR.uploadPartChecksumAlgorithm).isEqualTo("CRC32");
- ResponseInputStream objContent = s3.getObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY),
- ResponseTransformer.toInputStream());
-
assertThat(objContent.response().contentLength()).isEqualTo(testFile.length());
- byte[] expectedSum = ChecksumUtils.computeCheckSum(Files.newInputStream(testFile.toPath()));
- assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedSum);
- }
-
- @Test
- void putObject_byteAsyncRequestBody_objectSentCorrectly() throws Exception {
- byte[] bytes = RandomStringUtils.randomAscii(OBJ_SIZE).getBytes(Charset.defaultCharset());
- AsyncRequestBody body = AsyncRequestBody.fromBytes(bytes);
- mpuS3Client.putObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY), body).join();
-
- assertThat(CAPTURING_INTERCEPTOR.createMpuChecksumAlgorithm).isEqualTo("CRC32");
- assertThat(CAPTURING_INTERCEPTOR.uploadPartChecksumAlgorithm).isEqualTo("CRC32");
-
- ResponseInputStream objContent = s3.getObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY),
- ResponseTransformer.toInputStream());
-
- assertThat(objContent.response().contentLength()).isEqualTo(OBJ_SIZE);
- byte[] expectedSum = ChecksumUtils.computeCheckSum(new ByteArrayInputStream(bytes));
- assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedSum);
+ assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedChecksum);
}
- @Test
- void putObject_unknownContentLength_objectSentCorrectly() throws Exception {
- AsyncRequestBody body = FileAsyncRequestBody.builder()
- .path(testFile.toPath())
- .build();
- mpuS3Client.putObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY), new AsyncRequestBody() {
- @Override
- public Optional contentLength() {
- return Optional.empty();
- }
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- body.subscribe(s);
- }
- }).get(30, SECONDS);
+ @ParameterizedTest
+ @MethodSource("asyncRequestBodies")
+ void putObject_wrapWithBufferedSplittableAsyncRequestBody_objectSentCorrectly(String description,
+ AsyncRequestBody asyncRequestBody) throws Exception {
+ AsyncRequestBody body = BufferedSplittableAsyncRequestBody.create(asyncRequestBody);
+ mpuS3Client.putObject(r -> r.bucket(TEST_BUCKET).key(TEST_KEY), body).join();
assertThat(CAPTURING_INTERCEPTOR.createMpuChecksumAlgorithm).isEqualTo("CRC32");
assertThat(CAPTURING_INTERCEPTOR.uploadPartChecksumAlgorithm).isEqualTo("CRC32");
@@ -222,8 +196,7 @@ public void subscribe(Subscriber super ByteBuffer> s) {
ResponseTransformer.toInputStream());
assertThat(objContent.response().contentLength()).isEqualTo(testFile.length());
- byte[] expectedSum = ChecksumUtils.computeCheckSum(Files.newInputStream(testFile.toPath()));
- assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedSum);
+ assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedChecksum);
}
@Test
@@ -251,8 +224,7 @@ void putObject_withSSECAndChecksum_objectSentCorrectly() throws Exception {
ResponseTransformer.toInputStream());
assertThat(objContent.response().contentLength()).isEqualTo(testFile.length());
- byte[] expectedSum = ChecksumUtils.computeCheckSum(Files.newInputStream(testFile.toPath()));
- assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedSum);
+ assertThat(ChecksumUtils.computeCheckSum(objContent)).isEqualTo(expectedChecksum);
}
@Test
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
index 527119415ea6..d86005d85bc4 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriber.java
@@ -34,7 +34,7 @@
import org.reactivestreams.Subscription;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.listener.PublisherListener;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
@@ -48,7 +48,7 @@
import software.amazon.awssdk.utils.Pair;
@SdkInternalApi
-public class KnownContentLengthAsyncRequestBodySubscriber implements Subscriber {
+public class KnownContentLengthAsyncRequestBodySubscriber implements Subscriber {
private static final Logger log = Logger.loggerFor(KnownContentLengthAsyncRequestBodySubscriber.class);
@@ -145,7 +145,7 @@ public void onSubscribe(Subscription s) {
}
@Override
- public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
+ public void onNext(CloseableAsyncRequestBody asyncRequestBody) {
if (isPaused || isDone) {
return;
}
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
index 3808cb864de1..0fdeb1674798 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithKnownContentLengthHelper.java
@@ -186,8 +186,8 @@ private void splitAndSubscribe(MpuRequestContext mpuRequestContext, CompletableF
attachSubscriberToObservable(subscriber, mpuRequestContext.request().left());
mpuRequestContext.request().right()
- .splitClosable(b -> b.chunkSizeInBytes(mpuRequestContext.partSize())
- .bufferSizeInBytes(maxMemoryUsageInBytes))
+ .splitCloseable(b -> b.chunkSizeInBytes(mpuRequestContext.partSize())
+ .bufferSizeInBytes(maxMemoryUsageInBytes))
.subscribe(subscriber);
}
diff --git a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
index a02f643c70c8..cab480a540cb 100644
--- a/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
+++ b/services/s3/src/main/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelper.java
@@ -33,7 +33,7 @@
import org.reactivestreams.Subscription;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.core.async.listener.PublisherListener;
import software.amazon.awssdk.core.exception.SdkClientException;
@@ -82,9 +82,9 @@ public CompletableFuture uploadObject(PutObjectRequest putObj
AsyncRequestBody asyncRequestBody) {
CompletableFuture returnFuture = new CompletableFuture<>();
- SdkPublisher splitAsyncRequestBodyResponse =
- asyncRequestBody.splitClosable(b -> b.chunkSizeInBytes(partSizeInBytes)
- .bufferSizeInBytes(maxMemoryUsageInBytes));
+ SdkPublisher splitAsyncRequestBodyResponse =
+ asyncRequestBody.splitCloseable(b -> b.chunkSizeInBytes(partSizeInBytes)
+ .bufferSizeInBytes(maxMemoryUsageInBytes));
splitAsyncRequestBodyResponse.subscribe(new UnknownContentLengthAsyncRequestBodySubscriber(partSizeInBytes,
putObjectRequest,
@@ -92,7 +92,7 @@ public CompletableFuture uploadObject(PutObjectRequest putObj
return returnFuture;
}
- private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscriber {
+ private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscriber {
/**
* Indicates whether this is the first async request body or not.
*/
@@ -128,7 +128,7 @@ private class UnknownContentLengthAsyncRequestBodySubscriber implements Subscrib
private final CompletableFuture returnFuture;
private final PublisherListener progressListener;
private Subscription subscription;
- private ClosableAsyncRequestBody firstRequestBody;
+ private CloseableAsyncRequestBody firstRequestBody;
private String uploadId;
private volatile boolean isDone;
@@ -162,7 +162,7 @@ public void onSubscribe(Subscription s) {
}
@Override
- public void onNext(ClosableAsyncRequestBody asyncRequestBody) {
+ public void onNext(CloseableAsyncRequestBody asyncRequestBody) {
if (isDone) {
return;
}
@@ -232,7 +232,7 @@ private Optional validatePart(AsyncRequestBody asyncRequestB
}
private void sendUploadPartRequest(String uploadId,
- ClosableAsyncRequestBody asyncRequestBody,
+ CloseableAsyncRequestBody asyncRequestBody,
int currentPartNum) {
Long contentLengthCurrentPart = asyncRequestBody.contentLength().get();
this.contentLength.getAndAdd(contentLengthCurrentPart);
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java
index d2bd9d55dfb7..c18f088f1cd9 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/KnownContentLengthAsyncRequestBodySubscriberTest.java
@@ -38,7 +38,7 @@
import org.mockito.ArgumentCaptor;
import org.reactivestreams.Subscription;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
@@ -132,7 +132,7 @@ void validateTotalPartNum_receivedMoreParts_shouldFail() {
lastPartSubscriber.onSubscribe(subscription);
for (int i = 0; i < TOTAL_NUM_PARTS - 1; i++) {
- ClosableAsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
+ CloseableAsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
when(multipartUploadHelper.sendIndividualUploadPartRequest(eq(UPLOAD_ID), any(), any(), any(), any()))
.thenReturn(CompletableFuture.completedFuture(null));
lastPartSubscriber.onNext(regularPart);
@@ -154,7 +154,7 @@ void validateLastPartSize_withCorrectSize_shouldNotFail() {
subscriber.onSubscribe(subscription);
for (int i = 0; i < TOTAL_NUM_PARTS - 1; i++) {
- ClosableAsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
+ CloseableAsyncRequestBody regularPart = createMockAsyncRequestBody(PART_SIZE);
when(multipartUploadHelper.sendIndividualUploadPartRequest(eq(UPLOAD_ID), any(), any(), any(), any()))
.thenReturn(CompletableFuture.completedFuture(null));
subscriber.onNext(regularPart);
@@ -243,14 +243,14 @@ private KnownContentLengthAsyncRequestBodySubscriber createSubscriber(MpuRequest
return new KnownContentLengthAsyncRequestBodySubscriber(mpuRequestContext, returnFuture, multipartUploadHelper);
}
- private ClosableAsyncRequestBody createMockAsyncRequestBody(long contentLength) {
- ClosableAsyncRequestBody mockBody = mock(ClosableAsyncRequestBody.class);
+ private CloseableAsyncRequestBody createMockAsyncRequestBody(long contentLength) {
+ CloseableAsyncRequestBody mockBody = mock(CloseableAsyncRequestBody.class);
when(mockBody.contentLength()).thenReturn(Optional.of(contentLength));
return mockBody;
}
- private ClosableAsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLength() {
- ClosableAsyncRequestBody mockBody = mock(ClosableAsyncRequestBody.class);
+ private CloseableAsyncRequestBody createMockAsyncRequestBodyWithEmptyContentLength() {
+ CloseableAsyncRequestBody mockBody = mock(CloseableAsyncRequestBody.class);
when(mockBody.contentLength()).thenReturn(Optional.empty());
return mockBody;
}
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
index 63ae1d39875a..859f5aebacac 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/S3MultipartClientPutObjectWiremockTest.java
@@ -196,7 +196,7 @@ public void subscribe(Subscriber super ByteBuffer> s) {
assertThatThrownBy(() -> s3AsyncClient.putObject(b -> b.bucket(BUCKET).key(KEY), asyncRequestBody)
.join())
.hasCauseInstanceOf(NonRetryableException.class)
- .hasMessageContaining("A retry was attempted, but");
+ .hasMessageContaining("Multiple subscribers detected.");
verify(1, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(1))));
verify(1, putRequestedFor(anyUrl()).withQueryParam("partNumber", matching(String.valueOf(1))));
diff --git a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
index 82de9fa606d2..b7bd330a6e75 100644
--- a/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
+++ b/services/s3/src/test/java/software/amazon/awssdk/services/s3/internal/multipart/UploadWithUnknownContentLengthHelperTest.java
@@ -44,7 +44,7 @@
import org.reactivestreams.Subscriber;
import org.reactivestreams.Subscription;
import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.core.async.ClosableAsyncRequestBody;
+import software.amazon.awssdk.core.async.CloseableAsyncRequestBody;
import software.amazon.awssdk.core.async.BlockingInputStreamAsyncRequestBody;
import software.amazon.awssdk.core.async.SdkPublisher;
import software.amazon.awssdk.core.exception.SdkClientException;
@@ -111,14 +111,14 @@ void upload_blockingInputStream_shouldInOrder() throws FileNotFoundException {
@Test
void uploadObject_withMissingContentLength_shouldFailRequest() {
- ClosableAsyncRequestBody asyncRequestBody = createMockAsyncRequestBodyWithEmptyContentLength();
+ CloseableAsyncRequestBody asyncRequestBody = createMockAsyncRequestBodyWithEmptyContentLength();
CompletableFuture future = setupAndTriggerUploadFailure(asyncRequestBody);
verifyFailureWithMessage(future, "Content length is missing on the AsyncRequestBody for part number");
}
@Test
void uploadObject_withPartSizeExceedingLimit_shouldFailRequest() {
- ClosableAsyncRequestBody asyncRequestBody = createMockAsyncRequestBody(PART_SIZE + 1);
+ CloseableAsyncRequestBody asyncRequestBody = createMockAsyncRequestBody(PART_SIZE + 1);
CompletableFuture