* {@link #subscribe(Subscriber)} should be implemented to tie this publisher to a subscriber. Ideally each call to subscribe
- * should reproduce the content (i.e if you are reading from a file each subscribe call should produce a {@link
- * org.reactivestreams.Subscription} that reads the file fully). This allows for automatic retries to be performed in the SDK. If
- * the content is not reproducible, an exception may be thrown from any subsequent {@link #subscribe(Subscriber)} calls.
+ * should reproduce the content (i.e if you are reading from a file each subscribe call should produce a
+ * {@link org.reactivestreams.Subscription} that reads the file fully). This allows for automatic retries to be performed in the
+ * SDK. If the content is not reproducible, an exception may be thrown from any subsequent {@link #subscribe(Subscriber)} calls.
*
*
*
- * It is important to only send the number of chunks that the subscriber requests to avoid out of memory situations.
- * The subscriber does it's own buffering so it's usually not needed to buffer in the publisher. Additional permits
- * for chunks will be notified via the {@link org.reactivestreams.Subscription#request(long)} method.
+ * It is important to only send the number of chunks that the subscriber requests to avoid out of memory situations. The
+ * subscriber does it's own buffering so it's usually not needed to buffer in the publisher. Additional permits for chunks will be
+ * notified via the {@link org.reactivestreams.Subscription#request(long)} method.
*
*
* @see FileAsyncRequestBody
- * @see ByteArrayAsyncRequestBody
+ * @see ByteBuffersAsyncRequestBody
*/
@SdkPublicApi
public interface AsyncRequestBody extends SdkPublisher {
@@ -70,8 +71,8 @@ default String contentType() {
}
/**
- * Creates an {@link AsyncRequestBody} the produces data from the input ByteBuffer publisher.
- * The data is delivered when the publisher publishes the data.
+ * Creates an {@link AsyncRequestBody} the produces data from the input ByteBuffer publisher. The data is delivered when the
+ * publisher publishes the data.
*
* @param publisher Publisher of source data
* @return Implementation of {@link AsyncRequestBody} that produces data send by the publisher
@@ -124,11 +125,11 @@ static AsyncRequestBody fromFile(File file) {
* @param string The string to provide.
* @param cs The {@link Charset} to use.
* @return Implementation of {@link AsyncRequestBody} that uses the specified string.
- * @see ByteArrayAsyncRequestBody
+ * @see ByteBuffersAsyncRequestBody
*/
static AsyncRequestBody fromString(String string, Charset cs) {
- return new ByteArrayAsyncRequestBody(string.getBytes(cs),
- Mimetype.MIMETYPE_TEXT_PLAIN + "; charset=" + cs.name());
+ return ByteBuffersAsyncRequestBody.from(Mimetype.MIMETYPE_TEXT_PLAIN + "; charset=" + cs.name(),
+ string.getBytes(cs));
}
/**
@@ -143,29 +144,181 @@ static AsyncRequestBody fromString(String string) {
}
/**
- * Creates a {@link AsyncRequestBody} from a byte array. The contents of the byte array are copied so modifications to the
- * original byte array are not reflected in the {@link AsyncRequestBody}.
+ * Creates an {@link AsyncRequestBody} from a byte array. This will copy the contents of the byte array to prevent
+ * modifications to the provided byte array from being reflected in the {@link AsyncRequestBody}.
*
* @param bytes The bytes to send to the service.
* @return AsyncRequestBody instance.
*/
static AsyncRequestBody fromBytes(byte[] bytes) {
- return new ByteArrayAsyncRequestBody(bytes, Mimetype.MIMETYPE_OCTET_STREAM);
+ byte[] clonedBytes = bytes.clone();
+ return ByteBuffersAsyncRequestBody.from(clonedBytes);
}
/**
- * Creates a {@link AsyncRequestBody} from a {@link ByteBuffer}. Buffer contents are copied so any modifications
- * made to the original {@link ByteBuffer} are not reflected in the {@link AsyncRequestBody}.
+ * Creates an {@link AsyncRequestBody} from a byte array without copying the contents of the byte array. This
+ * introduces concurrency risks, allowing: (1) the caller to modify the byte array stored in this {@code AsyncRequestBody}
+ * implementation AND (2) any users of {@link #fromBytesUnsafe(byte[])} to modify the byte array passed into this
+ * {@code AsyncRequestBody} implementation.
+ *
+ * As the method name implies, this is unsafe. Use {@link #fromBytes(byte[])} unless you're sure you know the risks.
+ *
+ * @param bytes The bytes to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromBytesUnsafe(byte[] bytes) {
+ return ByteBuffersAsyncRequestBody.from(bytes);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer}. This will copy the contents of the {@link ByteBuffer} to
+ * prevent modifications to the provided {@link ByteBuffer} from being reflected in the {@link AsyncRequestBody}.
+ *
+ * NOTE: This method ignores the current read position. Use {@link #fromRemainingByteBuffer(ByteBuffer)} if you need
+ * it to copy only the remaining readable bytes.
*
* @param byteBuffer ByteBuffer to send to the service.
* @return AsyncRequestBody instance.
*/
static AsyncRequestBody fromByteBuffer(ByteBuffer byteBuffer) {
- return fromBytes(BinaryUtils.copyAllBytesFrom(byteBuffer));
+ ByteBuffer immutableCopy = BinaryUtils.immutableCopyOf(byteBuffer);
+ immutableCopy.rewind();
+ return ByteBuffersAsyncRequestBody.of((long) immutableCopy.remaining(), immutableCopy);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from the remaining readable bytes from a {@link ByteBuffer}. This will copy the
+ * remaining contents of the {@link ByteBuffer} to prevent modifications to the provided {@link ByteBuffer} from being
+ * reflected in the {@link AsyncRequestBody}.
+ *
Unlike {@link #fromByteBuffer(ByteBuffer)}, this method respects the current read position of the buffer and reads
+ * only the remaining bytes.
+ *
+ * @param byteBuffer ByteBuffer to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromRemainingByteBuffer(ByteBuffer byteBuffer) {
+ ByteBuffer immutableCopy = BinaryUtils.immutableCopyOfRemaining(byteBuffer);
+ return ByteBuffersAsyncRequestBody.of((long) immutableCopy.remaining(), immutableCopy);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer} without copying the contents of the
+ * {@link ByteBuffer}. This introduces concurrency risks, allowing the caller to modify the {@link ByteBuffer} stored in this
+ * {@code AsyncRequestBody} implementation.
+ *
+ * NOTE: This method ignores the current read position. Use {@link #fromRemainingByteBufferUnsafe(ByteBuffer)} if you
+ * need it to copy only the remaining readable bytes.
+ *
+ *
As the method name implies, this is unsafe. Use {@link #fromByteBuffer(ByteBuffer)}} unless you're sure you know the
+ * risks.
+ *
+ * @param byteBuffer ByteBuffer to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromByteBufferUnsafe(ByteBuffer byteBuffer) {
+ ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer();
+ readOnlyBuffer.rewind();
+ return ByteBuffersAsyncRequestBody.of((long) readOnlyBuffer.remaining(), readOnlyBuffer);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer} without copying the contents of the
+ * {@link ByteBuffer}. This introduces concurrency risks, allowing the caller to modify the {@link ByteBuffer} stored in this
+ * {@code AsyncRequestBody} implementation.
+ *
Unlike {@link #fromByteBufferUnsafe(ByteBuffer)}, this method respects the current read position of
+ * the buffer and reads only the remaining bytes.
+ *
+ *
As the method name implies, this is unsafe. Use {@link #fromByteBuffer(ByteBuffer)}} unless you're sure you know the
+ * risks.
+ *
+ * @param byteBuffer ByteBuffer to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromRemainingByteBufferUnsafe(ByteBuffer byteBuffer) {
+ ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer();
+ return ByteBuffersAsyncRequestBody.of((long) readOnlyBuffer.remaining(), readOnlyBuffer);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer} array. This will copy the contents of each {@link ByteBuffer}
+ * to prevent modifications to any provided {@link ByteBuffer} from being reflected in the {@link AsyncRequestBody}.
+ *
+ * NOTE: This method ignores the current read position of each {@link ByteBuffer}. Use
+ * {@link #fromRemainingByteBuffers(ByteBuffer...)} if you need it to copy only the remaining readable bytes.
+ *
+ * @param byteBuffers ByteBuffer array to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromByteBuffers(ByteBuffer... byteBuffers) {
+ ByteBuffer[] immutableCopy = Arrays.stream(byteBuffers)
+ .map(BinaryUtils::immutableCopyOf)
+ .peek(ByteBuffer::rewind)
+ .toArray(ByteBuffer[]::new);
+ return ByteBuffersAsyncRequestBody.of(immutableCopy);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer} array. This will copy the remaining contents of each
+ * {@link ByteBuffer} to prevent modifications to any provided {@link ByteBuffer} from being reflected in the
+ * {@link AsyncRequestBody}.
+ *
Unlike {@link #fromByteBufferUnsafe(ByteBuffer)},
+ * this method respects the current read position of each buffer and reads only the remaining bytes.
+ *
+ * @param byteBuffers ByteBuffer array to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromRemainingByteBuffers(ByteBuffer... byteBuffers) {
+ ByteBuffer[] immutableCopy = Arrays.stream(byteBuffers)
+ .map(BinaryUtils::immutableCopyOfRemaining)
+ .peek(ByteBuffer::rewind)
+ .toArray(ByteBuffer[]::new);
+ return ByteBuffersAsyncRequestBody.of(immutableCopy);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer} array without copying the contents of each
+ * {@link ByteBuffer}. This introduces concurrency risks, allowing the caller to modify any {@link ByteBuffer} stored in this
+ * {@code AsyncRequestBody} implementation.
+ *
+ * NOTE: This method ignores the current read position of each {@link ByteBuffer}. Use
+ * {@link #fromRemainingByteBuffers(ByteBuffer...)} if you need it to copy only the remaining readable bytes.
+ *
+ *
As the method name implies, this is unsafe. Use {@link #fromByteBuffers(ByteBuffer...)} unless you're sure you know the
+ * risks.
+ *
+ * @param byteBuffers ByteBuffer array to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromByteBuffersUnsafe(ByteBuffer... byteBuffers) {
+ ByteBuffer[] readOnlyBuffers = Arrays.stream(byteBuffers)
+ .map(ByteBuffer::asReadOnlyBuffer)
+ .peek(ByteBuffer::rewind)
+ .toArray(ByteBuffer[]::new);
+ return ByteBuffersAsyncRequestBody.of(readOnlyBuffers);
+ }
+
+ /**
+ * Creates an {@link AsyncRequestBody} from a {@link ByteBuffer} array without copying the contents of each
+ * {@link ByteBuffer}. This introduces concurrency risks, allowing the caller to modify any {@link ByteBuffer} stored in this
+ * {@code AsyncRequestBody} implementation.
+ *
Unlike {@link #fromByteBuffersUnsafe(ByteBuffer...)},
+ * this method respects the current read position of each buffer and reads only the remaining bytes.
+ *
+ *
As the method name implies, this is unsafe. Use {@link #fromByteBuffers(ByteBuffer...)} unless you're sure you know the
+ * risks.
+ *
+ * @param byteBuffers ByteBuffer array to send to the service.
+ * @return AsyncRequestBody instance.
+ */
+ static AsyncRequestBody fromRemainingByteBuffersUnsafe(ByteBuffer... byteBuffers) {
+ ByteBuffer[] readOnlyBuffers = Arrays.stream(byteBuffers)
+ .map(ByteBuffer::asReadOnlyBuffer)
+ .toArray(ByteBuffer[]::new);
+ return ByteBuffersAsyncRequestBody.of(readOnlyBuffers);
}
/**
- * Creates a {@link AsyncRequestBody} from a {@link InputStream}.
+ * Creates an {@link AsyncRequestBody} from an {@link InputStream}.
*
*
An {@link ExecutorService} is required in order to perform the blocking data reads, to prevent blocking the
* non-blocking event loop threads owned by the SDK.
@@ -239,7 +392,7 @@ static BlockingOutputStreamAsyncRequestBody forBlockingOutputStream(Long content
}
/**
- * Creates a {@link AsyncRequestBody} with no content.
+ * Creates an {@link AsyncRequestBody} with no content.
*
* @return AsyncRequestBody instance.
*/
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteArrayAsyncRequestBody.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteArrayAsyncRequestBody.java
deleted file mode 100644
index 29205479b798..000000000000
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteArrayAsyncRequestBody.java
+++ /dev/null
@@ -1,98 +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 org.reactivestreams.Subscriber;
-import org.reactivestreams.Subscription;
-import software.amazon.awssdk.annotations.SdkInternalApi;
-import software.amazon.awssdk.core.async.AsyncRequestBody;
-import software.amazon.awssdk.utils.Logger;
-
-/**
- * An implementation of {@link AsyncRequestBody} for providing data from memory. This is created using static
- * methods on {@link AsyncRequestBody}
- *
- * @see AsyncRequestBody#fromBytes(byte[])
- * @see AsyncRequestBody#fromByteBuffer(ByteBuffer)
- * @see AsyncRequestBody#fromString(String)
- */
-@SdkInternalApi
-public final class ByteArrayAsyncRequestBody implements AsyncRequestBody {
- private static final Logger log = Logger.loggerFor(ByteArrayAsyncRequestBody.class);
-
- private final byte[] bytes;
-
- private final String mimetype;
-
- public ByteArrayAsyncRequestBody(byte[] bytes, String mimetype) {
- this.bytes = bytes.clone();
- this.mimetype = mimetype;
- }
-
- @Override
- public Optional contentLength() {
- return Optional.of((long) bytes.length);
- }
-
- @Override
- public String contentType() {
- return mimetype;
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- // As per rule 1.9 we must throw NullPointerException if the subscriber parameter is null
- if (s == null) {
- throw new NullPointerException("Subscription MUST NOT be null.");
- }
-
- // As per 2.13, this method must return normally (i.e. not throw).
- try {
- s.onSubscribe(
- new Subscription() {
- private boolean done = false;
-
- @Override
- public void request(long n) {
- if (done) {
- return;
- }
- if (n > 0) {
- done = true;
- s.onNext(ByteBuffer.wrap(bytes));
- s.onComplete();
- } else {
- s.onError(new IllegalArgumentException("§3.9: non-positive requests are not allowed!"));
- }
- }
-
- @Override
- public void cancel() {
- synchronized (this) {
- if (!done) {
- done = true;
- }
- }
- }
- }
- );
- } catch (Throwable ex) {
- log.error(() -> s + " violated the Reactive Streams rule 2.13 by throwing an exception from onSubscribe.", ex);
- }
- }
-}
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
new file mode 100644
index 000000000000..e7e9d00dd0e5
--- /dev/null
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBody.java
@@ -0,0 +1,157 @@
+/*
+ * 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.Arrays;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.core.internal.util.Mimetype;
+import software.amazon.awssdk.utils.BinaryUtils;
+import software.amazon.awssdk.utils.Logger;
+
+/**
+ * An implementation of {@link AsyncRequestBody} for providing data from the supplied {@link ByteBuffer} array. This is created
+ * using static methods on {@link AsyncRequestBody}
+ *
+ * @see AsyncRequestBody#fromBytes(byte[])
+ * @see AsyncRequestBody#fromBytesUnsafe(byte[])
+ * @see AsyncRequestBody#fromByteBuffer(ByteBuffer)
+ * @see AsyncRequestBody#fromByteBufferUnsafe(ByteBuffer)
+ * @see AsyncRequestBody#fromByteBuffers(ByteBuffer...)
+ * @see AsyncRequestBody#fromByteBuffersUnsafe(ByteBuffer...)
+ * @see AsyncRequestBody#fromString(String)
+ */
+@SdkInternalApi
+public final class ByteBuffersAsyncRequestBody implements AsyncRequestBody {
+ private static final Logger log = Logger.loggerFor(ByteBuffersAsyncRequestBody.class);
+
+ private final String mimetype;
+ private final Long length;
+ private final ByteBuffer[] buffers;
+
+ private ByteBuffersAsyncRequestBody(String mimetype, Long length, ByteBuffer... buffers) {
+ this.mimetype = mimetype;
+ this.length = length;
+ this.buffers = buffers;
+ }
+
+ @Override
+ public Optional contentLength() {
+ return Optional.ofNullable(length);
+ }
+
+ @Override
+ public String contentType() {
+ return mimetype;
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ // As per rule 1.9 we must throw NullPointerException if the subscriber parameter is null
+ if (s == null) {
+ throw new NullPointerException("Subscription MUST NOT be null.");
+ }
+
+ // As per 2.13, this method must return normally (i.e. not throw).
+ try {
+ s.onSubscribe(
+ new Subscription() {
+ private final AtomicInteger index = new AtomicInteger(0);
+ private final AtomicBoolean completed = new AtomicBoolean(false);
+
+ @Override
+ public void request(long n) {
+ if (completed.get()) {
+ return;
+ }
+
+ if (n > 0) {
+ int i = index.getAndIncrement();
+
+ if (i >= buffers.length) {
+ return;
+ }
+
+ long remaining = n;
+
+ do {
+ ByteBuffer buffer = buffers[i];
+
+ // Pending discussions on https://github.com/aws/aws-sdk-java-v2/issues/3928
+ if (buffer.isDirect()) {
+ buffer = BinaryUtils.toNonDirectBuffer(buffer);
+ }
+
+ s.onNext(buffer.asReadOnlyBuffer());
+ remaining--;
+ } while (remaining > 0 && (i = index.getAndIncrement()) < buffers.length);
+
+ if (i >= buffers.length - 1 && completed.compareAndSet(false, true)) {
+ s.onComplete();
+ }
+ } else {
+ s.onError(new IllegalArgumentException("§3.9: non-positive requests are not allowed!"));
+ }
+ }
+
+ @Override
+ public void cancel() {
+ completed.set(true);
+ }
+ }
+ );
+ } catch (Throwable ex) {
+ log.error(() -> s + " violated the Reactive Streams rule 2.13 by throwing an exception from onSubscribe.", ex);
+ }
+ }
+
+ public static ByteBuffersAsyncRequestBody of(ByteBuffer... buffers) {
+ long length = Arrays.stream(buffers)
+ .mapToLong(ByteBuffer::remaining)
+ .sum();
+ return new ByteBuffersAsyncRequestBody(Mimetype.MIMETYPE_OCTET_STREAM, length, buffers);
+ }
+
+ public static ByteBuffersAsyncRequestBody of(Long length, ByteBuffer... buffers) {
+ return new ByteBuffersAsyncRequestBody(Mimetype.MIMETYPE_OCTET_STREAM, length, buffers);
+ }
+
+ public static ByteBuffersAsyncRequestBody of(String mimetype, ByteBuffer... buffers) {
+ long length = Arrays.stream(buffers)
+ .mapToLong(ByteBuffer::remaining)
+ .sum();
+ return new ByteBuffersAsyncRequestBody(mimetype, length, buffers);
+ }
+
+ public static ByteBuffersAsyncRequestBody of(String mimetype, Long length, ByteBuffer... buffers) {
+ return new ByteBuffersAsyncRequestBody(mimetype, length, buffers);
+ }
+
+ public static ByteBuffersAsyncRequestBody from(byte[] bytes) {
+ return new ByteBuffersAsyncRequestBody(Mimetype.MIMETYPE_OCTET_STREAM, (long) bytes.length,
+ ByteBuffer.wrap(bytes));
+ }
+
+ public static ByteBuffersAsyncRequestBody from(String mimetype, byte[] bytes) {
+ return new ByteBuffersAsyncRequestBody(mimetype, (long) bytes.length, ByteBuffer.wrap(bytes));
+ }
+}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ChunkBuffer.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ChunkBuffer.java
index 8fd7f0260b76..93d6d09578a6 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ChunkBuffer.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/async/ChunkBuffer.java
@@ -22,6 +22,7 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.utils.BinaryUtils;
import software.amazon.awssdk.utils.Validate;
import software.amazon.awssdk.utils.builder.SdkBuilder;
@@ -58,10 +59,11 @@ public synchronized Iterable bufferAndCreateChunks(ByteBuffer buffer
int availableToRead = bufferSize - bufferedBytes;
int bytesToMove = Math.min(availableToRead, currentBytesRead - startPosition);
+ byte[] bytes = BinaryUtils.copyAllBytesFrom(buffer);
if (bufferedBytes == 0) {
- currentBuffer.put(buffer.array(), startPosition, bytesToMove);
+ currentBuffer.put(bytes, startPosition, bytesToMove);
} else {
- currentBuffer.put(buffer.array(), 0, bytesToMove);
+ currentBuffer.put(bytes, 0, bytesToMove);
}
startPosition = startPosition + bytesToMove;
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/http/pipeline/stages/AfterTransmissionExecutionInterceptorsStage.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/http/pipeline/stages/AfterTransmissionExecutionInterceptorsStage.java
index 7521219a5030..a7cada02b06c 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/http/pipeline/stages/AfterTransmissionExecutionInterceptorsStage.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/http/pipeline/stages/AfterTransmissionExecutionInterceptorsStage.java
@@ -31,7 +31,7 @@ public class AfterTransmissionExecutionInterceptorsStage
@Override
public Pair execute(Pair input,
RequestExecutionContext context) throws Exception {
- InterruptMonitor.checkInterrupted();
+ InterruptMonitor.checkInterrupted(input.right());
// Update interceptor context
InterceptorContext interceptorContext =
context.executionContext().interceptorContext().copy(b -> b.httpResponse(input.right())
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumInHeaderInterceptor.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumInHeaderInterceptor.java
index 0ddf70959cae..f3c92a254bec 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumInHeaderInterceptor.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumInHeaderInterceptor.java
@@ -15,7 +15,6 @@
package software.amazon.awssdk.core.internal.interceptor;
-import static software.amazon.awssdk.core.HttpChecksumConstant.HTTP_CHECKSUM_VALUE;
import static software.amazon.awssdk.core.HttpChecksumConstant.SIGNING_METHOD;
import static software.amazon.awssdk.core.internal.util.HttpChecksumResolver.getResolvedChecksumSpecs;
@@ -23,7 +22,6 @@
import java.io.UncheckedIOException;
import java.util.Optional;
import software.amazon.awssdk.annotations.SdkInternalApi;
-import software.amazon.awssdk.core.checksums.Algorithm;
import software.amazon.awssdk.core.checksums.ChecksumSpecs;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
@@ -47,49 +45,27 @@
@SdkInternalApi
public class HttpChecksumInHeaderInterceptor implements ExecutionInterceptor {
- @Override
- public void afterMarshalling(Context.AfterMarshalling context, ExecutionAttributes executionAttributes) {
- ChecksumSpecs headerChecksumSpecs = HttpChecksumUtils.checksumSpecWithRequestAlgorithm(executionAttributes).orElse(null);
-
- if (shouldSkipHttpChecksumInHeader(context, executionAttributes, headerChecksumSpecs)) {
- return;
- }
- Optional syncContent = context.requestBody();
- syncContent.ifPresent(
- requestBody -> saveContentChecksum(requestBody, executionAttributes, headerChecksumSpecs.algorithm()));
- }
-
- @Override
- public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, ExecutionAttributes executionAttributes) {
- ChecksumSpecs checksumSpecs = getResolvedChecksumSpecs(executionAttributes);
-
- if (shouldSkipHttpChecksumInHeader(context, executionAttributes, checksumSpecs)) {
- return context.httpRequest();
- }
-
- String httpChecksumValue = executionAttributes.getAttribute(HTTP_CHECKSUM_VALUE);
- if (httpChecksumValue != null) {
- return context.httpRequest().copy(r -> r.putHeader(checksumSpecs.headerName(), httpChecksumValue));
- }
- return context.httpRequest();
-
- }
-
/**
- * Calculates the checksumSpecs of the provided request (and base64 encodes it), storing the result in
- * executionAttribute "HttpChecksumValue".
+ * Calculates the checksum of the provided request (and base64 encodes it), and adds the header to the request.
*
* Note: This assumes that the content stream provider can create multiple new streams. If it only supports one (e.g. with
* an input stream that doesn't support mark/reset), we could consider buffering the content in memory here and updating the
* request body to use that buffered content. We obviously don't want to do that for giant streams, so we haven't opted to do
* that yet.
*/
- private static void saveContentChecksum(RequestBody requestBody, ExecutionAttributes executionAttributes,
- Algorithm algorithm) {
+ @Override
+ public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, ExecutionAttributes executionAttributes) {
+ ChecksumSpecs checksumSpecs = getResolvedChecksumSpecs(executionAttributes);
+ Optional syncContent = context.requestBody();
+
+ if (shouldSkipHttpChecksumInHeader(context, executionAttributes, checksumSpecs) || !syncContent.isPresent()) {
+ return context.httpRequest();
+ }
+
try {
String payloadChecksum = BinaryUtils.toBase64(HttpChecksumUtils.computeChecksum(
- requestBody.contentStreamProvider().newStream(), algorithm));
- executionAttributes.putAttribute(HTTP_CHECKSUM_VALUE, payloadChecksum);
+ syncContent.get().contentStreamProvider().newStream(), checksumSpecs.algorithm()));
+ return context.httpRequest().copy(r -> r.putHeader(checksumSpecs.headerName(), payloadChecksum));
} catch (IOException e) {
throw new UncheckedIOException(e);
}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumRequiredInterceptor.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumRequiredInterceptor.java
index c98cde397f0c..9729cd2076d7 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumRequiredInterceptor.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/interceptor/HttpChecksumRequiredInterceptor.java
@@ -21,7 +21,6 @@
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.core.async.AsyncRequestBody;
import software.amazon.awssdk.core.interceptor.Context;
-import software.amazon.awssdk.core.interceptor.ExecutionAttribute;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
import software.amazon.awssdk.core.interceptor.SdkInternalExecutionAttribute;
@@ -41,10 +40,17 @@
*/
@SdkInternalApi
public class HttpChecksumRequiredInterceptor implements ExecutionInterceptor {
- private static final ExecutionAttribute CONTENT_MD5_VALUE = new ExecutionAttribute<>("ContentMd5");
+ /**
+ * Calculates the MD5 checksum of the provided request (and base64 encodes it), and adds the header to the request.
+ *
+ * Note: This assumes that the content stream provider can create multiple new streams. If it only supports one (e.g. with
+ * an input stream that doesn't support mark/reset), we could consider buffering the content in memory here and updating the
+ * request body to use that buffered content. We obviously don't want to do that for giant streams, so we haven't opted to do
+ * that yet.
+ */
@Override
- public void afterMarshalling(Context.AfterMarshalling context, ExecutionAttributes executionAttributes) {
+ public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, ExecutionAttributes executionAttributes) {
boolean isHttpChecksumRequired = isHttpChecksumRequired(executionAttributes);
boolean requestAlreadyHasMd5 = context.httpRequest().firstMatchingHeader(Header.CONTENT_MD5).isPresent();
@@ -52,7 +58,7 @@ public void afterMarshalling(Context.AfterMarshalling context, ExecutionAttribut
Optional asyncContent = context.asyncRequestBody();
if (!isHttpChecksumRequired || requestAlreadyHasMd5) {
- return;
+ return context.httpRequest();
}
if (asyncContent.isPresent()) {
@@ -60,14 +66,13 @@ public void afterMarshalling(Context.AfterMarshalling context, ExecutionAttribut
+ "for non-blocking content.");
}
- syncContent.ifPresent(requestBody -> saveContentMd5(requestBody, executionAttributes));
- }
-
- @Override
- public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, ExecutionAttributes executionAttributes) {
- String contentMd5 = executionAttributes.getAttribute(CONTENT_MD5_VALUE);
- if (contentMd5 != null) {
- return context.httpRequest().copy(r -> r.putHeader(Header.CONTENT_MD5, contentMd5));
+ if (syncContent.isPresent()) {
+ try {
+ String payloadMd5 = Md5Utils.md5AsBase64(syncContent.get().contentStreamProvider().newStream());
+ return context.httpRequest().copy(r -> r.putHeader(Header.CONTENT_MD5, payloadMd5));
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
}
return context.httpRequest();
}
@@ -76,22 +81,4 @@ private boolean isHttpChecksumRequired(ExecutionAttributes executionAttributes)
return executionAttributes.getAttribute(SdkInternalExecutionAttribute.HTTP_CHECKSUM_REQUIRED) != null
|| HttpChecksumUtils.isMd5ChecksumRequired(executionAttributes);
}
-
- /**
- * Calculates the MD5 checksum of the provided request (and base64 encodes it), storing the result in
- * {@link #CONTENT_MD5_VALUE}.
- *
- * Note: This assumes that the content stream provider can create multiple new streams. If it only supports one (e.g. with
- * an input stream that doesn't support mark/reset), we could consider buffering the content in memory here and updating the
- * request body to use that buffered content. We obviously don't want to do that for giant streams, so we haven't opted to do
- * that yet.
- */
- private void saveContentMd5(RequestBody requestBody, ExecutionAttributes executionAttributes) {
- try {
- String payloadMd5 = Md5Utils.md5AsBase64(requestBody.contentStreamProvider().newStream());
- executionAttributes.putAttribute(CONTENT_MD5_VALUE, payloadMd5);
- } catch (IOException e) {
- throw new UncheckedIOException(e);
- }
- }
}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/util/ClassLoaderHelper.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/util/ClassLoaderHelper.java
index 3b5b50f7e9a0..2894b2bd8dc4 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/util/ClassLoaderHelper.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/util/ClassLoaderHelper.java
@@ -69,8 +69,7 @@ private static Class> loadClassViaContext(String fqcn) {
* @throws ClassNotFoundException
* if failed to load the class
*/
- public static Class> loadClass(String fqcn, Class>... classes)
- throws ClassNotFoundException {
+ public static Class> loadClass(String fqcn, Class>... classes) throws ClassNotFoundException {
return loadClass(fqcn, true, classes);
}
diff --git a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutor.java b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutor.java
index 8610c32e49f3..5377e0f04e59 100644
--- a/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutor.java
+++ b/core/sdk-core/src/main/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutor.java
@@ -16,14 +16,12 @@
package software.amazon.awssdk.core.internal.waiters;
import java.util.List;
-import java.util.Optional;
import java.util.function.Supplier;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.annotations.ThreadSafe;
import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.core.waiters.WaiterAcceptor;
import software.amazon.awssdk.core.waiters.WaiterResponse;
-import software.amazon.awssdk.core.waiters.WaiterState;
import software.amazon.awssdk.utils.Either;
import software.amazon.awssdk.utils.Validate;
@@ -45,45 +43,42 @@ public WaiterExecutor(WaiterConfiguration configuration,
}
WaiterResponse execute(Supplier pollingFunction) {
- return doExecute(pollingFunction, 0, System.currentTimeMillis());
- }
-
- WaiterResponse doExecute(Supplier pollingFunction, int attemptNumber, long startTime) {
- attemptNumber++;
- T response;
- try {
- response = pollingFunction.get();
- } catch (Exception exception) {
- return evaluate(pollingFunction, Either.right(exception), attemptNumber, startTime);
- }
-
- return evaluate(pollingFunction, Either.left(response), attemptNumber, startTime);
- }
+ int attemptNumber = 0;
+ long startTime = System.currentTimeMillis();
- private WaiterResponse evaluate(Supplier pollingFunction,
- Either responseOrException,
- int attemptNumber,
- long startTime) {
- Optional> waiterAcceptor = executorHelper.firstWaiterAcceptorIfMatched(responseOrException);
+ while (true) {
+ attemptNumber++;
- if (waiterAcceptor.isPresent()) {
- WaiterState state = waiterAcceptor.get().waiterState();
- switch (state) {
+ Either polledResponse = pollResponse(pollingFunction);
+ WaiterAcceptor super T> waiterAcceptor = firstWaiterAcceptor(polledResponse);
+ switch (waiterAcceptor.waiterState()) {
case SUCCESS:
- return executorHelper.createWaiterResponse(responseOrException, attemptNumber);
+ return executorHelper.createWaiterResponse(polledResponse, attemptNumber);
case RETRY:
- return maybeRetry(pollingFunction, attemptNumber, startTime);
+ waitToRetry(attemptNumber, startTime);
+ break;
case FAILURE:
- throw executorHelper.waiterFailureException(waiterAcceptor.get());
+ throw executorHelper.waiterFailureException(waiterAcceptor);
default:
throw new UnsupportedOperationException();
}
}
+ }
+
+ private Either pollResponse(Supplier pollingFunction) {
+ try {
+ return Either.left(pollingFunction.get());
+ } catch (Exception exception) {
+ return Either.right(exception);
+ }
+ }
- throw executorHelper.noneMatchException(responseOrException);
+ private WaiterAcceptor super T> firstWaiterAcceptor(Either responseOrException) {
+ return executorHelper.firstWaiterAcceptorIfMatched(responseOrException)
+ .orElseThrow(() -> executorHelper.noneMatchException(responseOrException));
}
- private WaiterResponse maybeRetry(Supplier pollingFunction, int attemptNumber, long startTime) {
+ private void waitToRetry(int attemptNumber, long startTime) {
Either nextDelayOrUnretryableException =
executorHelper.nextDelayOrUnretryableException(attemptNumber, startTime);
@@ -97,6 +92,5 @@ private WaiterResponse maybeRetry(Supplier pollingFunction, int attemptNum
Thread.currentThread().interrupt();
throw SdkClientException.create("The thread got interrupted", e);
}
- return doExecute(pollingFunction, attemptNumber, startTime);
}
}
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 e0252c9ba6d2..aab643cbb6a6 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
@@ -15,44 +15,39 @@
package software.amazon.awssdk.core.async;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
import com.google.common.jimfs.Configuration;
import com.google.common.jimfs.Jimfs;
import io.reactivex.Flowable;
-import java.io.File;
-import java.io.FileWriter;
import java.io.IOException;
-import java.io.InputStream;
import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.nio.file.FileSystem;
import java.nio.file.Files;
import java.nio.file.Path;
-import java.time.Instant;
-import java.util.Collections;
import java.util.List;
-import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
import java.util.stream.Collectors;
import org.assertj.core.util.Lists;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
import software.amazon.awssdk.core.internal.util.Mimetype;
import software.amazon.awssdk.http.async.SimpleSubscriber;
import software.amazon.awssdk.utils.BinaryUtils;
-import software.amazon.awssdk.utils.StringInputStream;
-@RunWith(Parameterized.class)
public class AsyncRequestBodyTest {
- private final static String testString = "Hello!";
- private final static Path path;
+
+ private static final String testString = "Hello!";
+ private static final Path path;
static {
FileSystem fs = Jimfs.newFileSystem(Configuration.unix());
@@ -64,27 +59,16 @@ public class AsyncRequestBodyTest {
}
}
- @Parameterized.Parameters
- public static AsyncRequestBody[] data() {
- return new AsyncRequestBody[]{
- AsyncRequestBody.fromString(testString),
- AsyncRequestBody.fromFile(path)
- };
+ @ParameterizedTest
+ @MethodSource("contentIntegrityChecks")
+ void hasCorrectLength(AsyncRequestBody asyncRequestBody) {
+ assertEquals(testString.length(), asyncRequestBody.contentLength().get());
}
- private AsyncRequestBody provider;
-
- public AsyncRequestBodyTest(AsyncRequestBody provider) {
- this.provider = provider;
- }
- @Test
- public void hasCorrectLength() {
- assertThat(provider.contentLength().get()).isEqualTo(testString.length());
- }
-
- @Test
- public void hasCorrectContent() throws InterruptedException {
+ @ParameterizedTest
+ @MethodSource("contentIntegrityChecks")
+ void hasCorrectContent(AsyncRequestBody asyncRequestBody) throws InterruptedException {
StringBuilder sb = new StringBuilder();
CountDownLatch done = new CountDownLatch(1);
@@ -106,75 +90,268 @@ public void onComplete() {
}
};
- provider.subscribe(subscriber);
+ asyncRequestBody.subscribe(subscriber);
done.await();
- assertThat(sb.toString()).isEqualTo(testString);
+ assertEquals(testString, sb.toString());
+ }
+
+ private static AsyncRequestBody[] contentIntegrityChecks() {
+ return new AsyncRequestBody[] {
+ AsyncRequestBody.fromString(testString),
+ AsyncRequestBody.fromFile(path)
+ };
}
@Test
- public void stringConstructorHasCorrectContentType() {
- AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world");
- assertThat(requestBody.contentType()).isEqualTo("text/plain; charset=UTF-8");
+ void fromBytesCopiesTheProvidedByteArray() {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+ byte[] bytesClone = bytes.clone();
+
+ AsyncRequestBody asyncRequestBody = AsyncRequestBody.fromBytes(bytes);
+
+ for (int i = 0; i < bytes.length; i++) {
+ bytes[i] += 1;
+ }
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ byte[] publishedByteArray = BinaryUtils.copyAllBytesFrom(publishedBuffer.get());
+ assertArrayEquals(bytesClone, publishedByteArray);
}
@Test
- public void stringWithEncoding1ConstructorHasCorrectContentType() {
- AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world", StandardCharsets.ISO_8859_1);
- assertThat(requestBody.contentType()).isEqualTo("text/plain; charset=ISO-8859-1");
+ void fromBytesUnsafeDoesNotCopyTheProvidedByteArray() {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+
+ AsyncRequestBody asyncRequestBody = AsyncRequestBody.fromBytesUnsafe(bytes);
+
+ for (int i = 0; i < bytes.length; i++) {
+ bytes[i] += 1;
+ }
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ byte[] publishedByteArray = BinaryUtils.copyAllBytesFrom(publishedBuffer.get());
+ assertArrayEquals(bytes, publishedByteArray);
+ }
+
+ @ParameterizedTest
+ @MethodSource("safeByteBufferBodyBuilders")
+ void safeByteBufferBuildersCopyTheProvidedBuffer(Function bodyBuilder) {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+ byte[] bytesClone = bytes.clone();
+
+ AsyncRequestBody asyncRequestBody = bodyBuilder.apply(ByteBuffer.wrap(bytes));
+
+ for (int i = 0; i < bytes.length; i++) {
+ bytes[i] += 1;
+ }
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ byte[] publishedByteArray = BinaryUtils.copyAllBytesFrom(publishedBuffer.get());
+ assertArrayEquals(bytesClone, publishedByteArray);
+ }
+
+ private static Function[] safeByteBufferBodyBuilders() {
+ Function fromByteBuffer = AsyncRequestBody::fromByteBuffer;
+ Function fromRemainingByteBuffer = AsyncRequestBody::fromRemainingByteBuffer;
+ Function fromByteBuffers = AsyncRequestBody::fromByteBuffers;
+ Function fromRemainingByteBuffers = AsyncRequestBody::fromRemainingByteBuffers;
+ return new Function[] {fromByteBuffer, fromRemainingByteBuffer, fromByteBuffers, fromRemainingByteBuffers};
+ }
+
+ @ParameterizedTest
+ @MethodSource("unsafeByteBufferBodyBuilders")
+ void unsafeByteBufferBuildersDoNotCopyTheProvidedBuffer(Function bodyBuilder) {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+
+ AsyncRequestBody asyncRequestBody = bodyBuilder.apply(ByteBuffer.wrap(bytes));
+
+ for (int i = 0; i < bytes.length; i++) {
+ bytes[i] += 1;
+ }
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ byte[] publishedByteArray = BinaryUtils.copyAllBytesFrom(publishedBuffer.get());
+ assertArrayEquals(bytes, publishedByteArray);
+ }
+
+ private static Function[] unsafeByteBufferBodyBuilders() {
+ Function fromByteBuffer = AsyncRequestBody::fromByteBufferUnsafe;
+ Function fromRemainingByteBuffer = AsyncRequestBody::fromRemainingByteBufferUnsafe;
+ Function fromByteBuffers = AsyncRequestBody::fromByteBuffersUnsafe;
+ Function fromRemainingByteBuffers = AsyncRequestBody::fromRemainingByteBuffersUnsafe;
+ return new Function[] {fromByteBuffer, fromRemainingByteBuffer, fromByteBuffers, fromRemainingByteBuffers};
+ }
+
+ @ParameterizedTest
+ @MethodSource("nonRewindingByteBufferBodyBuilders")
+ void nonRewindingByteBufferBuildersReadFromTheInputBufferPosition(
+ Function bodyBuilder) {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+ ByteBuffer bb = ByteBuffer.wrap(bytes);
+ int expectedPosition = bytes.length / 2;
+ bb.position(expectedPosition);
+
+ AsyncRequestBody asyncRequestBody = bodyBuilder.apply(bb);
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ int remaining = bb.remaining();
+ assertEquals(remaining, publishedBuffer.get().remaining());
+ for (int i = 0; i < remaining; i++) {
+ assertEquals(bb.get(), publishedBuffer.get().get());
+ }
+ }
+
+ private static Function[] nonRewindingByteBufferBodyBuilders() {
+ Function fromRemainingByteBuffer = AsyncRequestBody::fromRemainingByteBuffer;
+ Function fromRemainingByteBufferUnsafe = AsyncRequestBody::fromRemainingByteBufferUnsafe;
+ Function fromRemainingByteBuffers = AsyncRequestBody::fromRemainingByteBuffers;
+ Function fromRemainingByteBuffersUnsafe = AsyncRequestBody::fromRemainingByteBuffersUnsafe;
+ return new Function[] {fromRemainingByteBuffer, fromRemainingByteBufferUnsafe, fromRemainingByteBuffers,
+ fromRemainingByteBuffersUnsafe};
+ }
+
+ @ParameterizedTest
+ @MethodSource("safeNonRewindingByteBufferBodyBuilders")
+ void safeNonRewindingByteBufferBuildersCopyFromTheInputBufferPosition(
+ Function bodyBuilder) {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+ ByteBuffer bb = ByteBuffer.wrap(bytes);
+ int expectedPosition = bytes.length / 2;
+ bb.position(expectedPosition);
+
+ AsyncRequestBody asyncRequestBody = bodyBuilder.apply(bb);
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ int remaining = bb.remaining();
+ assertEquals(remaining, publishedBuffer.get().capacity());
+ for (int i = 0; i < remaining; i++) {
+ assertEquals(bb.get(), publishedBuffer.get().get());
+ }
+ }
+
+ private static Function[] safeNonRewindingByteBufferBodyBuilders() {
+ Function fromRemainingByteBuffer = AsyncRequestBody::fromRemainingByteBuffer;
+ Function fromRemainingByteBuffers = AsyncRequestBody::fromRemainingByteBuffers;
+ return new Function[] {fromRemainingByteBuffer, fromRemainingByteBuffers};
+ }
+
+ @ParameterizedTest
+ @MethodSource("rewindingByteBufferBodyBuilders")
+ void rewindingByteBufferBuildersDoNotRewindTheInputBuffer(Function bodyBuilder) {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+ ByteBuffer bb = ByteBuffer.wrap(bytes);
+ int expectedPosition = bytes.length / 2;
+ bb.position(expectedPosition);
+
+ AsyncRequestBody asyncRequestBody = bodyBuilder.apply(bb);
+
+ Subscriber subscriber = new SimpleSubscriber(buffer -> {
+ });
+
+ asyncRequestBody.subscribe(subscriber);
+
+ assertEquals(expectedPosition, bb.position());
+ }
+
+ @ParameterizedTest
+ @MethodSource("rewindingByteBufferBodyBuilders")
+ void rewindingByteBufferBuildersReadTheInputBufferFromTheBeginning(
+ Function bodyBuilder) {
+ byte[] bytes = testString.getBytes(StandardCharsets.UTF_8);
+ ByteBuffer bb = ByteBuffer.wrap(bytes);
+ bb.position(bytes.length / 2);
+
+ AsyncRequestBody asyncRequestBody = bodyBuilder.apply(bb);
+
+ AtomicReference publishedBuffer = new AtomicReference<>();
+ Subscriber subscriber = new SimpleSubscriber(publishedBuffer::set);
+
+ asyncRequestBody.subscribe(subscriber);
+
+ assertEquals(0, publishedBuffer.get().position());
+ publishedBuffer.get().rewind();
+ bb.rewind();
+ assertEquals(bb, publishedBuffer.get());
+ }
+
+ private static Function[] rewindingByteBufferBodyBuilders() {
+ Function fromByteBuffer = AsyncRequestBody::fromByteBuffer;
+ Function fromByteBufferUnsafe = AsyncRequestBody::fromByteBufferUnsafe;
+ Function fromByteBuffers = AsyncRequestBody::fromByteBuffers;
+ Function fromByteBuffersUnsafe = AsyncRequestBody::fromByteBuffersUnsafe;
+ return new Function[] {fromByteBuffer, fromByteBufferUnsafe, fromByteBuffers, fromByteBuffersUnsafe};
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {"US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16"})
+ void charsetsAreConvertedToTheCorrectContentType(Charset charset) {
+ AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world", charset);
+ assertEquals("text/plain; charset=" + charset.name(), requestBody.contentType());
}
@Test
- public void stringWithEncoding2ConstructorHasCorrectContentType() {
- AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world", StandardCharsets.UTF_16BE);
- assertThat(requestBody.contentType()).isEqualTo("text/plain; charset=UTF-16BE");
+ void stringConstructorHasCorrectDefaultContentType() {
+ AsyncRequestBody requestBody = AsyncRequestBody.fromString("hello world");
+ assertEquals("text/plain; charset=UTF-8", requestBody.contentType());
}
@Test
- public void fileConstructorHasCorrectContentType() {
+ void fileConstructorHasCorrectContentType() {
AsyncRequestBody requestBody = AsyncRequestBody.fromFile(path);
- assertThat(requestBody.contentType()).isEqualTo(Mimetype.MIMETYPE_OCTET_STREAM);
+ assertEquals(Mimetype.MIMETYPE_OCTET_STREAM, requestBody.contentType());
}
@Test
- public void bytesArrayConstructorHasCorrectContentType() {
+ void bytesArrayConstructorHasCorrectContentType() {
AsyncRequestBody requestBody = AsyncRequestBody.fromBytes("hello world".getBytes());
- assertThat(requestBody.contentType()).isEqualTo(Mimetype.MIMETYPE_OCTET_STREAM);
+ assertEquals(Mimetype.MIMETYPE_OCTET_STREAM, requestBody.contentType());
}
@Test
- public void bytesBufferConstructorHasCorrectContentType() {
+ void bytesBufferConstructorHasCorrectContentType() {
ByteBuffer byteBuffer = ByteBuffer.wrap("hello world".getBytes());
AsyncRequestBody requestBody = AsyncRequestBody.fromByteBuffer(byteBuffer);
- assertThat(requestBody.contentType()).isEqualTo(Mimetype.MIMETYPE_OCTET_STREAM);
+ assertEquals(Mimetype.MIMETYPE_OCTET_STREAM, requestBody.contentType());
}
@Test
- public void emptyBytesConstructorHasCorrectContentType() {
+ void emptyBytesConstructorHasCorrectContentType() {
AsyncRequestBody requestBody = AsyncRequestBody.empty();
- assertThat(requestBody.contentType()).isEqualTo(Mimetype.MIMETYPE_OCTET_STREAM);
+ assertEquals(Mimetype.MIMETYPE_OCTET_STREAM, requestBody.contentType());
}
@Test
- public void publisherConstructorHasCorrectContentType() {
+ void publisherConstructorHasCorrectContentType() {
List requestBodyStrings = Lists.newArrayList("A", "B", "C");
List bodyBytes = requestBodyStrings.stream()
- .map(s -> ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8)))
- .collect(Collectors.toList());
+ .map(s -> ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8)))
+ .collect(Collectors.toList());
Publisher bodyPublisher = Flowable.fromIterable(bodyBytes);
AsyncRequestBody requestBody = AsyncRequestBody.fromPublisher(bodyPublisher);
- assertThat(requestBody.contentType()).isEqualTo(Mimetype.MIMETYPE_OCTET_STREAM);
- }
-
- @Test
- public void fromBytes_byteArrayNotNull_createsCopy() {
- byte[] original = {0x1, 0x2, 0x3, 0x4};
- byte[] toModify = new byte[original.length];
- System.arraycopy(original, 0, toModify, 0, original.length);
- AsyncRequestBody body = AsyncRequestBody.fromBytes(toModify);
- for (int i = 0; i < toModify.length; ++i) {
- toModify[i]++;
- }
- ByteBuffer publishedBb = Flowable.fromPublisher(body).toList().blockingGet().get(0);
- assertThat(BinaryUtils.copyAllBytesFrom(publishedBb)).isEqualTo(original);
+ assertEquals(Mimetype.MIMETYPE_OCTET_STREAM, requestBody.contentType());
}
}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ByteArrayAsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ByteArrayAsyncRequestBodyTest.java
deleted file mode 100644
index 378fbf2f59c3..000000000000
--- a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ByteArrayAsyncRequestBodyTest.java
+++ /dev/null
@@ -1,65 +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.junit.jupiter.api.Assertions.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.junit.jupiter.api.Test;
-import org.reactivestreams.Subscriber;
-import org.reactivestreams.Subscription;
-import software.amazon.awssdk.core.internal.util.Mimetype;
-
-public class ByteArrayAsyncRequestBodyTest {
- private class testSubscriber implements Subscriber {
- private Subscription subscription;
- protected AtomicBoolean onCompleteCalled = new AtomicBoolean(false);
-
- @Override
- public void onSubscribe(Subscription s) {
- this.subscription = s;
- s.request(1);
- }
-
- @Override
- public void onNext(ByteBuffer byteBuffer) {
-
- }
-
- @Override
- public void onError(Throwable throwable) {
-
- }
-
- @Override
- public void onComplete() {
- subscription.request(1);
- onCompleteCalled.set(true);
- }
- }
-
- testSubscriber subscriber = new testSubscriber();
-
- @Test
- public void concurrentRequests_shouldCompleteNormally() {
- ByteArrayAsyncRequestBody byteArrayReq = new ByteArrayAsyncRequestBody("Hello World!".getBytes(),
- Mimetype.MIMETYPE_OCTET_STREAM);
- byteArrayReq.subscribe(subscriber);
- assertTrue(subscriber.onCompleteCalled.get());
- }
-
-}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBodyTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBodyTest.java
new file mode 100644
index 000000000000..b4073247f8b9
--- /dev/null
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/async/ByteBuffersAsyncRequestBodyTest.java
@@ -0,0 +1,227 @@
+/*
+ * 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.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+import org.junit.jupiter.api.Test;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import software.amazon.awssdk.core.async.AsyncRequestBody;
+import software.amazon.awssdk.utils.BinaryUtils;
+
+class ByteBuffersAsyncRequestBodyTest {
+
+ private static class TestSubscriber implements Subscriber {
+ private Subscription subscription;
+ private boolean onCompleteCalled = false;
+ private int callsToComplete = 0;
+ private final List publishedResults = Collections.synchronizedList(new ArrayList<>());
+
+ public void request(long n) {
+ subscription.request(n);
+ }
+
+ @Override
+ public void onSubscribe(Subscription s) {
+ this.subscription = s;
+ }
+
+ @Override
+ public void onNext(ByteBuffer byteBuffer) {
+ publishedResults.add(byteBuffer);
+ }
+
+ @Override
+ public void onError(Throwable throwable) {
+ throw new IllegalStateException(throwable);
+ }
+
+ @Override
+ public void onComplete() {
+ onCompleteCalled = true;
+ callsToComplete++;
+ }
+ }
+
+ @Test
+ public void subscriberIsMarkedAsCompleted() {
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.from("Hello World!".getBytes(StandardCharsets.UTF_8));
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+ subscriber.request(1);
+
+ assertTrue(subscriber.onCompleteCalled);
+ assertEquals(1, subscriber.publishedResults.size());
+ }
+
+ @Test
+ public void subscriberIsMarkedAsCompletedWhenARequestIsMadeForMoreBuffersThanAreAvailable() {
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.from("Hello World!".getBytes(StandardCharsets.UTF_8));
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+ subscriber.request(2);
+
+ assertTrue(subscriber.onCompleteCalled);
+ assertEquals(1, subscriber.publishedResults.size());
+ }
+
+ @Test
+ public void subscriberIsThreadSafeAndMarkedAsCompletedExactlyOnce() throws InterruptedException {
+ int numBuffers = 100;
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.of(IntStream.range(0, numBuffers)
+ .mapToObj(i -> ByteBuffer.wrap(new byte[1]))
+ .toArray(ByteBuffer[]::new));
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+
+ int parallelism = 8;
+ ExecutorService executorService = Executors.newFixedThreadPool(parallelism);
+ for (int i = 0; i < parallelism; i++) {
+ executorService.submit(() -> {
+ for (int j = 0; j < numBuffers; j++) {
+ subscriber.request(2);
+ }
+ });
+ }
+ executorService.shutdown();
+ executorService.awaitTermination(1, TimeUnit.MINUTES);
+
+ assertTrue(subscriber.onCompleteCalled);
+ assertEquals(1, subscriber.callsToComplete);
+ assertEquals(numBuffers, subscriber.publishedResults.size());
+ }
+
+ @Test
+ public void subscriberIsNotMarkedAsCompletedWhenThereAreRemainingBuffersToPublish() {
+ byte[] helloWorld = "Hello World!".getBytes(StandardCharsets.UTF_8);
+ byte[] goodbyeWorld = "Goodbye World!".getBytes(StandardCharsets.UTF_8);
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.of((long) (helloWorld.length + goodbyeWorld.length),
+ ByteBuffer.wrap(helloWorld),
+ ByteBuffer.wrap(goodbyeWorld));
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+ subscriber.request(1);
+
+ assertFalse(subscriber.onCompleteCalled);
+ assertEquals(1, subscriber.publishedResults.size());
+ }
+
+ @Test
+ public void subscriberReceivesAllBuffers() {
+ byte[] helloWorld = "Hello World!".getBytes(StandardCharsets.UTF_8);
+ byte[] goodbyeWorld = "Goodbye World!".getBytes(StandardCharsets.UTF_8);
+
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.of((long) (helloWorld.length + goodbyeWorld.length),
+ ByteBuffer.wrap(helloWorld),
+ ByteBuffer.wrap(goodbyeWorld));
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+ subscriber.request(2);
+
+ assertEquals(2, subscriber.publishedResults.size());
+ assertTrue(subscriber.onCompleteCalled);
+ assertArrayEquals(helloWorld, BinaryUtils.copyAllBytesFrom(subscriber.publishedResults.get(0)));
+ assertArrayEquals(goodbyeWorld, BinaryUtils.copyAllBytesFrom(subscriber.publishedResults.get(1)));
+ }
+
+ @Test
+ public void multipleSubscribersReceiveTheSameResults() {
+ ByteBuffer sourceBuffer = ByteBuffer.wrap("Hello World!".getBytes(StandardCharsets.UTF_8));
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.of(sourceBuffer);
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+ subscriber.request(1);
+ TestSubscriber otherSubscriber = new TestSubscriber();
+ requestBody.subscribe(otherSubscriber);
+ otherSubscriber.request(1);
+
+ ByteBuffer publishedBuffer = subscriber.publishedResults.get(0);
+ ByteBuffer otherPublishedBuffer = otherSubscriber.publishedResults.get(0);
+
+ assertEquals(publishedBuffer, otherPublishedBuffer);
+ }
+
+ @Test
+ public void canceledSubscriberDoesNotReturnNewResults() {
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.of(ByteBuffer.wrap(new byte[0]));
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+
+ subscriber.subscription.cancel();
+ subscriber.request(1);
+
+ assertTrue(subscriber.publishedResults.isEmpty());
+ }
+
+ // Pending discussions on https://github.com/aws/aws-sdk-java-v2/issues/3928
+ @Test
+ public void directBuffersAreCoppiedToNonDirectBuffers() {
+ byte[] bytes = "Hello World!".getBytes(StandardCharsets.UTF_8);
+ ByteBuffer buffer = ByteBuffer.allocateDirect(bytes.length)
+ .put(bytes);
+ buffer.flip();
+ AsyncRequestBody requestBody = ByteBuffersAsyncRequestBody.of(buffer);
+
+ TestSubscriber subscriber = new TestSubscriber();
+ requestBody.subscribe(subscriber);
+ subscriber.request(1);
+
+ ByteBuffer publishedBuffer = subscriber.publishedResults.get(0);
+ assertFalse(publishedBuffer.isDirect());
+ byte[] publishedBytes = new byte[publishedBuffer.remaining()];
+ publishedBuffer.get(publishedBytes);
+ assertArrayEquals(bytes, publishedBytes);
+ }
+
+ @Test
+ public void staticOfByteBufferConstructorSetsLengthBasedOnBufferRemaining() {
+ ByteBuffer bb1 = ByteBuffer.allocate(2);
+ ByteBuffer bb2 = ByteBuffer.allocate(2);
+ bb2.position(1);
+ ByteBuffersAsyncRequestBody body = ByteBuffersAsyncRequestBody.of(bb1, bb2);
+ assertTrue(body.contentLength().isPresent());
+ assertEquals(bb1.remaining() + bb2.remaining(), body.contentLength().get());
+ }
+
+ @Test
+ public void staticFromBytesConstructorSetsLengthBasedOnArrayLength() {
+ byte[] bytes = new byte[2];
+ ByteBuffersAsyncRequestBody body = ByteBuffersAsyncRequestBody.from(bytes);
+ assertTrue(body.contentLength().isPresent());
+ assertEquals(bytes.length, body.contentLength().get());
+ }
+
+}
diff --git a/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutorTest.java b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutorTest.java
new file mode 100644
index 000000000000..2df65de46e0b
--- /dev/null
+++ b/core/sdk-core/src/test/java/software/amazon/awssdk/core/internal/waiters/WaiterExecutorTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.waiters;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.LongAdder;
+import org.junit.jupiter.api.Test;
+import org.testng.Assert;
+import software.amazon.awssdk.core.retry.backoff.BackoffStrategy;
+import software.amazon.awssdk.core.waiters.WaiterAcceptor;
+import software.amazon.awssdk.core.waiters.WaiterOverrideConfiguration;
+
+class WaiterExecutorTest {
+ @Test
+ void largeMaxAttempts() {
+
+ int expectedAttempts = 10_000;
+
+ WaiterOverrideConfiguration conf =
+ WaiterOverrideConfiguration.builder()
+ .maxAttempts(expectedAttempts)
+ .backoffStrategy(BackoffStrategy.none())
+ .build();
+
+ WaiterExecutor sut =
+ new WaiterExecutor<>(new WaiterConfiguration(conf),
+ Arrays.asList(
+ WaiterAcceptor.retryOnResponseAcceptor(c -> c < expectedAttempts),
+ WaiterAcceptor.successOnResponseAcceptor(c -> c == expectedAttempts)
+ ));
+
+ LongAdder attemptCounter = new LongAdder();
+ sut.execute(() -> {
+ attemptCounter.increment();
+ return attemptCounter.intValue();
+ });
+
+ Assert.assertEquals(attemptCounter.intValue(), expectedAttempts);
+ }
+}
\ No newline at end of file
diff --git a/http-client-spi/pom.xml b/http-client-spi/pom.xml
index 839f2722f80a..4403c74e095d 100644
--- a/http-client-spi/pom.xml
+++ b/http-client-spi/pom.xml
@@ -22,7 +22,7 @@
aws-sdk-java-pom
software.amazon.awssdk
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
http-client-spi
AWS Java SDK :: HTTP Client Interface
diff --git a/http-clients/apache-client/pom.xml b/http-clients/apache-client/pom.xml
index d260918088dc..8dec73ab5f2e 100644
--- a/http-clients/apache-client/pom.xml
+++ b/http-clients/apache-client/pom.xml
@@ -21,7 +21,7 @@
http-clients
software.amazon.awssdk
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
apache-client
diff --git a/http-clients/apache-client/src/main/java/software/amazon/awssdk/http/apache/internal/conn/IdleConnectionReaper.java b/http-clients/apache-client/src/main/java/software/amazon/awssdk/http/apache/internal/conn/IdleConnectionReaper.java
index c40d7671f86d..af46691e5a19 100644
--- a/http-clients/apache-client/src/main/java/software/amazon/awssdk/http/apache/internal/conn/IdleConnectionReaper.java
+++ b/http-clients/apache-client/src/main/java/software/amazon/awssdk/http/apache/internal/conn/IdleConnectionReaper.java
@@ -16,8 +16,9 @@
package software.amazon.awssdk.http.apache.internal.conn;
import java.time.Duration;
+import java.util.Collections;
import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.WeakHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
@@ -48,7 +49,7 @@ public final class IdleConnectionReaper {
private volatile ReaperTask reaperTask;
private IdleConnectionReaper() {
- this.connectionManagers = new ConcurrentHashMap<>();
+ this.connectionManagers = Collections.synchronizedMap(new WeakHashMap<>());
this.executorServiceSupplier = () -> {
ExecutorService e = Executors.newSingleThreadExecutor(r -> {
diff --git a/http-clients/aws-crt-client/pom.xml b/http-clients/aws-crt-client/pom.xml
index d71e03a1d58f..bdf6f373a95c 100644
--- a/http-clients/aws-crt-client/pom.xml
+++ b/http-clients/aws-crt-client/pom.xml
@@ -21,7 +21,7 @@
http-clients
software.amazon.awssdk
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
4.0.0
diff --git a/http-clients/netty-nio-client/pom.xml b/http-clients/netty-nio-client/pom.xml
index ee1f50375d89..eb8c08cc618c 100644
--- a/http-clients/netty-nio-client/pom.xml
+++ b/http-clients/netty-nio-client/pom.xml
@@ -20,7 +20,7 @@
http-clients
software.amazon.awssdk
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
4.0.0
@@ -85,6 +85,15 @@
io.netty
netty-transport-classes-epoll
+
+ io.netty
+ netty-resolver
+
+
+ io.netty
+ netty-resolver-dns
+ true
+
diff --git a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClient.java b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClient.java
index 78a3fa80fa87..c12aeab10180 100644
--- a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClient.java
+++ b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClient.java
@@ -103,6 +103,7 @@ private NettyNioAsyncHttpClient(DefaultBuilder builder, AttributeMap serviceDefa
.sdkEventLoopGroup(sdkEventLoopGroup)
.sslProvider(resolveSslProvider(builder))
.proxyConfiguration(builder.proxyConfiguration)
+ .useNonBlockingDnsResolver(builder.useNonBlockingDnsResolver)
.build();
}
@@ -475,6 +476,15 @@ public interface Builder extends SdkAsyncHttpClient.Builder http2ConfigurationBuilderConsumer);
+
+ /**
+ * Configure whether to use a non-blocking dns resolver or not. False by default, as netty's default dns resolver is
+ * blocking; it namely calls java.net.InetAddress.getByName.
+ *
+ * When enabled, a non-blocking dns resolver will be used instead, by modifying netty's bootstrap configuration.
+ * See https://netty.io/news/2016/05/26/4-1-0-Final.html
+ */
+ Builder useNonBlockingDnsResolver(Boolean useNonBlockingDnsResolver);
}
/**
@@ -492,6 +502,7 @@ private static final class DefaultBuilder implements Builder {
private Http2Configuration http2Configuration;
private SslProvider sslProvider;
private ProxyConfiguration proxyConfiguration;
+ private Boolean useNonBlockingDnsResolver;
private DefaultBuilder() {
}
@@ -716,6 +727,16 @@ public void setHttp2Configuration(Http2Configuration http2Configuration) {
http2Configuration(http2Configuration);
}
+ @Override
+ public Builder useNonBlockingDnsResolver(Boolean useNonBlockingDnsResolver) {
+ this.useNonBlockingDnsResolver = useNonBlockingDnsResolver;
+ return this;
+ }
+
+ public void setUseNonBlockingDnsResolver(Boolean useNonBlockingDnsResolver) {
+ useNonBlockingDnsResolver(useNonBlockingDnsResolver);
+ }
+
@Override
public SdkAsyncHttpClient buildWithDefaults(AttributeMap serviceDefaults) {
if (standardOptions.get(SdkHttpConfigurationOption.TLS_NEGOTIATION_TIMEOUT) == null) {
diff --git a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroup.java b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroup.java
index abb665f2c39a..254211e9303f 100644
--- a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroup.java
+++ b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroup.java
@@ -19,11 +19,13 @@
import io.netty.channel.ChannelFactory;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.DatagramChannel;
+import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import java.util.Optional;
import java.util.concurrent.ThreadFactory;
import software.amazon.awssdk.annotations.SdkPublicApi;
-import software.amazon.awssdk.http.nio.netty.internal.utils.SocketChannelResolver;
+import software.amazon.awssdk.http.nio.netty.internal.utils.ChannelResolver;
import software.amazon.awssdk.utils.ThreadFactoryBuilder;
import software.amazon.awssdk.utils.Validate;
@@ -39,7 +41,8 @@
*
*
Using {@link #create(EventLoopGroup)} to provide a custom {@link EventLoopGroup}. {@link ChannelFactory} will
* be resolved based on the type of {@link EventLoopGroup} provided via
- * {@link SocketChannelResolver#resolveSocketChannelFactory(EventLoopGroup)}
+ * {@link ChannelResolver#resolveSocketChannelFactory(EventLoopGroup)} and
+ * {@link ChannelResolver#resolveDatagramChannelFactory(EventLoopGroup)}
*
*
* Using {@link #create(EventLoopGroup, ChannelFactory)} to provide a custom {@link EventLoopGroup} and
@@ -63,12 +66,14 @@ public final class SdkEventLoopGroup {
private final EventLoopGroup eventLoopGroup;
private final ChannelFactory extends Channel> channelFactory;
+ private final ChannelFactory extends DatagramChannel> datagramChannelFactory;
SdkEventLoopGroup(EventLoopGroup eventLoopGroup, ChannelFactory extends Channel> channelFactory) {
Validate.paramNotNull(eventLoopGroup, "eventLoopGroup");
Validate.paramNotNull(channelFactory, "channelFactory");
this.eventLoopGroup = eventLoopGroup;
this.channelFactory = channelFactory;
+ this.datagramChannelFactory = ChannelResolver.resolveDatagramChannelFactory(eventLoopGroup);
}
/**
@@ -76,7 +81,8 @@ public final class SdkEventLoopGroup {
*/
private SdkEventLoopGroup(DefaultBuilder builder) {
this.eventLoopGroup = resolveEventLoopGroup(builder);
- this.channelFactory = resolveChannelFactory();
+ this.channelFactory = resolveSocketChannelFactory(builder);
+ this.datagramChannelFactory = resolveDatagramChannelFactory(builder);
}
/**
@@ -93,6 +99,13 @@ public ChannelFactory extends Channel> channelFactory() {
return channelFactory;
}
+ /**
+ * @return the {@link ChannelFactory} for datagram channels to be used with Netty Http Client.
+ */
+ public ChannelFactory extends DatagramChannel> datagramChannelFactory() {
+ return datagramChannelFactory;
+ }
+
/**
* Creates a new instance of SdkEventLoopGroup with {@link EventLoopGroup} and {@link ChannelFactory}
* to be used with {@link NettyNioAsyncHttpClient}.
@@ -116,7 +129,7 @@ public static SdkEventLoopGroup create(EventLoopGroup eventLoopGroup, ChannelFac
* @return a new instance of SdkEventLoopGroup
*/
public static SdkEventLoopGroup create(EventLoopGroup eventLoopGroup) {
- return create(eventLoopGroup, SocketChannelResolver.resolveSocketChannelFactory(eventLoopGroup));
+ return create(eventLoopGroup, ChannelResolver.resolveSocketChannelFactory(eventLoopGroup));
}
public static Builder builder() {
@@ -141,11 +154,22 @@ private EventLoopGroup resolveEventLoopGroup(DefaultBuilder builder) {
}*/
}
- private ChannelFactory extends Channel> resolveChannelFactory() {
- // Currently we only support NioEventLoopGroup
+ private ChannelFactory extends Channel> resolveSocketChannelFactory(DefaultBuilder builder) {
+ return builder.channelFactory;
+ }
+
+ private ChannelFactory extends DatagramChannel> resolveDatagramChannelFactory(DefaultBuilder builder) {
+ return builder.datagramChannelFactory;
+ }
+
+ private static ChannelFactory extends Channel> defaultSocketChannelFactory() {
return NioSocketChannel::new;
}
+ private static ChannelFactory extends DatagramChannel> defaultDatagramChannelFactory() {
+ return NioDatagramChannel::new;
+ }
+
/**
* A builder for {@link SdkEventLoopGroup}.
*
@@ -172,6 +196,24 @@ public interface Builder {
*/
Builder threadFactory(ThreadFactory threadFactory);
+ /**
+ * {@link ChannelFactory} to create socket channels used by the {@link EventLoopGroup}. If not set,
+ * NioSocketChannel is used.
+ *
+ * @param channelFactory ChannelFactory to use.
+ * @return This builder for method chaining.
+ */
+ Builder channelFactory(ChannelFactory extends Channel> channelFactory);
+
+ /**
+ * {@link ChannelFactory} to create datagram channels used by the {@link EventLoopGroup}. If not set,
+ * NioDatagramChannel is used.
+ *
+ * @param datagramChannelFactory ChannelFactory to use.
+ * @return This builder for method chaining.
+ */
+ Builder datagramChannelFactory(ChannelFactory extends DatagramChannel> datagramChannelFactory);
+
SdkEventLoopGroup build();
}
@@ -179,6 +221,8 @@ private static final class DefaultBuilder implements Builder {
private Integer numberOfThreads;
private ThreadFactory threadFactory;
+ private ChannelFactory extends Channel> channelFactory = defaultSocketChannelFactory();
+ private ChannelFactory extends DatagramChannel> datagramChannelFactory = defaultDatagramChannelFactory();
private DefaultBuilder() {
}
@@ -203,6 +247,26 @@ public void setThreadFactory(ThreadFactory threadFactory) {
threadFactory(threadFactory);
}
+ @Override
+ public Builder channelFactory(ChannelFactory extends Channel> channelFactory) {
+ this.channelFactory = channelFactory;
+ return this;
+ }
+
+ public void setChannelFactory(ChannelFactory extends Channel> channelFactory) {
+ channelFactory(channelFactory);
+ }
+
+ @Override
+ public Builder datagramChannelFactory(ChannelFactory extends DatagramChannel> datagramChannelFactory) {
+ this.datagramChannelFactory = datagramChannelFactory;
+ return this;
+ }
+
+ public void setDatagramChannelFactory(ChannelFactory extends DatagramChannel> datagramChannelFactory) {
+ datagramChannelFactory(datagramChannelFactory);
+ }
+
@Override
public SdkEventLoopGroup build() {
return new SdkEventLoopGroup(this);
diff --git a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMap.java b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMap.java
index 1d55e1841aa2..fbd727239239 100644
--- a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMap.java
+++ b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMap.java
@@ -83,6 +83,7 @@ public void channelCreated(Channel ch) throws Exception {
private final ProxyConfiguration proxyConfiguration;
private final BootstrapProvider bootstrapProvider;
private final SslContextProvider sslContextProvider;
+ private final Boolean useNonBlockingDnsResolver;
private AwaitCloseChannelPoolMap(Builder builder, Function createBootStrapProvider) {
this.configuration = builder.configuration;
@@ -94,6 +95,7 @@ private AwaitCloseChannelPoolMap(Builder builder, Function init(ChannelFactory extends DatagramChannel> datagramChannelFactory) {
+ try {
+ Class> addressResolver = ClassLoaderHelper.loadClass(getAddressResolverGroup(), false, (Class) null);
+ Class> dnsNameResolverBuilder = ClassLoaderHelper.loadClass(getDnsNameResolverBuilder(), false, (Class) null);
+
+ Object dnsResolverObj = dnsNameResolverBuilder.newInstance();
+ Method method = dnsResolverObj.getClass().getMethod("channelFactory", ChannelFactory.class);
+ method.invoke(dnsResolverObj, datagramChannelFactory);
+
+ Object e = addressResolver.getConstructor(dnsNameResolverBuilder).newInstance(dnsResolverObj);
+ return (AddressResolverGroup) e;
+ } catch (ClassNotFoundException e) {
+ throw new IllegalStateException("Cannot find module io.netty.resolver.dns "
+ + " To use netty non blocking dns," +
+ " the 'netty-resolver-dns' module from io.netty must be on the class path. ", e);
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException | InstantiationException e) {
+ throw new IllegalStateException("Failed to create AddressResolverGroup", e);
+ }
+ }
+
+ private static String getAddressResolverGroup() {
+ return "io.netty.resolver.dns.DnsAddressResolverGroup";
+ }
+
+ private static String getDnsNameResolverBuilder() {
+ return "io.netty.resolver.dns.DnsNameResolverBuilder";
+ }
+}
diff --git a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/utils/ChannelResolver.java b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/utils/ChannelResolver.java
new file mode 100644
index 000000000000..8770d683a679
--- /dev/null
+++ b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/utils/ChannelResolver.java
@@ -0,0 +1,112 @@
+/*
+ * 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.http.nio.netty.internal.utils;
+
+import static software.amazon.awssdk.utils.FunctionalUtils.invokeSafely;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFactory;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.ReflectiveChannelFactory;
+import io.netty.channel.epoll.EpollDatagramChannel;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.DatagramChannel;
+import io.netty.channel.socket.nio.NioDatagramChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import java.util.HashMap;
+import java.util.Map;
+import software.amazon.awssdk.annotations.SdkInternalApi;
+import software.amazon.awssdk.http.nio.netty.internal.DelegatingEventLoopGroup;
+
+@SdkInternalApi
+public final class ChannelResolver {
+
+ private static final Map KNOWN_EL_GROUPS_SOCKET_CHANNELS = new HashMap<>();
+ private static final Map KNOWN_EL_GROUPS_DATAGRAM_CHANNELS = new HashMap<>();
+
+ static {
+ KNOWN_EL_GROUPS_SOCKET_CHANNELS.put("io.netty.channel.kqueue.KQueueEventLoopGroup",
+ "io.netty.channel.kqueue.KQueueSocketChannel");
+ KNOWN_EL_GROUPS_SOCKET_CHANNELS.put("io.netty.channel.oio.OioEventLoopGroup",
+ "io.netty.channel.socket.oio.OioSocketChannel");
+
+ KNOWN_EL_GROUPS_DATAGRAM_CHANNELS.put("io.netty.channel.kqueue.KQueueEventLoopGroup",
+ "io.netty.channel.kqueue.KQueueDatagramChannel");
+ KNOWN_EL_GROUPS_DATAGRAM_CHANNELS.put("io.netty.channel.oio.OioEventLoopGroup",
+ "io.netty.channel.socket.oio.OioDatagramChannel");
+ }
+
+ private ChannelResolver() {
+ }
+
+ /**
+ * Attempts to determine the {@link ChannelFactory} class that corresponds to the given
+ * event loop group.
+ *
+ * @param eventLoopGroup the event loop group to determine the {@link ChannelFactory} for
+ * @return A {@link ChannelFactory} instance for the given event loop group.
+ */
+ @SuppressWarnings("unchecked")
+ public static ChannelFactory extends Channel> resolveSocketChannelFactory(EventLoopGroup eventLoopGroup) {
+ if (eventLoopGroup instanceof DelegatingEventLoopGroup) {
+ return resolveSocketChannelFactory(((DelegatingEventLoopGroup) eventLoopGroup).getDelegate());
+ }
+
+ if (eventLoopGroup instanceof NioEventLoopGroup) {
+ return NioSocketChannel::new;
+ }
+ if (eventLoopGroup instanceof EpollEventLoopGroup) {
+ return EpollSocketChannel::new;
+ }
+
+ String socketFqcn = KNOWN_EL_GROUPS_SOCKET_CHANNELS.get(eventLoopGroup.getClass().getName());
+ if (socketFqcn == null) {
+ throw new IllegalArgumentException("Unknown event loop group : " + eventLoopGroup.getClass());
+ }
+
+ return invokeSafely(() -> new ReflectiveChannelFactory(Class.forName(socketFqcn)));
+ }
+
+ /**
+ * Attempts to determine the {@link ChannelFactory} class for datagram channels that corresponds to the given
+ * event loop group.
+ *
+ * @param eventLoopGroup the event loop group to determine the {@link ChannelFactory} for
+ * @return A {@link ChannelFactory} instance for the given event loop group.
+ */
+ @SuppressWarnings("unchecked")
+ public static ChannelFactory extends DatagramChannel> resolveDatagramChannelFactory(EventLoopGroup eventLoopGroup) {
+ if (eventLoopGroup instanceof DelegatingEventLoopGroup) {
+ return resolveDatagramChannelFactory(((DelegatingEventLoopGroup) eventLoopGroup).getDelegate());
+ }
+
+ if (eventLoopGroup instanceof NioEventLoopGroup) {
+ return NioDatagramChannel::new;
+ }
+ if (eventLoopGroup instanceof EpollEventLoopGroup) {
+ return EpollDatagramChannel::new;
+ }
+
+ String datagramFqcn = KNOWN_EL_GROUPS_DATAGRAM_CHANNELS.get(eventLoopGroup.getClass().getName());
+ if (datagramFqcn == null) {
+ throw new IllegalArgumentException("Unknown event loop group : " + eventLoopGroup.getClass());
+ }
+
+ return invokeSafely(() -> new ReflectiveChannelFactory(Class.forName(datagramFqcn)));
+ }
+}
diff --git a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/utils/SocketChannelResolver.java b/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/utils/SocketChannelResolver.java
deleted file mode 100644
index 1d80dad5850f..000000000000
--- a/http-clients/netty-nio-client/src/main/java/software/amazon/awssdk/http/nio/netty/internal/utils/SocketChannelResolver.java
+++ /dev/null
@@ -1,73 +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.http.nio.netty.internal.utils;
-
-import static software.amazon.awssdk.utils.FunctionalUtils.invokeSafely;
-
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFactory;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.ReflectiveChannelFactory;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollSocketChannel;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import java.util.HashMap;
-import java.util.Map;
-import software.amazon.awssdk.annotations.SdkInternalApi;
-import software.amazon.awssdk.http.nio.netty.internal.DelegatingEventLoopGroup;
-
-@SdkInternalApi
-public final class SocketChannelResolver {
-
- private static final Map KNOWN_EL_GROUPS = new HashMap<>();
-
- static {
- KNOWN_EL_GROUPS.put("io.netty.channel.kqueue.KQueueEventLoopGroup", "io.netty.channel.kqueue.KQueueSocketChannel");
- KNOWN_EL_GROUPS.put("io.netty.channel.oio.OioEventLoopGroup", "io.netty.channel.socket.oio.OioSocketChannel");
- }
-
- private SocketChannelResolver() {
- }
-
- /**
- * Attempts to determine the {@link ChannelFactory} class that corresponds to the given
- * event loop group.
- *
- * @param eventLoopGroup the event loop group to determine the {@link ChannelFactory} for
- * @return A {@link ChannelFactory} instance for the given event loop group.
- */
- @SuppressWarnings("unchecked")
- public static ChannelFactory extends Channel> resolveSocketChannelFactory(EventLoopGroup eventLoopGroup) {
- if (eventLoopGroup instanceof DelegatingEventLoopGroup) {
- return resolveSocketChannelFactory(((DelegatingEventLoopGroup) eventLoopGroup).getDelegate());
- }
-
- if (eventLoopGroup instanceof NioEventLoopGroup) {
- return NioSocketChannel::new;
- }
- if (eventLoopGroup instanceof EpollEventLoopGroup) {
- return EpollSocketChannel::new;
- }
-
- String socketFqcn = KNOWN_EL_GROUPS.get(eventLoopGroup.getClass().getName());
- if (socketFqcn == null) {
- throw new IllegalArgumentException("Unknown event loop group : " + eventLoopGroup.getClass());
- }
-
- return invokeSafely(() -> new ReflectiveChannelFactory(Class.forName(socketFqcn)));
- }
-}
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyClientTlsAuthTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyClientTlsAuthTest.java
index dc7c408c3c9f..f35c0914609d 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyClientTlsAuthTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyClientTlsAuthTest.java
@@ -39,6 +39,7 @@
import software.amazon.awssdk.http.EmptyPublisher;
import software.amazon.awssdk.http.FileStoreTlsKeyManagersProvider;
import software.amazon.awssdk.http.HttpTestUtils;
+import software.amazon.awssdk.http.SdkHttpConfigurationOption;
import software.amazon.awssdk.http.SdkHttpFullRequest;
import software.amazon.awssdk.http.SdkHttpMethod;
import software.amazon.awssdk.http.TlsKeyManagersProvider;
@@ -185,6 +186,24 @@ public void nonProxy_noKeyManagerGiven_shouldThrowException() {
.hasRootCauseInstanceOf(SSLException.class);
}
+ @Test
+ public void builderUsesProvidedKeyManagersProviderNonBlockingDns() {
+ TlsKeyManagersProvider mockKeyManagersProvider = mock(TlsKeyManagersProvider.class);
+ netty = NettyNioAsyncHttpClient.builder()
+ .useNonBlockingDnsResolver(true)
+ .proxyConfiguration(proxyCfg)
+ .tlsKeyManagersProvider(mockKeyManagersProvider)
+ .buildWithDefaults(AttributeMap.builder()
+ .put(TRUST_ALL_CERTIFICATES, true)
+ .build());
+
+ try {
+ sendRequest(netty, new RecordingResponseHandler());
+ } catch (Exception ignored) {
+ }
+ verify(mockKeyManagersProvider).keyManagers();
+ }
+
private void sendRequest(SdkAsyncHttpClient client, SdkAsyncHttpResponseHandler responseHandler) {
AsyncExecuteRequest req = AsyncExecuteRequest.builder()
.request(testSdkRequest())
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientNonBlockingDnsTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientNonBlockingDnsTest.java
new file mode 100644
index 000000000000..9535c41c2b0a
--- /dev/null
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientNonBlockingDnsTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.http.nio.netty;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
+import static com.github.tomakehurst.wiremock.client.WireMock.any;
+import static com.github.tomakehurst.wiremock.client.WireMock.equalTo;
+import static com.github.tomakehurst.wiremock.client.WireMock.postRequestedFor;
+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 com.github.tomakehurst.wiremock.core.WireMockConfiguration.wireMockConfig;
+import static java.util.Collections.singletonMap;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.commons.lang3.StringUtils.reverse;
+import static org.assertj.core.api.Assertions.assertThat;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.assertCanReceiveBasicRequest;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.createProvider;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.createRequest;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.makeSimpleRequest;
+
+import com.github.tomakehurst.wiremock.junit.WireMockRule;
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.assertj.core.api.Condition;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+import software.amazon.awssdk.http.SdkHttpConfigurationOption;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+import software.amazon.awssdk.http.SdkHttpMethod;
+import software.amazon.awssdk.http.SdkHttpRequest;
+import software.amazon.awssdk.http.async.AsyncExecuteRequest;
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.utils.AttributeMap;
+
+@RunWith(MockitoJUnitRunner.class)
+public class NettyNioAsyncHttpClientNonBlockingDnsTest {
+
+ private final RecordingNetworkTrafficListener wiremockTrafficListener = new RecordingNetworkTrafficListener();
+
+ private static final SdkAsyncHttpClient client = NettyNioAsyncHttpClient.builder()
+ .useNonBlockingDnsResolver(true)
+ .buildWithDefaults(
+ AttributeMap.builder()
+ .put(SdkHttpConfigurationOption.TRUST_ALL_CERTIFICATES, true)
+ .build());
+ @Rule
+ public WireMockRule mockServer = new WireMockRule(wireMockConfig()
+ .dynamicPort()
+ .dynamicHttpsPort()
+ .networkTrafficListener(wiremockTrafficListener));
+
+ @Before
+ public void methodSetup() {
+ wiremockTrafficListener.reset();
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ client.close();
+ }
+
+ @Test
+ public void canSendContentAndGetThatContentBackNonBlockingDns() throws Exception {
+ String body = randomAlphabetic(50);
+ stubFor(any(urlEqualTo("/echo?reversed=true"))
+ .withRequestBody(equalTo(body))
+ .willReturn(aResponse().withBody(reverse(body))));
+ URI uri = URI.create("http://localhost:" + mockServer.port());
+
+ SdkHttpRequest request = createRequest(uri, "/echo", body, SdkHttpMethod.POST, singletonMap("reversed", "true"));
+
+ RecordingResponseHandler recorder = new RecordingResponseHandler();
+
+ client.execute(AsyncExecuteRequest.builder().request(request).requestContentPublisher(createProvider(body)).responseHandler(recorder).build());
+
+ recorder.completeFuture.get(5, TimeUnit.SECONDS);
+
+ verify(1, postRequestedFor(urlEqualTo("/echo?reversed=true")));
+
+ assertThat(recorder.fullResponseAsString()).isEqualTo(reverse(body));
+ }
+
+ @Test
+ public void defaultThreadFactoryUsesHelpfulName() throws Exception {
+ // Make a request to ensure a thread is primed
+ makeSimpleRequest(client, mockServer);
+
+ String expectedPattern = "aws-java-sdk-NettyEventLoop-\\d+-\\d+";
+ assertThat(Thread.getAllStackTraces().keySet())
+ .areAtLeast(1, new Condition<>(t -> t.getName().matches(expectedPattern),
+ "Matches default thread pattern: `%s`", expectedPattern));
+ }
+
+ @Test
+ public void canMakeBasicRequestOverHttp() throws Exception {
+ String smallBody = randomAlphabetic(10);
+ URI uri = URI.create("http://localhost:" + mockServer.port());
+
+ assertCanReceiveBasicRequest(client, uri, smallBody);
+ }
+
+ @Test
+ public void canMakeBasicRequestOverHttps() throws Exception {
+ String smallBody = randomAlphabetic(10);
+ URI uri = URI.create("https://localhost:" + mockServer.httpsPort());
+
+ assertCanReceiveBasicRequest(client, uri, smallBody);
+ }
+
+ @Test
+ public void canHandleLargerPayloadsOverHttp() throws Exception {
+ String largishBody = randomAlphabetic(25000);
+
+ URI uri = URI.create("http://localhost:" + mockServer.port());
+
+ assertCanReceiveBasicRequest(client, uri, largishBody);
+ }
+
+ @Test
+ public void canHandleLargerPayloadsOverHttps() throws Exception {
+ String largishBody = randomAlphabetic(25000);
+
+ URI uri = URI.create("https://localhost:" + mockServer.httpsPort());
+
+ assertCanReceiveBasicRequest(client, uri, largishBody);
+ }
+
+ @Test
+ public void requestContentOnlyEqualToContentLengthHeaderFromProvider() throws InterruptedException, ExecutionException, TimeoutException, IOException {
+ final String content = randomAlphabetic(32);
+ final String streamContent = content + reverse(content);
+ stubFor(any(urlEqualTo("/echo?reversed=true"))
+ .withRequestBody(equalTo(content))
+ .willReturn(aResponse().withBody(reverse(content))));
+ URI uri = URI.create("http://localhost:" + mockServer.port());
+
+ SdkHttpFullRequest request = createRequest(uri, "/echo", streamContent, SdkHttpMethod.POST, singletonMap("reversed", "true"));
+ request = request.toBuilder().putHeader("Content-Length", Integer.toString(content.length())).build();
+ RecordingResponseHandler recorder = new RecordingResponseHandler();
+
+ client.execute(AsyncExecuteRequest.builder().request(request).requestContentPublisher(createProvider(streamContent)).responseHandler(recorder).build());
+
+ recorder.completeFuture.get(5, TimeUnit.SECONDS);
+
+ // HTTP servers will stop processing the request as soon as it reads
+ // bytes equal to 'Content-Length' so we need to inspect the raw
+ // traffic to ensure that there wasn't anything after that.
+ assertThat(wiremockTrafficListener.requests().toString()).endsWith(content);
+ }
+}
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientTestUtils.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientTestUtils.java
new file mode 100644
index 000000000000..04f9a906ee04
--- /dev/null
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientTestUtils.java
@@ -0,0 +1,148 @@
+/*
+ * 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.http.nio.netty;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
+import static com.github.tomakehurst.wiremock.client.WireMock.any;
+import static com.github.tomakehurst.wiremock.client.WireMock.getRequestedFor;
+import static com.github.tomakehurst.wiremock.client.WireMock.stubFor;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlMatching;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo;
+import static com.github.tomakehurst.wiremock.client.WireMock.verify;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Collections.emptyMap;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.commons.lang3.StringUtils.isBlank;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.github.tomakehurst.wiremock.WireMockServer;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+import software.amazon.awssdk.http.SdkHttpMethod;
+import software.amazon.awssdk.http.SdkHttpRequest;
+import software.amazon.awssdk.http.async.AsyncExecuteRequest;
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.http.async.SdkHttpContentPublisher;
+
+public class NettyNioAsyncHttpClientTestUtils {
+
+ /**
+ * Make a simple async request and wait for it to fiish.
+ *
+ * @param client Client to make request with.
+ */
+ public static void makeSimpleRequest(SdkAsyncHttpClient client, WireMockServer mockServer) throws Exception {
+ String body = randomAlphabetic(10);
+ URI uri = URI.create("http://localhost:" + mockServer.port());
+ stubFor(any(urlPathEqualTo("/")).willReturn(aResponse().withBody(body)));
+ SdkHttpRequest request = createRequest(uri);
+ RecordingResponseHandler recorder = new RecordingResponseHandler();
+ client.execute(AsyncExecuteRequest.builder().request(request).requestContentPublisher(createProvider("")).responseHandler(recorder).build());
+ recorder.completeFuture.get(5, TimeUnit.SECONDS);
+ }
+
+ public static SdkHttpContentPublisher createProvider(String body) {
+ Stream chunks = splitStringBySize(body).stream()
+ .map(chunk -> ByteBuffer.wrap(chunk.getBytes(UTF_8)));
+ return new SdkHttpContentPublisher() {
+
+ @Override
+ public Optional contentLength() {
+ return Optional.of(Long.valueOf(body.length()));
+ }
+
+ @Override
+ public void subscribe(Subscriber super ByteBuffer> s) {
+ s.onSubscribe(new Subscription() {
+ @Override
+ public void request(long n) {
+ chunks.forEach(s::onNext);
+ s.onComplete();
+ }
+
+ @Override
+ public void cancel() {
+
+ }
+ });
+ }
+ };
+ }
+
+ public static SdkHttpFullRequest createRequest(URI uri) {
+ return createRequest(uri, "/", null, SdkHttpMethod.GET, emptyMap());
+ }
+
+ public static SdkHttpFullRequest createRequest(URI uri,
+ String resourcePath,
+ String body,
+ SdkHttpMethod method,
+ Map params) {
+ String contentLength = body == null ? null : String.valueOf(body.getBytes(UTF_8).length);
+ return SdkHttpFullRequest.builder()
+ .uri(uri)
+ .method(method)
+ .encodedPath(resourcePath)
+ .applyMutation(b -> params.forEach(b::putRawQueryParameter))
+ .applyMutation(b -> {
+ b.putHeader("Host", uri.getHost());
+ if (contentLength != null) {
+ b.putHeader("Content-Length", contentLength);
+ }
+ }).build();
+ }
+
+ public static void assertCanReceiveBasicRequest(SdkAsyncHttpClient client, URI uri, String body) throws Exception {
+ stubFor(any(urlPathEqualTo("/")).willReturn(aResponse().withHeader("Some-Header", "With Value").withBody(body)));
+
+ SdkHttpRequest request = createRequest(uri);
+
+ RecordingResponseHandler recorder = new RecordingResponseHandler();
+ client.execute(AsyncExecuteRequest.builder().request(request).requestContentPublisher(createProvider("")).responseHandler(recorder).build());
+
+ recorder.completeFuture.get(5, TimeUnit.SECONDS);
+
+ assertThat(recorder.responses).hasOnlyOneElementSatisfying(
+ headerResponse -> {
+ assertThat(headerResponse.headers()).containsKey("Some-Header");
+ assertThat(headerResponse.statusCode()).isEqualTo(200);
+ });
+
+ assertThat(recorder.fullResponseAsString()).isEqualTo(body);
+ verify(1, getRequestedFor(urlMatching("/")));
+ }
+
+ private static Collection splitStringBySize(String str) {
+ if (isBlank(str)) {
+ return Collections.emptyList();
+ }
+ ArrayList split = new ArrayList<>();
+ for (int i = 0; i <= str.length() / 1000; i++) {
+ split.add(str.substring(i * 1000, Math.min((i + 1) * 1000, str.length())));
+ }
+ return split;
+ }
+}
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientWireMockTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientWireMockTest.java
index 9a1121e201f5..116119d36ea5 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientWireMockTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/NettyNioAsyncHttpClientWireMockTest.java
@@ -18,19 +18,14 @@
import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
import static com.github.tomakehurst.wiremock.client.WireMock.any;
import static com.github.tomakehurst.wiremock.client.WireMock.equalTo;
-import static com.github.tomakehurst.wiremock.client.WireMock.getRequestedFor;
import static com.github.tomakehurst.wiremock.client.WireMock.postRequestedFor;
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.urlMatching;
import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo;
import static com.github.tomakehurst.wiremock.client.WireMock.verify;
import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.wireMockConfig;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonMap;
import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
-import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.apache.commons.lang3.StringUtils.reverse;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
@@ -40,6 +35,10 @@
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.when;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.assertCanReceiveBasicRequest;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.createProvider;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.createRequest;
+import static software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClientTestUtils.makeSimpleRequest;
import com.github.tomakehurst.wiremock.WireMockServer;
import com.github.tomakehurst.wiremock.http.Fault;
@@ -49,25 +48,22 @@
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.DatagramChannel;
+import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.SslProvider;
import io.netty.util.AttributeKey;
import java.io.IOException;
import java.net.URI;
-import java.nio.ByteBuffer;
import java.time.Duration;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
-import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.stream.Stream;
import javax.net.ssl.TrustManagerFactory;
import org.assertj.core.api.Condition;
import org.junit.AfterClass;
@@ -78,8 +74,6 @@
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import org.mockito.stubbing.Answer;
-import org.reactivestreams.Subscriber;
-import org.reactivestreams.Subscription;
import software.amazon.awssdk.http.HttpMetric;
import software.amazon.awssdk.http.HttpTestUtils;
import software.amazon.awssdk.http.SdkHttpConfigurationOption;
@@ -88,7 +82,6 @@
import software.amazon.awssdk.http.SdkHttpRequest;
import software.amazon.awssdk.http.async.AsyncExecuteRequest;
import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
-import software.amazon.awssdk.http.async.SdkHttpContentPublisher;
import software.amazon.awssdk.http.nio.netty.internal.NettyConfiguration;
import software.amazon.awssdk.http.nio.netty.internal.SdkChannelPool;
import software.amazon.awssdk.http.nio.netty.internal.SdkChannelPoolMap;
@@ -183,7 +176,8 @@ public void invalidMaxPendingConnectionAcquireConfig_shouldPropagateException()
.maxConcurrency(1)
.maxPendingConnectionAcquires(0)
.build()) {
- assertThatThrownBy(() -> makeSimpleRequest(customClient)).hasMessageContaining("java.lang.IllegalArgumentException: maxPendingAcquires: 0 (expected: >= 1)");
+ assertThatThrownBy(() -> makeSimpleRequest(customClient, mockServer)).hasMessageContaining("java.lang"
+ + ".IllegalArgumentException: maxPendingAcquires: 0 (expected: >= 1)");
}
}
@@ -196,7 +190,7 @@ public void customFactoryIsUsed() throws Exception {
.threadFactory(threadFactory))
.build();
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
customClient.close();
Mockito.verify(threadFactory, atLeastOnce()).newThread(Mockito.any());
@@ -208,7 +202,7 @@ public void openSslBeingUsed() throws Exception {
NettyNioAsyncHttpClient.builder()
.sslProvider(SslProvider.OPENSSL)
.build()) {
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
}
}
@@ -218,7 +212,7 @@ public void defaultJdkSslProvider() throws Exception {
NettyNioAsyncHttpClient.builder()
.sslProvider(SslProvider.JDK)
.build()) {
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
customClient.close();
}
}
@@ -226,7 +220,7 @@ public void defaultJdkSslProvider() throws Exception {
@Test
public void defaultThreadFactoryUsesHelpfulName() throws Exception {
// Make a request to ensure a thread is primed
- makeSimpleRequest(client);
+ makeSimpleRequest(client, mockServer);
String expectedPattern = "aws-java-sdk-NettyEventLoop-\\d+-\\d+";
assertThat(Thread.getAllStackTraces().keySet())
@@ -247,7 +241,7 @@ public void customThreadCountIsRespected() throws Exception {
// Have to make enough requests to prime the threads
for (int i = 0; i < threadCount + 1; i++) {
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
}
customClient.close();
@@ -267,7 +261,7 @@ public void customEventLoopGroup_NotClosedWhenClientIsClosed() throws Exception
.eventLoopGroup(SdkEventLoopGroup.create(eventLoopGroup, NioSocketChannel::new))
.build();
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
customClient.close();
Mockito.verify(threadFactory, atLeastOnce()).newThread(Mockito.any());
@@ -287,7 +281,7 @@ public void customChannelFactoryIsUsed() throws Exception {
.eventLoopGroup(SdkEventLoopGroup.create(customEventLoopGroup, channelFactory))
.build();
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
customClient.close();
Mockito.verify(channelFactory, atLeastOnce()).newChannel();
@@ -335,7 +329,7 @@ public void responseConnectionReused_shouldReleaseChannel() throws Exception {
.maxConcurrency(1)
.build();
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
verifyChannelRelease(channel);
assertThat(channel.isShutdown()).isFalse();
@@ -446,27 +440,12 @@ public void builderUsesProvidedTrustManagersProvider() throws Exception {
}
}
- /**
- * Make a simple async request and wait for it to fiish.
- *
- * @param client Client to make request with.
- */
- private void makeSimpleRequest(SdkAsyncHttpClient client) throws Exception {
- String body = randomAlphabetic(10);
- URI uri = URI.create("http://localhost:" + mockServer.port());
- stubFor(any(urlPathEqualTo("/")).willReturn(aResponse().withBody(body)));
- SdkHttpRequest request = createRequest(uri);
- RecordingResponseHandler recorder = new RecordingResponseHandler();
- client.execute(AsyncExecuteRequest.builder().request(request).requestContentPublisher(createProvider("")).responseHandler(recorder).build());
- recorder.completeFuture.get(5, TimeUnit.SECONDS);
- }
-
@Test
public void canMakeBasicRequestOverHttp() throws Exception {
String smallBody = randomAlphabetic(10);
URI uri = URI.create("http://localhost:" + mockServer.port());
- assertCanReceiveBasicRequest(uri, smallBody);
+ assertCanReceiveBasicRequest(client, uri, smallBody);
}
@Test
@@ -474,7 +453,7 @@ public void canMakeBasicRequestOverHttps() throws Exception {
String smallBody = randomAlphabetic(10);
URI uri = URI.create("https://localhost:" + mockServer.httpsPort());
- assertCanReceiveBasicRequest(uri, smallBody);
+ assertCanReceiveBasicRequest(client, uri, smallBody);
}
@Test
@@ -483,7 +462,7 @@ public void canHandleLargerPayloadsOverHttp() throws Exception {
URI uri = URI.create("http://localhost:" + mockServer.port());
- assertCanReceiveBasicRequest(uri, largishBody);
+ assertCanReceiveBasicRequest(client, uri, largishBody);
}
@Test
@@ -492,7 +471,7 @@ public void canHandleLargerPayloadsOverHttps() throws Exception {
URI uri = URI.create("https://localhost:" + mockServer.httpsPort());
- assertCanReceiveBasicRequest(uri, largishBody);
+ assertCanReceiveBasicRequest(client, uri, largishBody);
}
@Test
@@ -579,88 +558,6 @@ public ChannelFuture close() {
assertThat(channelClosedFuture.get(5, TimeUnit.SECONDS)).isTrue();
}
- private void assertCanReceiveBasicRequest(URI uri, String body) throws Exception {
- stubFor(any(urlPathEqualTo("/")).willReturn(aResponse().withHeader("Some-Header", "With Value").withBody(body)));
-
- SdkHttpRequest request = createRequest(uri);
-
- RecordingResponseHandler recorder = new RecordingResponseHandler();
- client.execute(AsyncExecuteRequest.builder().request(request).requestContentPublisher(createProvider("")).responseHandler(recorder).build());
-
- recorder.completeFuture.get(5, TimeUnit.SECONDS);
-
- assertThat(recorder.responses).hasOnlyOneElementSatisfying(
- headerResponse -> {
- assertThat(headerResponse.headers()).containsKey("Some-Header");
- assertThat(headerResponse.statusCode()).isEqualTo(200);
- });
-
- assertThat(recorder.fullResponseAsString()).isEqualTo(body);
- verify(1, getRequestedFor(urlMatching("/")));
- }
-
- private SdkHttpContentPublisher createProvider(String body) {
- Stream chunks = splitStringBySize(body).stream()
- .map(chunk -> ByteBuffer.wrap(chunk.getBytes(UTF_8)));
- return new SdkHttpContentPublisher() {
-
- @Override
- public Optional contentLength() {
- return Optional.of(Long.valueOf(body.length()));
- }
-
- @Override
- public void subscribe(Subscriber super ByteBuffer> s) {
- s.onSubscribe(new Subscription() {
- @Override
- public void request(long n) {
- chunks.forEach(s::onNext);
- s.onComplete();
- }
-
- @Override
- public void cancel() {
-
- }
- });
- }
- };
- }
-
- private SdkHttpFullRequest createRequest(URI uri) {
- return createRequest(uri, "/", null, SdkHttpMethod.GET, emptyMap());
- }
-
- private SdkHttpFullRequest createRequest(URI uri,
- String resourcePath,
- String body,
- SdkHttpMethod method,
- Map params) {
- String contentLength = body == null ? null : String.valueOf(body.getBytes(UTF_8).length);
- return SdkHttpFullRequest.builder()
- .uri(uri)
- .method(method)
- .encodedPath(resourcePath)
- .applyMutation(b -> params.forEach(b::putRawQueryParameter))
- .applyMutation(b -> {
- b.putHeader("Host", uri.getHost());
- if (contentLength != null) {
- b.putHeader("Content-Length", contentLength);
- }
- }).build();
- }
-
- private static Collection splitStringBySize(String str) {
- if (isBlank(str)) {
- return Collections.emptyList();
- }
- ArrayList split = new ArrayList<>();
- for (int i = 0; i <= str.length() / 1000; i++) {
- split.add(str.substring(i * 1000, Math.min((i + 1) * 1000, str.length())));
- }
- return split;
- }
-
// Needs to be a non-anon class in order to spy
public static class CustomThreadFactory implements ThreadFactory {
@Override
@@ -719,7 +616,7 @@ public void createNettyClient_ReadWriteTimeoutCanBeZero() throws Exception {
.writeTimeout(Duration.ZERO)
.build();
- makeSimpleRequest(customClient);
+ makeSimpleRequest(customClient, mockServer);
customClient.close();
}
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/ProxyWireMockTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/ProxyWireMockTest.java
index f797a760fdf7..438d65e1f9fc 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/ProxyWireMockTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/ProxyWireMockTest.java
@@ -126,6 +126,30 @@ public void proxyConfigured_hostInNonProxySet_doesNotConnect() {
assertThat(responseHandler.fullResponseAsString()).isEqualTo("hello");
}
+ @Test
+ public void proxyConfigured_hostInNonProxySet_nonBlockingDns_doesNotConnect() {
+ RecordingResponseHandler responseHandler = new RecordingResponseHandler();
+ AsyncExecuteRequest req = AsyncExecuteRequest.builder()
+ .request(testSdkRequest())
+ .responseHandler(responseHandler)
+ .requestContentPublisher(new EmptyPublisher())
+ .build();
+
+ ProxyConfiguration cfg = proxyCfg.toBuilder()
+ .nonProxyHosts(Stream.of("localhost").collect(Collectors.toSet()))
+ .build();
+
+ client = NettyNioAsyncHttpClient.builder()
+ .proxyConfiguration(cfg)
+ .useNonBlockingDnsResolver(true)
+ .build();
+
+ client.execute(req).join();
+
+ responseHandler.completeFuture.join();
+ assertThat(responseHandler.fullResponseAsString()).isEqualTo("hello");
+ }
+
private SdkHttpFullRequest testSdkRequest() {
return SdkHttpFullRequest.builder()
.method(SdkHttpMethod.GET)
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroupTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroupTest.java
index a3ae76469359..bb2598345cff 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroupTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/SdkEventLoopGroupTest.java
@@ -18,8 +18,15 @@
import static org.assertj.core.api.Assertions.assertThat;
import io.netty.channel.DefaultEventLoopGroup;
+import io.netty.channel.epoll.EpollDatagramChannel;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollSocketChannel;
import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.oio.OioEventLoopGroup;
+import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.channel.socket.oio.OioDatagramChannel;
+import io.netty.channel.socket.oio.OioSocketChannel;
import org.junit.Test;
public class SdkEventLoopGroupTest {
@@ -28,13 +35,24 @@ public class SdkEventLoopGroupTest {
public void creatingUsingBuilder() {
SdkEventLoopGroup sdkEventLoopGroup = SdkEventLoopGroup.builder().numberOfThreads(1).build();
assertThat(sdkEventLoopGroup.channelFactory()).isNotNull();
+ assertThat(sdkEventLoopGroup.datagramChannelFactory()).isNotNull();
assertThat(sdkEventLoopGroup.eventLoopGroup()).isNotNull();
}
@Test
- public void creatingUsingStaticMethod() {
+ public void creatingUsingStaticMethod_A() {
SdkEventLoopGroup sdkEventLoopGroup = SdkEventLoopGroup.create(new NioEventLoopGroup(), NioSocketChannel::new);
assertThat(sdkEventLoopGroup.channelFactory()).isNotNull();
+ assertThat(sdkEventLoopGroup.datagramChannelFactory().newChannel()).isInstanceOf(NioDatagramChannel.class);
+ assertThat(sdkEventLoopGroup.eventLoopGroup()).isNotNull();
+ }
+
+ @Test
+ public void creatingUsingStaticMethod_B() {
+ SdkEventLoopGroup sdkEventLoopGroup = SdkEventLoopGroup.create(new OioEventLoopGroup(), OioSocketChannel::new);
+ assertThat(sdkEventLoopGroup.channelFactory()).isNotNull();
+ assertThat(sdkEventLoopGroup.datagramChannelFactory()).isNotNull();
+ assertThat(sdkEventLoopGroup.datagramChannelFactory().newChannel()).isInstanceOf(OioDatagramChannel.class);
assertThat(sdkEventLoopGroup.eventLoopGroup()).isNotNull();
}
@@ -43,6 +61,7 @@ public void notProvidingChannelFactory_channelFactoryResolved() {
SdkEventLoopGroup sdkEventLoopGroup = SdkEventLoopGroup.create(new NioEventLoopGroup());
assertThat(sdkEventLoopGroup.channelFactory()).isNotNull();
+ assertThat(sdkEventLoopGroup.datagramChannelFactory().newChannel()).isInstanceOf(NioDatagramChannel.class);
}
@Test(expected = IllegalArgumentException.class)
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMapTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMapTest.java
index 3b72f71be4db..17289d1ca3b3 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMapTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/AwaitCloseChannelPoolMapTest.java
@@ -118,7 +118,7 @@ public void get_callsInjectedBootstrapProviderCorrectly() {
channelPoolMap = new AwaitCloseChannelPoolMap(builder, null, bootstrapProvider);
channelPoolMap.get(targetUri);
- verify(bootstrapProvider).createBootstrap("some-awesome-service-1234.amazonaws.com", 8080);
+ verify(bootstrapProvider).createBootstrap("some-awesome-service-1234.amazonaws.com", 8080, null);
}
@Test
@@ -151,7 +151,7 @@ public void get_usingProxy_callsInjectedBootstrapProviderCorrectly() {
channelPoolMap = new AwaitCloseChannelPoolMap(builder, shouldProxyCache, bootstrapProvider);
channelPoolMap.get(targetUri);
- verify(bootstrapProvider).createBootstrap("localhost", mockProxy.port());
+ verify(bootstrapProvider).createBootstrap("localhost", mockProxy.port(), null);
}
@Test
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/BootstrapProviderTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/BootstrapProviderTest.java
index 337cb7ba2ec2..914587b85df3 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/BootstrapProviderTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/BootstrapProviderTest.java
@@ -42,7 +42,19 @@ public class BootstrapProviderTest {
// connection attempt and not cached between connection attempts.
@Test
public void createBootstrap_usesUnresolvedInetSocketAddress() {
- Bootstrap bootstrap = bootstrapProvider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443);
+ Bootstrap bootstrap = bootstrapProvider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443, false);
+
+ SocketAddress socketAddress = bootstrap.config().remoteAddress();
+
+ assertThat(socketAddress).isInstanceOf(InetSocketAddress.class);
+ InetSocketAddress inetSocketAddress = (InetSocketAddress)socketAddress;
+
+ assertThat(inetSocketAddress.isUnresolved()).isTrue();
+ }
+
+ @Test
+ public void createBootstrapNonBlockingDns_usesUnresolvedInetSocketAddress() {
+ Bootstrap bootstrap = bootstrapProvider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443, true);
SocketAddress socketAddress = bootstrap.config().remoteAddress();
@@ -54,7 +66,7 @@ public void createBootstrap_usesUnresolvedInetSocketAddress() {
@Test
public void createBootstrap_defaultConfiguration_tcpKeepAliveShouldBeFalse() {
- Bootstrap bootstrap = bootstrapProvider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443);
+ Bootstrap bootstrap = bootstrapProvider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443, false);
Boolean keepAlive = (Boolean) bootstrap.config().options().get(ChannelOption.SO_KEEPALIVE);
assertThat(keepAlive).isFalse();
@@ -70,7 +82,7 @@ public void createBootstrap_tcpKeepAliveTrue_shouldApply() {
nettyConfiguration,
new SdkChannelOptions());
- Bootstrap bootstrap = provider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443);
+ Bootstrap bootstrap = provider.createBootstrap("some-awesome-service-1234.amazonaws.com", 443, false);
Boolean keepAlive = (Boolean) bootstrap.config().options().get(ChannelOption.SO_KEEPALIVE);
assertThat(keepAlive).isTrue();
}
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/DnsResolverLoaderTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/DnsResolverLoaderTest.java
new file mode 100644
index 000000000000..40db804aacaf
--- /dev/null
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/DnsResolverLoaderTest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.http.nio.netty.internal;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import io.netty.channel.epoll.EpollDatagramChannel;
+import io.netty.channel.socket.nio.NioDatagramChannel;
+import io.netty.channel.socket.oio.OioDatagramChannel;
+import io.netty.resolver.dns.DnsAddressResolverGroup;
+import org.junit.jupiter.api.Test;
+
+public class DnsResolverLoaderTest {
+
+ @Test
+ public void canResolveChannelFactory() {
+ assertThat(DnsResolverLoader.init(NioDatagramChannel::new)).isInstanceOf(DnsAddressResolverGroup.class);
+ assertThat(DnsResolverLoader.init(EpollDatagramChannel::new)).isInstanceOf(DnsAddressResolverGroup.class);
+ assertThat(DnsResolverLoader.init(OioDatagramChannel::new)).isInstanceOf(DnsAddressResolverGroup.class);
+ }
+}
diff --git a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/utils/SocketChannelResolverTest.java b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/utils/ChannelResolverTest.java
similarity index 70%
rename from http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/utils/SocketChannelResolverTest.java
rename to http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/utils/ChannelResolverTest.java
index 472c417d4485..45edd2b81bb1 100644
--- a/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/utils/SocketChannelResolverTest.java
+++ b/http-clients/netty-nio-client/src/test/java/software/amazon/awssdk/http/nio/netty/internal/utils/ChannelResolverTest.java
@@ -16,39 +16,47 @@
package software.amazon.awssdk.http.nio.netty.internal.utils;
import static org.assertj.core.api.Assertions.assertThat;
-import static software.amazon.awssdk.http.nio.netty.internal.utils.SocketChannelResolver.resolveSocketChannelFactory;
+import static software.amazon.awssdk.http.nio.netty.internal.utils.ChannelResolver.resolveDatagramChannelFactory;
+import static software.amazon.awssdk.http.nio.netty.internal.utils.ChannelResolver.resolveSocketChannelFactory;
import io.netty.channel.epoll.Epoll;
+import io.netty.channel.epoll.EpollDatagramChannel;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.epoll.EpollSocketChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.oio.OioEventLoopGroup;
+import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.channel.socket.oio.OioDatagramChannel;
import io.netty.channel.socket.oio.OioSocketChannel;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.Test;
import software.amazon.awssdk.http.nio.netty.internal.DelegatingEventLoopGroup;
-public class SocketChannelResolverTest {
+public class ChannelResolverTest {
@Test
public void canDetectFactoryForStandardNioEventLoopGroup() {
assertThat(resolveSocketChannelFactory(new NioEventLoopGroup()).newChannel()).isInstanceOf(NioSocketChannel.class);
+ assertThat(resolveDatagramChannelFactory(new NioEventLoopGroup()).newChannel()).isInstanceOf(NioDatagramChannel.class);
}
@Test
public void canDetectEpollEventLoopGroupFactory() {
Assumptions.assumeTrue(Epoll.isAvailable());
assertThat(resolveSocketChannelFactory(new EpollEventLoopGroup()).newChannel()).isInstanceOf(EpollSocketChannel.class);
+ assertThat(resolveDatagramChannelFactory(new EpollEventLoopGroup()).newChannel()).isInstanceOf(EpollDatagramChannel.class);
}
@Test
public void worksWithDelegateEventLoopGroupsFactory() {
assertThat(resolveSocketChannelFactory(new DelegatingEventLoopGroup(new NioEventLoopGroup()) {}).newChannel()).isInstanceOf(NioSocketChannel.class);
+ assertThat(resolveDatagramChannelFactory(new DelegatingEventLoopGroup(new NioEventLoopGroup()) {}).newChannel()).isInstanceOf(NioDatagramChannel.class);
}
@Test
public void worksWithOioEventLoopGroupFactory() {
assertThat(resolveSocketChannelFactory(new OioEventLoopGroup()).newChannel()).isInstanceOf(OioSocketChannel.class);
+ assertThat(resolveDatagramChannelFactory(new OioEventLoopGroup()).newChannel()).isInstanceOf(OioDatagramChannel.class);
}
}
diff --git a/http-clients/pom.xml b/http-clients/pom.xml
index 7c5d32540aac..bbd349e6e80d 100644
--- a/http-clients/pom.xml
+++ b/http-clients/pom.xml
@@ -21,7 +21,7 @@
aws-sdk-java-pom
software.amazon.awssdk
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
4.0.0
diff --git a/http-clients/url-connection-client/pom.xml b/http-clients/url-connection-client/pom.xml
index afedc11653ef..60e61061fbbe 100644
--- a/http-clients/url-connection-client/pom.xml
+++ b/http-clients/url-connection-client/pom.xml
@@ -20,7 +20,7 @@
http-clients
software.amazon.awssdk
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
4.0.0
diff --git a/metric-publishers/cloudwatch-metric-publisher/pom.xml b/metric-publishers/cloudwatch-metric-publisher/pom.xml
index d19d3a34950d..9bf10b0a07df 100644
--- a/metric-publishers/cloudwatch-metric-publisher/pom.xml
+++ b/metric-publishers/cloudwatch-metric-publisher/pom.xml
@@ -17,7 +17,7 @@
software.amazon.awssdk
metric-publishers
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
cloudwatch-metric-publisher
diff --git a/metric-publishers/pom.xml b/metric-publishers/pom.xml
index 547bd7ae2f00..93d3eafa592b 100644
--- a/metric-publishers/pom.xml
+++ b/metric-publishers/pom.xml
@@ -17,7 +17,7 @@
software.amazon.awssdk
aws-sdk-java-pom
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
metric-publishers
diff --git a/pom.xml b/pom.xml
index 25f092c33ac7..4e094e34898d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -20,7 +20,7 @@
4.0.0
software.amazon.awssdk
aws-sdk-java-pom
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
pom
AWS Java SDK :: Parent
The Amazon Web Services SDK for Java provides Java APIs
@@ -90,7 +90,7 @@
${project.version}
- 2.20.82
+ 2.20.89
2.13.2
2.13.4.2
2.13.2
diff --git a/release-scripts/pom.xml b/release-scripts/pom.xml
index 28abfade104e..bb792f0e5e61 100644
--- a/release-scripts/pom.xml
+++ b/release-scripts/pom.xml
@@ -22,7 +22,7 @@
software.amazon.awssdk
aws-sdk-java-pom
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
../pom.xml
release-scripts
diff --git a/services-custom/dynamodb-enhanced/pom.xml b/services-custom/dynamodb-enhanced/pom.xml
index 20b7b81a4f50..7c0e236f6f9a 100644
--- a/services-custom/dynamodb-enhanced/pom.xml
+++ b/services-custom/dynamodb-enhanced/pom.xml
@@ -21,7 +21,7 @@
software.amazon.awssdk
services-custom
- 2.20.83-SNAPSHOT
+ 2.20.90-SNAPSHOT
dynamodb-enhanced
AWS Java SDK :: DynamoDB :: Enhanced Client
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/DefaultAttributeConverterProvider.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/DefaultAttributeConverterProvider.java
index 45db89f5283c..88cfbe39e82f 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/DefaultAttributeConverterProvider.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/DefaultAttributeConverterProvider.java
@@ -43,7 +43,6 @@
import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.DocumentAttributeConverter;
import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.DoubleAttributeConverter;
import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.DurationAttributeConverter;
-import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.EnumAttributeConverter;
import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.FloatAttributeConverter;
import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.InstantAsStringAttributeConverter;
import software.amazon.awssdk.enhanced.dynamodb.internal.converter.attribute.IntegerAttributeConverter;
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/EnumAttributeConverter.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/EnumAttributeConverter.java
new file mode 100644
index 000000000000..a44a5e2070f0
--- /dev/null
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/EnumAttributeConverter.java
@@ -0,0 +1,138 @@
+/*
+ * 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.enhanced.dynamodb;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.function.Function;
+import software.amazon.awssdk.annotations.SdkPublicApi;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.utils.Validate;
+
+/**
+ * A converter between an {@link Enum} and {@link AttributeValue}.
+ *
+ *
+ * This stores values in DynamoDB as a string.
+ *
+ *
+ * Use EnumAttributeConverter::create in order to use Enum::toString as the enum identifier
+ *
+ *
+ * Use EnumAttributeConverter::createWithNameAsKeys in order to use Enum::name as the enum identifier
+ *
+ *
+ * This can be created via {@link #create(Class)}.
+ */
+@SdkPublicApi
+public final class EnumAttributeConverter> implements AttributeConverter {
+
+ private final Class enumClass;
+ private final Map enumValueMap;
+
+ private final Function keyExtractor;
+
+ private EnumAttributeConverter(Class enumClass, Function keyExtractor) {
+ this.enumClass = enumClass;
+ this.keyExtractor = keyExtractor;
+
+ Map mutableEnumValueMap = new LinkedHashMap<>();
+ Arrays.stream(enumClass.getEnumConstants())
+ .forEach(enumConstant -> mutableEnumValueMap.put(keyExtractor.apply(enumConstant), enumConstant));
+
+ this.enumValueMap = Collections.unmodifiableMap(mutableEnumValueMap);
+ }
+
+ /**
+ * Creates an EnumAttributeConverter for an {@link Enum}.
+ *
+ *
+ * Uses Enum::toString as the enum identifier.
+ *
+ * @param enumClass The enum class to be used
+ * @return an EnumAttributeConverter
+ * @param the enum subclass
+ */
+ public static > EnumAttributeConverter create(Class enumClass) {
+ return new EnumAttributeConverter<>(enumClass, Enum::toString);
+ }
+
+ /**
+ * Creates an EnumAttributeConverter for an {@link Enum}.
+ *
+ *
+ * Uses Enum::name as the enum identifier.
+ *
+ * @param enumClass The enum class to be used
+ * @return an EnumAttributeConverter
+ * @param the enum subclass
+ */
+ public static > EnumAttributeConverter createWithNameAsKeys(Class enumClass) {
+ return new EnumAttributeConverter<>(enumClass, Enum::name);
+ }
+
+ /**
+ * Returns the proper {@link AttributeValue} for the given enum type.
+ *
+ * @param input the enum type to be converted
+ * @return AttributeValue
+ */
+ @Override
+ public AttributeValue transformFrom(T input) {
+ return AttributeValue.builder().s(keyExtractor.apply(input)).build();
+ }
+
+ /**
+ * Returns the proper enum type for the given {@link AttributeValue} input.
+ *
+ * @param input the AttributeValue to be converted
+ * @return an enum type
+ */
+ @Override
+ public T transformTo(AttributeValue input) {
+ Validate.isTrue(input.s() != null, "Cannot convert non-string value to enum.");
+ T returnValue = enumValueMap.get(input.s());
+
+ if (returnValue == null) {
+ throw new IllegalArgumentException(String.format("Unable to convert string value '%s' to enum type '%s'",
+ input.s(), enumClass));
+ }
+
+ return returnValue;
+ }
+
+ /**
+ * Returns the {@link EnhancedType} of the converter.
+ *
+ * @return EnhancedType
+ */
+ @Override
+ public EnhancedType type() {
+ return EnhancedType.of(enumClass);
+ }
+
+ /**
+ * Returns the {@link AttributeValueType} of the converter.
+ *
+ * @return AttributeValueType
+ */
+ @Override
+ public AttributeValueType attributeValueType() {
+ return AttributeValueType.S;
+ }
+}
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/TableSchema.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/TableSchema.java
index bdf4ee35cbdb..2aa9d100d2c2 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/TableSchema.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/TableSchema.java
@@ -53,6 +53,18 @@ static StaticTableSchema.Builder builder(Class itemClass) {
return StaticTableSchema.builder(itemClass);
}
+ /**
+ * Returns a builder for the {@link StaticTableSchema} implementation of this interface which allows all attributes,
+ * tags and table structure to be directly declared in the builder.
+ *
+ * @param itemType The {@link EnhancedType} of the item this {@link TableSchema} will map records to.
+ * @param The type of the item this {@link TableSchema} will map records to.
+ * @return A newly initialized {@link StaticTableSchema.Builder}.
+ */
+ static StaticTableSchema.Builder builder(EnhancedType itemType) {
+ return StaticTableSchema.builder(itemType);
+ }
+
/**
* Returns a builder for the {@link StaticImmutableTableSchema} implementation of this interface which allows all
* attributes, tags and table structure to be directly declared in the builder.
@@ -69,6 +81,22 @@ static StaticImmutableTableSchema.Builder builder(Class immutabl
return StaticImmutableTableSchema.builder(immutableItemClass, immutableBuilderClass);
}
+ /**
+ * Returns a builder for the {@link StaticImmutableTableSchema} implementation of this interface which allows all
+ * attributes, tags and table structure to be directly declared in the builder.
+ *
+ * @param immutableItemType The {@link EnhancedType} of the immutable item this {@link TableSchema} will map records to.
+ * @param immutableBuilderType The {@link EnhancedType} of the class that can be used to construct immutable items this
+ * {@link TableSchema} maps records to.
+ * @param The type of the immutable item this {@link TableSchema} will map records to.
+ * @param The type of the builder used by this {@link TableSchema} to construct immutable items with.
+ * @return A newly initialized {@link StaticImmutableTableSchema.Builder}
+ */
+ static StaticImmutableTableSchema.Builder builder(EnhancedType immutableItemType,
+ EnhancedType immutableBuilderType) {
+ return StaticImmutableTableSchema.builder(immutableItemType, immutableBuilderType);
+ }
+
/**
* Scans a bean class that has been annotated with DynamoDb bean annotations and then returns a
* {@link BeanTableSchema} implementation of this interface that can map records to and from items of that bean
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/DefaultDynamoDbTable.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/DefaultDynamoDbTable.java
index 6c9f0f69265e..8b6d8412969b 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/DefaultDynamoDbTable.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/DefaultDynamoDbTable.java
@@ -15,13 +15,20 @@
package software.amazon.awssdk.enhanced.dynamodb.internal.client;
+import static java.util.Collections.emptyList;
import static software.amazon.awssdk.enhanced.dynamodb.internal.EnhancedClientUtils.createKeyFromItem;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
import java.util.function.Consumer;
+import java.util.stream.Collectors;
import software.amazon.awssdk.annotations.SdkInternalApi;
import software.amazon.awssdk.enhanced.dynamodb.DynamoDbEnhancedClientExtension;
import software.amazon.awssdk.enhanced.dynamodb.DynamoDbTable;
+import software.amazon.awssdk.enhanced.dynamodb.IndexMetadata;
import software.amazon.awssdk.enhanced.dynamodb.Key;
+import software.amazon.awssdk.enhanced.dynamodb.KeyAttributeMetadata;
import software.amazon.awssdk.enhanced.dynamodb.TableMetadata;
import software.amazon.awssdk.enhanced.dynamodb.TableSchema;
import software.amazon.awssdk.enhanced.dynamodb.internal.operations.CreateTableOperation;
@@ -39,6 +46,8 @@
import software.amazon.awssdk.enhanced.dynamodb.model.DeleteItemEnhancedRequest;
import software.amazon.awssdk.enhanced.dynamodb.model.DeleteItemEnhancedResponse;
import software.amazon.awssdk.enhanced.dynamodb.model.DescribeTableEnhancedResponse;
+import software.amazon.awssdk.enhanced.dynamodb.model.EnhancedGlobalSecondaryIndex;
+import software.amazon.awssdk.enhanced.dynamodb.model.EnhancedLocalSecondaryIndex;
import software.amazon.awssdk.enhanced.dynamodb.model.GetItemEnhancedRequest;
import software.amazon.awssdk.enhanced.dynamodb.model.PageIterable;
import software.amazon.awssdk.enhanced.dynamodb.model.PutItemEnhancedRequest;
@@ -51,6 +60,7 @@
import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest;
import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse;
+import software.amazon.awssdk.services.dynamodb.model.ProjectionType;
@SdkInternalApi
public class DefaultDynamoDbTable implements DynamoDbTable {
@@ -115,7 +125,51 @@ public void createTable(Consumer requestCons
@Override
public void createTable() {
- createTable(CreateTableEnhancedRequest.builder().build());
+ Map> indexGroups = splitSecondaryIndicesToLocalAndGlobalOnes();
+ createTable(CreateTableEnhancedRequest.builder()
+ .localSecondaryIndices(extractLocalSecondaryIndices(indexGroups))
+ .globalSecondaryIndices(extractGlobalSecondaryIndices(indexGroups))
+ .build());
+ }
+
+ private Map> splitSecondaryIndicesToLocalAndGlobalOnes() {
+ String primaryPartitionKeyName = tableSchema.tableMetadata().primaryPartitionKey();
+ Collection indices = tableSchema.tableMetadata().indices();
+ return indices.stream()
+ .filter(index -> !TableMetadata.primaryIndexName().equals(index.name()))
+ .collect(Collectors.groupingBy(metadata -> {
+ String partitionKeyName = metadata.partitionKey().map(KeyAttributeMetadata::name).orElse(null);
+ if (partitionKeyName == null || primaryPartitionKeyName.equals(partitionKeyName)) {
+ return IndexType.LSI;
+ }
+ return IndexType.GSI;
+ }));
+ }
+
+ private List extractLocalSecondaryIndices(Map> indicesGroups) {
+ return indicesGroups.getOrDefault(IndexType.LSI, emptyList()).stream()
+ .map(this::mapIndexMetadataToEnhancedLocalSecondaryIndex)
+ .collect(Collectors.toList());
+ }
+
+ private EnhancedLocalSecondaryIndex mapIndexMetadataToEnhancedLocalSecondaryIndex(IndexMetadata indexMetadata) {
+ return EnhancedLocalSecondaryIndex.builder()
+ .indexName(indexMetadata.name())
+ .projection(pb -> pb.projectionType(ProjectionType.ALL))
+ .build();
+ }
+
+ private List extractGlobalSecondaryIndices(Map> indicesGroups) {
+ return indicesGroups.getOrDefault(IndexType.GSI, emptyList()).stream()
+ .map(this::mapIndexMetadataToEnhancedGlobalSecondaryIndex)
+ .collect(Collectors.toList());
+ }
+
+ private EnhancedGlobalSecondaryIndex mapIndexMetadataToEnhancedGlobalSecondaryIndex(IndexMetadata indexMetadata) {
+ return EnhancedGlobalSecondaryIndex.builder()
+ .indexName(indexMetadata.name())
+ .projection(pb -> pb.projectionType(ProjectionType.ALL))
+ .build();
}
@Override
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/IndexType.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/IndexType.java
new file mode 100644
index 000000000000..0fd1fc28cd82
--- /dev/null
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/client/IndexType.java
@@ -0,0 +1,27 @@
+/*
+ * 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.enhanced.dynamodb.internal.client;
+
+import software.amazon.awssdk.annotations.SdkInternalApi;
+
+/**
+ * Enum collecting types of secondary indexes
+ */
+@SdkInternalApi
+public enum IndexType {
+ LSI,
+ GSI
+}
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/converter/attribute/EnumAttributeConverter.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/converter/attribute/EnumAttributeConverter.java
deleted file mode 100644
index 18395a82656b..000000000000
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/converter/attribute/EnumAttributeConverter.java
+++ /dev/null
@@ -1,85 +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.enhanced.dynamodb.internal.converter.attribute;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import software.amazon.awssdk.annotations.SdkInternalApi;
-import software.amazon.awssdk.enhanced.dynamodb.AttributeConverter;
-import software.amazon.awssdk.enhanced.dynamodb.AttributeValueType;
-import software.amazon.awssdk.enhanced.dynamodb.EnhancedType;
-import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
-import software.amazon.awssdk.utils.Validate;
-
-/**
- * A converter between an {@link Enum} and {@link AttributeValue}.
- *
- *
- * This stores values in DynamoDB as a string.
- *
- *
- * This can be created via {@link #create(Class)}.
- */
-@SdkInternalApi
-public class EnumAttributeConverter> implements AttributeConverter {
-
- private final Class enumClass;
- private final Map enumValueMap;
-
- private EnumAttributeConverter(Class enumClass) {
- this.enumClass = enumClass;
-
- Map mutableEnumValueMap = new LinkedHashMap<>();
- Arrays.stream(enumClass.getEnumConstants())
- .forEach(enumConstant -> mutableEnumValueMap.put(enumConstant.toString(), enumConstant));
-
- this.enumValueMap = Collections.unmodifiableMap(mutableEnumValueMap);
- }
-
- public static > EnumAttributeConverter create(Class enumClass) {
- return new EnumAttributeConverter<>(enumClass);
- }
-
- @Override
- public AttributeValue transformFrom(T input) {
- return AttributeValue.builder().s(input.toString()).build();
- }
-
- @Override
- public T transformTo(AttributeValue input) {
- Validate.isTrue(input.s() != null, "Cannot convert non-string value to enum.");
- T returnValue = enumValueMap.get(input.s());
-
- if (returnValue == null) {
- throw new IllegalArgumentException(String.format("Unable to convert string value '%s' to enum type '%s'",
- input.s(), enumClass));
- }
-
- return returnValue;
- }
-
- @Override
- public EnhancedType type() {
- return EnhancedType.of(enumClass);
- }
-
- @Override
- public AttributeValueType attributeValueType() {
- return AttributeValueType.S;
- }
-}
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/operations/CreateTableOperation.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/operations/CreateTableOperation.java
index 24a144ae08b8..de3887081515 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/operations/CreateTableOperation.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/internal/operations/CreateTableOperation.java
@@ -74,7 +74,7 @@ public CreateTableRequest generateRequest(TableSchema tableSchema,
List sdkGlobalSecondaryIndices = null;
List sdkLocalSecondaryIndices = null;
- if (this.request.globalSecondaryIndices() != null) {
+ if (this.request.globalSecondaryIndices() != null && !this.request.globalSecondaryIndices().isEmpty()) {
sdkGlobalSecondaryIndices =
this.request.globalSecondaryIndices().stream().map(gsi -> {
String indexPartitionKey = tableSchema.tableMetadata().indexPartitionKey(gsi.indexName());
@@ -92,7 +92,7 @@ public CreateTableRequest generateRequest(TableSchema tableSchema,
}).collect(Collectors.toList());
}
- if (this.request.localSecondaryIndices() != null) {
+ if (this.request.localSecondaryIndices() != null && !this.request.localSecondaryIndices().isEmpty()) {
sdkLocalSecondaryIndices =
this.request.localSecondaryIndices().stream().map(lsi -> {
Optional indexSortKey = tableSchema.tableMetadata().indexSortKey(lsi.indexName());
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/ImmutableAttribute.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/ImmutableAttribute.java
index d22f87af61a9..d5622bfc6df6 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/ImmutableAttribute.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/ImmutableAttribute.java
@@ -91,6 +91,19 @@ public static Builder builder(Class itemClass,
return new Builder<>(attributeType);
}
+ /**
+ * Constructs a new builder for this class using supplied types.
+ * @param itemType The {@link EnhancedType} of the immutable item that this attribute composes.
+ * @param builderType The {@link EnhancedType} of the builder for the immutable item that this attribute composes.
+ * @param attributeType A {@link EnhancedType} that represents the type of the value this attribute stores.
+ * @return A new typed builder for an attribute.
+ */
+ public static Builder builder(EnhancedType itemType,
+ EnhancedType builderType,
+ EnhancedType attributeType) {
+ return new Builder<>(attributeType);
+ }
+
/**
* Constructs a new builder for this class using supplied types.
* @param itemClass The class of the item that this attribute composes.
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticAttribute.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticAttribute.java
index 2957311d7417..5071869347c8 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticAttribute.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticAttribute.java
@@ -69,7 +69,17 @@ private StaticAttribute(Builder builder) {
* @return A new typed builder for an attribute.
*/
public static Builder builder(Class itemClass, EnhancedType attributeType) {
- return new Builder<>(itemClass, attributeType);
+ return new Builder<>(EnhancedType.of(itemClass), attributeType);
+ }
+
+ /**
+ * Constructs a new builder for this class using supplied types.
+ * @param itemType The {@link EnhancedType} of the item that this attribute composes.
+ * @param attributeType A {@link EnhancedType} that represents the type of the value this attribute stores.
+ * @return A new typed builder for an attribute.
+ */
+ public static Builder builder(EnhancedType itemType, EnhancedType attributeType) {
+ return new Builder<>(itemType, attributeType);
}
/**
@@ -79,7 +89,7 @@ public static Builder builder(Class itemClass, EnhancedType a
* @return A new typed builder for an attribute.
*/
public static Builder builder(Class itemClass, Class attributeClass) {
- return new Builder<>(itemClass, EnhancedType.of(attributeClass));
+ return new Builder<>(EnhancedType.of(itemClass), EnhancedType.of(attributeClass));
}
/**
@@ -146,8 +156,8 @@ ImmutableAttribute toImmutableAttribute() {
public static final class Builder {
private final ImmutableAttribute.Builder delegateBuilder;
- private Builder(Class itemClass, EnhancedType type) {
- this.delegateBuilder = ImmutableAttribute.builder(itemClass, itemClass, type);
+ private Builder(EnhancedType itemType, EnhancedType type) {
+ this.delegateBuilder = ImmutableAttribute.builder(itemType, itemType, type);
}
private Builder(ImmutableAttribute.Builder delegateBuilder) {
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticImmutableTableSchema.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticImmutableTableSchema.java
index 5d08ee4a3ae3..ea86ac9fcec4 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticImmutableTableSchema.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticImmutableTableSchema.java
@@ -210,7 +210,7 @@ private StaticImmutableTableSchema(Builder builder) {
this.newBuilderSupplier = builder.newBuilderSupplier;
this.buildItemFunction = builder.buildItemFunction;
this.tableMetadata = tableMetadataBuilder.build();
- this.itemType = EnhancedType.of(builder.itemClass);
+ this.itemType = builder.itemType;
}
/**
@@ -220,7 +220,18 @@ private StaticImmutableTableSchema(Builder builder) {
* @return A newly initialized builder
*/
public static Builder builder(Class itemClass, Class builderClass) {
- return new Builder<>(itemClass, builderClass);
+ return new Builder<>(EnhancedType.of(itemClass), EnhancedType.of(builderClass));
+ }
+
+ /**
+ * Creates a builder for a {@link StaticImmutableTableSchema} typed to specific immutable data item class.
+ * @param itemType The {@link EnhancedType} of the immutable data item class object that the
+ * {@link StaticImmutableTableSchema} is to map to.
+ * @param builderType The builder {@link EnhancedType} that can be used to construct instances of the immutable data item.
+ * @return A newly initialized builder
+ */
+ public static Builder builder(EnhancedType itemType, EnhancedType builderType) {
+ return new Builder<>(itemType, builderType);
}
/**
@@ -230,8 +241,8 @@ public static Builder builder(Class itemClass, Class builderC
*/
@NotThreadSafe
public static final class Builder {
- private final Class itemClass;
- private final Class builderClass;
+ private final EnhancedType itemType;
+ private final EnhancedType builderType;
private final List> additionalAttributes = new ArrayList<>();
private final List> flattenedMappers = new ArrayList<>();
@@ -242,9 +253,9 @@ public static final class Builder {
private List attributeConverterProviders =
Collections.singletonList(ConverterProviderResolver.defaultConverterProvider());
- private Builder(Class itemClass, Class builderClass) {
- this.itemClass = itemClass;
- this.builderClass = builderClass;
+ private Builder(EnhancedType itemType, EnhancedType builderType) {
+ this.itemType = itemType;
+ this.builderType = builderType;
}
/**
@@ -285,7 +296,7 @@ public Builder addAttribute(EnhancedType attributeType,
Consumer> immutableAttribute) {
ImmutableAttribute.Builder builder =
- ImmutableAttribute.builder(itemClass, builderClass, attributeType);
+ ImmutableAttribute.builder(itemType, builderType, attributeType);
immutableAttribute.accept(builder);
return addAttribute(builder.build());
}
diff --git a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticTableSchema.java b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticTableSchema.java
index 5d8dbfd94b76..6dc6b2d4f211 100644
--- a/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticTableSchema.java
+++ b/services-custom/dynamodb-enhanced/src/main/java/software/amazon/awssdk/enhanced/dynamodb/mapper/StaticTableSchema.java
@@ -75,7 +75,16 @@ private StaticTableSchema(Builder builder) {
* @return A newly initialized builder
*/
public static Builder builder(Class itemClass) {
- return new Builder<>(itemClass);
+ return new Builder<>(EnhancedType.of(itemClass));
+ }
+
+ /**
+ * Creates a builder for a {@link StaticTableSchema} typed to specific data item class.
+ * @param itemType The {@link EnhancedType} of the data item class object that the {@link StaticTableSchema} is to map to.
+ * @return A newly initialized builder
+ */
+ public static Builder builder(EnhancedType itemType) {
+ return new Builder<>(itemType);
}
/**
@@ -85,11 +94,11 @@ public static Builder builder(Class itemClass) {
@NotThreadSafe
public static final class Builder {
private final StaticImmutableTableSchema.Builder delegateBuilder;
- private final Class itemClass;
+ private final EnhancedType