diff --git a/docs/src/main/asciidoc/images/kinesis-binder.png b/docs/src/main/asciidoc/images/kinesis-binder.png
new file mode 100644
index 000000000..8ee54f1f5
Binary files /dev/null and b/docs/src/main/asciidoc/images/kinesis-binder.png differ
diff --git a/docs/src/main/asciidoc/index.adoc b/docs/src/main/asciidoc/index.adoc
index 95df4eb93..5adaf4055 100644
--- a/docs/src/main/asciidoc/index.adoc
+++ b/docs/src/main/asciidoc/index.adoc
@@ -160,6 +160,8 @@ include::cloudwatch.adoc[]
include::spring-modulith.adoc[]
+include::kinesis-stream-binder.adoc[]
+
include::testing.adoc[]
include::docker-compose.adoc[]
diff --git a/docs/src/main/asciidoc/kinesis-stream-binder.adoc b/docs/src/main/asciidoc/kinesis-stream-binder.adoc
new file mode 100644
index 000000000..54605f5f2
--- /dev/null
+++ b/docs/src/main/asciidoc/kinesis-stream-binder.adoc
@@ -0,0 +1,520 @@
+[#spring-cloud-aws-kinesis-stream-binder]
+== Kinesis Binder for Spring Cloud Stream
+
+This guide describes the https://aws.amazon.com/kinesis/[AWS Kinesis] implementation of the Spring Cloud Stream Binder.
+It contains information about its design, usage and configuration options, as well as information on how the Stream Cloud Stream concepts map into AWS Kinesis specific constructs.
+
+== Usage
+
+For using Spring Cloud Stream with AWS Kinesis, the dedicated dependency for Kinesis Binder has to be added to the target project:
+
+[source,xml]
+----
+
+ io.awspring.cloud
+ spring-cloud-aws-kinesis-stream-binder
+
+----
+
+== Kinesis Binder Overview
+
+The Spring Cloud Stream Binder for AWS Kinesis provides the binding implementation for the Spring Cloud Stream.
+This implementation uses Spring Integration AWS Kinesis Channel Adapters at its foundation.
+The following captures how the Kinesis Binder implementation maps each of the configured destinations to AWS Kinesis Streams:
+
+.Kinesis Binder
+image::images/kinesis-binder.png[width=300,scaledwidth="50%"]
+
+Unlike https://kafka.apache.org/[Apache Kafka], the AWS Kinesis doesn't provide out-of-the-box support for consumer groups.
+The support of this feature is implemented as a part of `MetadataStore` key for shard checkpoints in the `KinesisMessageDrivenChannelAdapter` - `[CONSUMER_GROUP]:[STREAM]:[SHARD_ID]`.
+In addition, the `LockRegistry` is used to ensure exclusive access to each shard.
+This way only one channel adapter in the same consumer group will consumer messages from a single shard in the stream it is configured for.
+
+The partitioning logic in AWS Kinesis is similar to the Apache Kafka support but with slightly different logic.
+The `partitionKey` on the producer side determines which shard in the stream the data record is assigned to.
+Partition keys are Unicode strings with a maximum length limit of 256 characters for each key.
+AWS Kinesis uses the partition key as input to a hash function that maps the partition key and associated data to a specific shard.
+Specifically, an MD5 hash function is used to map partition keys to 128-bit integer values and to map associated data records to shards.
+As a result of this hashing mechanism, all data records with the same partition key map to the same shard within the stream.
+But at the same time we can't select a target shard to send explicitly.
+Although calculating the hash manually (and use `explicitHashKeyExpression` for producer, respectively), we may track the target shard by inclusion into its `HashKeyRange`.
+
+By default, a partition key is a result of the `Object.hash()` from the message `payload`.
+
+The Spring Cloud Stream partition handling logic is excluded in case of AWS Kinesis Binder since it is out of use and the provided `producer.partitionKeyExpression` is propagated to the `KinesisMessageHandler` directly.
+
+On the consumer side the `instanceCount` and `instanceIndex` are used to distribute shards between consumers in a group evenly.
+This has an effect only for regular `KinesisMessageDrivenChannelAdapter` which can assign specific shards for the target Kinesis consumer.
+
+== Consumer Groups
+Consumer groups are implemented with a focus on high availability, message ordering and guaranteed message delivery in Spring cloud stream.
+A `single consumer` for the message is ensured by https://docs.spring.io/spring-cloud-stream/reference/spring-cloud-stream/consumer-groups.html[consumer group abstraction].
+
+To have a highly available consumer group for your kinesis stream:
+
+- Ensure all instances of your consumer applications use a shared `DynamoDbMetadataStore` and `DynamoDbLockRegistry` (See below for configuration options).
+- Use same group name for the channel in all application instances by using property `spring.cloud.stream.bindings..group`.
+
+These configurations alone guarantee HA, message ordering and guaranteed message delivery.
+However, even distribution across instances is not guaranteed as of now.
+There is a very high chance that a single instance in a consumer group will pick up all the shards for consuming.
+But, when that instance goes down (couldn't send heartbeat for any reason), another instance in the consumer group will start processing from the last checkpoint of the previous consumer (for shardIterator type `TRIM_HORIZON`).
+
+So, configuring consumer concurrency is important to achieve throughput.
+It can be configured using `spring.cloud.stream.bindings..consumer.concurrency`.
+
+=== Static shard distribution within a single consumer group
+It is possible to evenly distribute shard across all instances within a single consumer group.
+This is done by configuring:
+
+- `spring.cloud.stream.instanceCount=` to number of instances
+- `spring.cloud.stream.instanceIndex=` current instance's index
+
+The only way to achieve HA in this case is that, when an instance processing a particular shard goes down, another instance must have `spring.cloud.stream.instanceIndex=` to be the same as the failed instance's index to start processing from those shards.
+
+== Configuration Options
+
+This section contains settings specific to the Kinesis Binder and bound channels.
+
+For general binding configuration options and properties, please refer to the https://docs.spring.io/spring-cloud-stream/reference/spring-cloud-stream/configuration-options.html[Spring Cloud Stream core documentation].
+
+Also, Kinesis Binder exposes `KinesisAsyncClient`, `DynamoDbAsyncClient`, and `CloudWatchAsyncClient` beans into its child application context.
+By default, these beans rely on the auto-configuration from Spring Cloud AWS.
+See `CredentialsProperties`, `RegionProperties`, and `AwsClientProperties` extensions for more information.
+
+[[kinesis-binder-properties]]
+=== Kinesis Binder Properties
+
+The following properties are available for Kinesis Binder configuration, which start with the `spring.cloud.stream.kinesis.binder.` prefix
+
+headers::
+The set of custom headers to transfer over AWS Kinesis
++
+Default: "correlationId", "sequenceSize", "sequenceNumber", "contentType", "originalContentType".
+describeStreamBackoff::
+The amount of time in milliseconds in between retries for the `DescribeStream` operation
++
+Default: `1000`.
+describeStreamRetries::
+The amount of times the consumer will retry a `DescribeStream` operation waiting for the stream to be in `ACTIVE` state
++
+Default: `50`.
+autoCreateStream::
+If set to `true`, the binder will create the stream automatically.
+If set to `false`, the binder will rely on the stream being already created.
++
+Default: `true`
+autoAddShards::
+If set to `true`, the binder will create new shards automatically.
+If set to `false`, the binder will rely on the shard size of the stream being already configured.
+If the shard count of the target stream is smaller than the expected value, the binder will ignore that value
++
+Default: `false`
+minShardCount::
+Effective only if `autoAddShards` is set to `true`.
+The minimum number of shards that the binder will configure on the stream from which it produces/consumes data.
+It can be superseded by the `partitionCount` setting of the producer or by the value of `instanceCount * concurrency` settings of the producer (if either is larger)
++
+Default: `1`
+kplKclEnabled::
+Enable the usage of https://docs.aws.amazon.com/streams/latest/dev/kcl2-standard-consumer-java-example.html[Kinesis Client Library] / https://docs.aws.amazon.com/streams/latest/dev/developing-producers-with-kpl.html[Kinesis Producer Library] for all message consumption and production
++
+Default: `false`
++
+enableObservation::
+Enable Micrometer Observation instrumentation on Kinesis Binder producer and consumer.
+By default, Kinesis binder propagates traces producer to consumer via embedded headers.
+The standard W3C `traceparent` and Brave pattern `X-B3*` for headers are mapped by default.
+The Kinesis Binder observation is fully based on https://docs.spring.io/spring-integration/reference/metrics.html#micrometer-observation[Spring Integration conventions]
++
+Default: `false`
+
+=== MetadataStore
+Support for consumer groups is implemented using xref:dynamodb.adoc#spring-integration-support[DynamoDbMetadataStore].
+The `partitionKey` name used in the table is `metadataKey`.
+This is not configurable.
+
+DynamoDB Checkpoint properties are prefixed with `spring.cloud.stream.kinesis.binder.checkpoint.`
+
+table::
+The name to give the DynamoDb table
++
+Default: `SpringIntegrationMetadataStore`
+createDelay::
+The amount of time in seconds between each polling attempt while waiting for the checkpoint DynamoDB table to be created
++
+Default: `1`
+createRetries::
+The amount of times the consumer will poll DynamoDB while waiting for the checkpoint table to be created
++
+Default: `25`
+billingMode::
+The Billing Mode of the DynamoDB table. See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadWriteCapacityMode.html#HowItWorks.OnDemand[DynamoDB On-Demand Mode]. Possible values are `provisioned` and `payPerRequest`. If left empty or set to `payPerRequest` both `readCapacity` and `writeCapacity` are ignored
++
+Default: `payPerRequest`
+readCapacity::
+The Read capacity of the DynamoDb table.
+See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadWriteCapacityMode.html#HowItWorks.ProvisionedThroughput.Manual[DynamoDB Provisioned Throughput]. This property is used only when `billingMode` is set to `provisioned`
++
+Default: `1`
+writeCapacity::
+The `write` capacity of the DynamoDb table.
+See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadWriteCapacityMode.html#HowItWorks.ProvisionedThroughput.Manual[DynamoDB Provisioned Throughput]. This property is used only when `billingMode` is set to `provisioned`
++
+Default: `1`
+timeToLive::
+A period in seconds for items expiration.
+See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/TTL.html[DynamoDB TTL]
++
+No default - means no record expiration.
+
+=== LockRegistry
+LockRegistry is used to ensure exclusive access to each shard so that only one channel adapter in the same consumer group will consumer messages from a single shard in the stream.
+This is implemented using xref:dynamodb.adoc#spring-integration-support[DynamoDbLockRegistry]
+
+DynamoDB `LockRegistry` properties are prefixed with `spring.cloud.stream.kinesis.binder.locks.`
+
+table::
+The name to give the DynamoDB table
++
+Default: `SpringIntegrationLockRegistry`
+billingMode::
+The Billing Mode of the DynamoDB table. See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadWriteCapacityMode.html#HowItWorks.OnDemand[DynamoDB On-Demand Mode]. Possible values are `provisioned` and `payPerRequest`. If left empty or set to `payPerRequest` both `readCapacity` and `writeCapacity` are ignored
++
+Default: `payPerRequest`
+readCapacity::
+The Read capacity of the DynamoDB table.
+See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadWriteCapacityMode.html#HowItWorks.ProvisionedThroughput.Manual[DynamoDB Provisioned Throughput]. This property is used only when `billingMode` is set to `provisioned`
++
+Default: `1`
+writeCapacity::
+The `write` capacity of the DynamoDb table.
+See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadWriteCapacityMode.html#HowItWorks.ProvisionedThroughput.Manual[DynamoDB Provisioned Throughput]. This property is used only when `billingMode` is set to `provisioned`
++
+Default: `1`
+leaseDuration::
+The length of time that the lease for the lock will be granted for.
+If this is set to, for example, 30 seconds, then the lock will be considered as expired after that period and a new owner can acquire it.
+See also See https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/TTL.html[DynamoDB TTL]
++
+Default: `20`
+refreshPeriod::
+How long to wait before trying to get the lock again (if set to 10 seconds, for example, it would attempt to do so every 10 seconds)
++
+Default: `1`
+
+=== Kinesis Consumer Properties
+
+The following properties are available for Kinesis consumers only and must be prefixed with `spring.cloud.stream.kinesis.bindings..consumer`
+
+startTimeout::
+The amount of time to wait for the consumer to start, in milliseconds.
++
+Default: `60000`.
+listenerMode::
+The mode in which records are processed.
+If `record`, each `Message` will contain `byte[]` from a single `Record.data`.
+If `batch`, each `Message` will contain a `List` extracted from the consumed records.
+When `useNativeDecoding = true` is used on the consumer together with the `listenerMode = batch`, there is no any out-of-the-box conversion happened and a result message contains a payload like `List`.
+It's up to the target application to convert those records manually.
++
+Default: `record`
+checkpointMode::
+The mode in which checkpoints are updated.
+If `record`, checkpoints occur after each record is processed (but this option is only effective if `listenerMode` is set to `record`). If `batch`, checkpoints occur after each batch of records is processed.
+If `manual`, checkpoints occur on demand via the `Checkpointer` callback.
+If `periodic`, checkpoints occurs at a specified time interval (from `interval` property in checkpoint configuration)
++
+Default: `batch`
+checkpointInterval::
+The interval, in milliseconds, between two checkpoints when checkpoint mode is `periodic`.
++
+Default - `5000`
+workerId::
+The worker identifier used to distinguish different workers/processes (only used when Kinesis Client Library is enabled).
++
+No default - if not set, default value inside spring-integration-aws will be used (random UUID).
+fanOut::
+The KCL retrieval mode: `true` - fan-out, `false` - polling.
++
+Default: `true`
+metricsLevel::
+The level of metrics to be collected by the Kinesis Client Library.
+Possible values are `NONE`, `SUMMARY`, `DETAILED`.
++
+Default: `DETAILED`
+recordsLimit::
+The maximum number of records to poll per `GetRecords` request.
+Must not be greater than `10000`.
++
+Default: `1000`
+idleBetweenPolls::
+The delay between Kinesis records requests to satisfy AWS throughput requirements.
++
+Default: `1000`
+consumerBackoff::
+The amount of time the consumer will wait to attempt another `GetRecords` operation after a read with no results, in milliseconds.
++
+Default: `1000`
+shardIteratorType::
+The `com.amazonaws.services.kinesis.model.ShardIteratorType` name with an optional `sequenceNumber` for the `AT_SEQUENCE_NUMBER/AFTER_SEQUENCE_NUMBER` or milliseconds for the `AT_TIMESTAMP` after `:`.
+For example: `AT_TIMESTAMP:1515090166767`.
+If `shardIteratorType` is set to `TRIM_HORIZON` explicitly for existing groups, then stored sequence number is ignored, and shard iterator is reset to `TRIM_HORIZON`.
++
+Default: `LATEST` for anonymous groups and `TRIM_HORIZON` otherwise.
+
+NOTE: When `TRIM_HORIZON` shard iterator type is used, we need to take into account the time lag which happens during pointing the `ShardIterator` to the last untrimmed record in the shard in the system (the oldest data record in the shard).
+So the `getRecords()` will move from that point to the last point, which takes time.
+It is by default 1 day, and it can be extended to 7 days.
+This happens only for new consumer groups.
+Any subsequent starts of the consumer in the same group are adjusted according the stored checkpoint via `AFTER_SEQUENCE_NUMBER` iterator type.
+
+shardId::
+An explicit shard id to consume from.
+
+NOTE: Kinesis Client Library does not support a configuration for a specific shard.
+When `shardId` property is used, it is ignored for Kinesis Client Library and standard stream consumer distribution is applied.
+Also, in case of an `instanceCount > 1`, application will throw validation exception.
+The `instanceCount` and `shardId` are considered as mutually exclusive.
+
+embedHeaders::
+Whether to extract headers and payload from Kinesis record data.
++
+Default: `false`
+
+emptyRecordList::
+Whether to emit an empty list of records into a consumer.
+Works only in `listenerMode.batch`.
++
+Default: `false`
+
+leaseTableName::
+The KCL table name for leases.
++
+Default: consumer group
+
+pollingMaxRecords::
+The KCL max records for request in polling mode.
++
+Default: 10000
+
+pollingIdleTime::
+The KCL idle between requests in polling mode.
++
+Default: 1500L
+
+gracefulShutdownTimeout::
+The KCL graceful shutdown timeout in milliseconds.
++
+Default: 0 - regular shutdown process
+
+dynamoDbStreams::
+Consume from DynamoDB Streams.
++
+Default: false
+
+The `KclMessageDrivenChannelAdapter` can be customized programmatically for the `ConfigsBuilder` parts.
+For example, to set a custom value for the `LeaseManagementConfig.maxLeasesForWorker` property, the `ConsumerEndpointCustomizer` bean has to be provided:
+
+[source,java]
+----
+@Bean
+ConsumerEndpointCustomizer consumerEndpointCustomizer() {
+ return (endpoint, destinationName, group) ->
+ endpoint.setLeaseManagementConfigCustomizer(leaseManagementConfig ->
+ leaseManagementConfig.maxLeasesForWorker(10));
+}
+----
+
+Other similar setters on the `KclMessageDrivenChannelAdapter` are:
+
+[source,java]
+----
+setCoordinatorConfigCustomizer(Consumer coordinatorConfigCustomizer);
+
+setLifecycleConfigCustomizer(Consumer lifecycleConfigCustomizer);
+
+setMetricsConfigCustomizer(Consumer metricsConfigCustomizer);
+----
+
+=== Kinesis Producer Properties
+
+The following properties are available for Kinesis producers only and must be prefixed with `spring.cloud.stream.kinesis.bindings..producer.`.
+
+sync::
+Whether the producer should act in a synchronous manner with respect to writing records into a stream.
+If true, the producer will wait for a response from Kinesis after a `PutRecord` operation.
++
+Default: `false`
+
+sendTimeout::
+Effective only if `sync` is set to `true`. The amount of time to wait for a response from Kinesis after a `PutRecord` operation, in milliseconds.
++
+Default: `10000`
+
+Also, if you'd like to produce a batch of records into Kinesis stream, the message payload must be as a `PutRecordsRequest` instance and general Spring Cloud Stream producer property `useNativeEncoding` must be set to `true`, so Spring Cloud Stream won't try to convert a `PutRecordsRequest` into a `byte[]`. The content of the `PutRecordsRequest` is now end-user responsibility.
+
+embedHeaders::
+Whether to serialize message headers and payload into Kinesis record data.
++
+Default: `false`
+
+recordMetadataChannel::
+The bean name of a MessageChannel to which successful send results should be sent.
+Works only for async mode.
+
+kplBackPressureThreshold::
+Maximum records in flight for handling backpressure.
+No backpressure by default.
+When backpressure handling is enabled and number of records in flight exceeds the threshold, a `KplBackpressureException` would be thrown.
+
+[[kinesis-error-channels]]
+== Error Channels
+
+The binder can be configured to send producer exceptions to an error channel.
+See https://docs.spring.io/spring-cloud-stream/docs/current/reference/html/spring-cloud-stream.html#spring-cloud-stream-overview-error-handling[the section on Spring Cloud Stream error handling] for more information.
+
+The payload of the `ErrorMessage` for a `send` failure is an `AwsRequestFailureException` with properties:
+
+* `failedMessage` - the spring-messaging `Message>` that failed to be sent.
+* `request` - the raw `AwsRequest` (either `PutRecordRequest` or `PutRecordsRequest`) that was created from the `failedMessage`.
+
+There is no automatic handling of these exceptions, (such as sending to a dead letter queue), but you can consume these exceptions with your own Spring Integration flow.
+
+[[aws-roles-and-policies]]
+== AWS Roles and Policies
+
+In order to be able to run properly on AWS, the role that will be used by the application needs to have a set of policies configured.
+Here are the policy statements that your application role needs:
+
+[source,json]
+----
+{
+ "Version": "2012-10-17",
+ "Statement": [
+ {
+ "Effect": "Allow",
+ "Action": [
+ "kinesis:ListShards",
+ "kinesis:SubscribeToShard",
+ "kinesis:DescribeStreamSummary",
+ "kinesis:DescribeStreamConsumer",
+ "kinesis:GetShardIterator",
+ "kinesis:GetRecords",
+ "kinesis:PutRecords",
+ "kinesis:DescribeStream"
+ ],
+ "Resource": [
+ "arn:aws:kinesis:::*/*/consumer/*:*",
+ "arn:aws:kinesis:::stream/"
+ ]
+ },
+ {
+ "Effect": "Allow",
+ "Action": "kinesis:DescribeLimits",
+ "Resource": "*"
+ },
+ {
+ "Sid": "DynamoDB",
+ "Effect": "Allow",
+ "Action": [
+ "dynamodb:BatchGetItem",
+ "dynamodb:BatchWriteItem",
+ "dynamodb:PutItem",
+ "dynamodb:GetItem",
+ "dynamodb:Scan",
+ "dynamodb:Query",
+ "dynamodb:UpdateItem",
+ "dynamodb:DeleteItem",
+ "dynamodb:DescribeTable"
+ ],
+ "Resource": [
+ "arn:aws:dynamodb:::table/",
+ "arn:aws:dynamodb:::table/"
+ ]
+ }
+ ]
+}
+----
+
+Keep in mind that these are only the policies to allow the application to consume/produce records from/to Kinesis.
+If you're going to allow spring-cloud-stream-binder-kinesis to create the resources for you, you'll need an extra set of policies.
+[source,json]
+----
+{
+ "Version": "2012-10-17",
+ "Statement": [
+ {
+ "Effect": "Allow",
+ "Action": [
+ "dynamodb:CreateTable",
+ "kinesis:CreateStream",
+ "kinesis:UpdateShardCount",
+ "kinesis:EnableEnhancedMonitoring",
+ "kinesis:DisableEnhancedMonitoring",
+ "dynamodb:DeleteTable",
+ "dynamodb:UpdateTable"
+ ],
+ "Resource": [
+ "arn:aws:dynamodb:::table/",
+ "arn:aws:kinesis:::stream/"
+ ]
+ }
+ ]
+}
+----
+
+If [Server-Side Encryption](https://docs.aws.amazon.com/streams/latest/dev/what-is-sse.html) is enabled, you'll need the following set of policies to encrypt and decrypt Kinesis messages.
+[source,json]
+----
+{
+ "Version": "2012-10-17",
+ "Statement": [
+ {
+ "Effect": "Allow",
+ "Action": [
+ "kms:GenerateDataKey",
+ "kms:Decrypt"
+ ],
+ "Resource": [
+ "arn:aws:kms:::key/"
+ ]
+ }
+ ]
+}
+----
+
+[[dynamodb-streams-support]]
+=== DynamoDB Streams Support
+
+The Kinesis Binder provides support for consuming CDC events from the https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Streams.html[DynamoDB Streams].
+The input binding has to explicitly opt-in for this feature, e.g., `spring.cloud.stream.kinesis.bindings.dynamoDbConsumer-in-0.consumer.dynamoDbStreams=true`.
+The rest of the configuration is done automatically by the binder for both classic and KCL consumers.
+
+NOTE: The binding destination for DynamoDB Stream has to be specified as a Stream ARN on the table to capture changes.
+
+[[telling-the-binder-to-use-your-local-endpoint]]
+=== Telling the binder to use your local endpoint
+
+By default, the Kinesis and DynamoDB Client will try to hit the real AWS Endpoint.
+To change this behavior, there is just enough to provide these properties for Spring Cloud AWS auto-configuration:
+
+----
+spring.cloud.aws.endpoint
+spring.cloud.aws.region.static
+spring.cloud.aws.credentials.access-key
+spring.cloud.aws.credentials.secret-key
+----
+
+Another way to test against LocalStack is to use `Testcontainers` with a `localstack/localstack` image container.
+The `LocalstackContainerTest` interface in this project can be a good sample how to configure container and how to use AWS clients.
+
+[[health-indicator]]
+== Kinesis Binder Health Indicator
+
+The `KinesisBinderHealthIndicator` implementation which is a part of `BindersHealthContributor` composition under the `binders` path.
+An out-of-the-box implementation iterates over Kinesis streams involved in the binder configuration calling a `describeStream` command against them.
+If any of streams doesn't exist, the health is treated as `DOWN`.
+If `LimitExceededException` is thrown according `describeStream` limitations, the `KinesisBinderHealthIndicator` tries over again after one second interval.
+Ony when all the configured streams are described properly the `UP` health is returned.
+You can override out-of-the-box implementation provided your own bean with the `kinesisBinderHealthIndicator` name.
diff --git a/docs/src/main/asciidoc/kinesis.adoc b/docs/src/main/asciidoc/kinesis.adoc
index c0eaa7f57..2f8532db9 100644
--- a/docs/src/main/asciidoc/kinesis.adoc
+++ b/docs/src/main/asciidoc/kinesis.adoc
@@ -179,7 +179,17 @@ public KclMessageDrivenChannelAdapter kclMessageDrivenChannelAdapter(KinesisAsyn
NOTE: Unlike `KinesisMessageDrivenChannelAdapter`, the `KclMessageDrivenChannelAdapter` does not support explicit shard assignments.
-=== Spring Integration Starters
+==== DynamoDB Streams Support
+
+The `KinesisMessageDrivenChannelAdapter` and `KclMessageDrivenChannelAdapter` provide support for consuming CDC events from the https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Streams.html[DynamoDB Streams].
+The `com.amazonaws:dynamodb-streams-kinesis-adapter` dependency must be present on classpath.
+The KCL provides native support for the mentioned adapter, and only instance of `DynamoDbStreamsClient` has to be injected into the `KclMessageDrivenChannelAdapter` to switch consuming logic from the Kinesis stream to DynamoDB stream.
+For consuming via `KinesisMessageDrivenChannelAdapter`, the `SpringDynamoDBStreamsAdapterClient` has to be injected instead of regular `KinesisAsyncClient`.
+The `SpringDynamoDBStreamsAdapterClient` is an extension of the `AmazonDynamoDBStreamsAdapterClient` with overridden `listShards()` and `getRecords()` operations to mimic `KinesisAsyncClient` API called from the `KinesisMessageDrivenChannelAdapter` logic.
+Both channel adapters require a Stream ARN for DynamoDB Stream on the table.
+Using AWS SDK API, such a Stream ARN value is available as a result of the `DescribeTableResponse.table().latestStreamArn()` in the answer to `DynamoDbAsyncClient.describeTable()` request.
+
+==== Spring Integration Starters
The Spring Integration dependency in the `spring-cloud-aws-kinesis` module is `optional` to avoid unnecessary artifacts on classpath when Spring Integration is not used.
For convenience, a dedicated `spring-cloud-aws-starter-integration-kinesis` is provided managing all the required dependencies for Spring Integration support with a classical Amazon Kinesis client.
diff --git a/pom.xml b/pom.xml
index 70f1954dd..0cc50b9e0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,6 +45,7 @@
spring-cloud-aws-sqs
spring-cloud-aws-dynamodb
spring-cloud-aws-kinesis
+ spring-cloud-aws-kinesis-stream-binder
spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis
spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis-producer
spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis-client
diff --git a/spring-cloud-aws-dependencies/pom.xml b/spring-cloud-aws-dependencies/pom.xml
index 210d2d436..e5375deb6 100644
--- a/spring-cloud-aws-dependencies/pom.xml
+++ b/spring-cloud-aws-dependencies/pom.xml
@@ -28,9 +28,11 @@
3.1.2
1.0.4
2.0.6
+ 2.0.1
3.3.5
1.6
5.0.0-RC1
+ 5.0.0
2.1.3
2.0.3
2.0.0-RC1
@@ -54,6 +56,13 @@
pom
import
+
+ org.springframework.cloud
+ spring-cloud-stream-dependencies
+ ${spring-cloud-stream.version}
+ pom
+ import
+
org.springframework.modulith
@@ -118,6 +127,11 @@
spring-cloud-aws-kinesis
${project.version}
+
+ io.awspring.cloud
+ spring-cloud-aws-kinesis-stream-binder
+ ${project.version}
+
io.awspring.cloud
spring-cloud-aws-starter-integration-kinesis
@@ -281,7 +295,11 @@
amazon-kinesis-producer
${kpl.version}
-
+
+ com.amazonaws
+ dynamodb-streams-kinesis-adapter
+ ${dynamodb-streams.version}
+
diff --git a/spring-cloud-aws-kinesis-stream-binder/pom.xml b/spring-cloud-aws-kinesis-stream-binder/pom.xml
new file mode 100644
index 000000000..ea018da12
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/pom.xml
@@ -0,0 +1,106 @@
+
+
+ 4.0.0
+
+ io.awspring.cloud
+ spring-cloud-aws
+ 4.0.0-M1
+
+
+ spring-cloud-aws-kinesis-stream-binder
+ Spring Cloud Stream AWS Kinesis Binder
+
+
+
+ io.awspring.cloud
+ spring-cloud-aws-starter
+
+
+ io.awspring.cloud
+ spring-cloud-aws-kinesis
+
+
+ io.awspring.cloud
+ spring-cloud-aws-dynamodb
+
+
+ software.amazon.awssdk
+ kinesis
+
+
+ software.amazon.kinesis
+ amazon-kinesis-client
+
+
+ software.amazon.kinesis
+ amazon-kinesis-producer
+
+
+ com.amazonaws
+ dynamodb-streams-kinesis-adapter
+
+
+ org.springframework.cloud
+ spring-cloud-stream
+
+
+
+ org.springframework.boot
+ spring-boot-configuration-processor
+ true
+
+
+ org.springframework.boot
+ spring-boot-starter-actuator
+ true
+
+
+
+ org.springframework.cloud
+ spring-cloud-stream-test-support
+ test
+
+
+ org.springframework.boot
+ spring-boot-micrometer-tracing-brave
+ test
+
+
+ org.springframework.boot
+ spring-boot-micrometer-tracing-test
+ test
+
+
+ io.micrometer
+ micrometer-tracing-integration-test
+ test
+
+
+ io.opentelemetry
+ *
+
+
+ com.wavefront
+ *
+
+
+ io.micrometer
+ micrometer-tracing-bridge-otel
+
+
+
+
+ org.testcontainers
+ testcontainers-localstack
+ test
+
+
+ org.testcontainers
+ testcontainers-junit-jupiter
+ test
+
+
+
+
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderHealthIndicator.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderHealthIndicator.java
new file mode 100644
index 000000000..7e5dbefee
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderHealthIndicator.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeUnit;
+import org.springframework.boot.health.contributor.Health;
+import org.springframework.boot.health.contributor.HealthIndicator;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
+
+/**
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+public class KinesisBinderHealthIndicator implements HealthIndicator {
+
+ private final KinesisMessageChannelBinder kinesisMessageChannelBinder;
+
+ public KinesisBinderHealthIndicator(KinesisMessageChannelBinder kinesisMessageChannelBinder) {
+ this.kinesisMessageChannelBinder = kinesisMessageChannelBinder;
+ }
+
+ @Override
+ public Health health() {
+ KinesisAsyncClient amazonKinesis = this.kinesisMessageChannelBinder.getAmazonKinesis();
+ List streamsInUse = new ArrayList<>(this.kinesisMessageChannelBinder.getStreamsInUse());
+ for (String stream : streamsInUse) {
+ while (true) {
+ try {
+ amazonKinesis.listShards(request -> request.streamName(stream).maxResults(1)).join();
+ break;
+ }
+ catch (CompletionException ex) {
+ Throwable cause = ex.getCause();
+ if (cause instanceof LimitExceededException) {
+ try {
+ TimeUnit.SECONDS.sleep(1);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return Health.down().withException(ex).build();
+ }
+ }
+ else {
+ return Health.down().withException(ex).build();
+ }
+ }
+ }
+ }
+ return Health.up().build();
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/KinesisMessageChannelBinder.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/KinesisMessageChannelBinder.java
new file mode 100644
index 000000000..595983361
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/KinesisMessageChannelBinder.java
@@ -0,0 +1,569 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder;
+
+import io.awspring.cloud.kinesis.integration.KclMessageDrivenChannelAdapter;
+import io.awspring.cloud.kinesis.integration.KinesisMessageDrivenChannelAdapter;
+import io.awspring.cloud.kinesis.integration.KinesisMessageHandler;
+import io.awspring.cloud.kinesis.integration.KinesisMessageHeaderErrorMessageStrategy;
+import io.awspring.cloud.kinesis.integration.KinesisShardOffset;
+import io.awspring.cloud.kinesis.integration.KplMessageHandler;
+import io.awspring.cloud.kinesis.integration.SpringDynamoDBStreamsAdapterClient;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisBinderConfigurationProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisConsumerProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisExtendedBindingProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisProducerProperties;
+import io.awspring.cloud.kinesis.stream.binder.provisioning.KinesisConsumerDestination;
+import io.awspring.cloud.kinesis.stream.binder.provisioning.KinesisStreamProvisioner;
+import io.micrometer.observation.ObservationRegistry;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.springframework.cloud.stream.binder.AbstractMessageChannelBinder;
+import org.springframework.cloud.stream.binder.BinderHeaders;
+import org.springframework.cloud.stream.binder.BinderSpecificPropertiesProvider;
+import org.springframework.cloud.stream.binder.ExtendedConsumerProperties;
+import org.springframework.cloud.stream.binder.ExtendedProducerProperties;
+import org.springframework.cloud.stream.binder.ExtendedPropertiesBinder;
+import org.springframework.cloud.stream.binder.PartitionKeyExtractorStrategy;
+import org.springframework.cloud.stream.provisioning.ConsumerDestination;
+import org.springframework.cloud.stream.provisioning.ProducerDestination;
+import org.springframework.expression.EvaluationContext;
+import org.springframework.integration.channel.NullChannel;
+import org.springframework.integration.core.MessageProducer;
+import org.springframework.integration.endpoint.MessageProducerSupport;
+import org.springframework.integration.expression.ExpressionUtils;
+import org.springframework.integration.expression.FunctionExpression;
+import org.springframework.integration.handler.AbstractMessageProducingHandler;
+import org.springframework.integration.mapping.BytesMessageMapper;
+import org.springframework.integration.metadata.ConcurrentMetadataStore;
+import org.springframework.integration.support.ErrorMessageStrategy;
+import org.springframework.integration.support.locks.LockRegistry;
+import org.springframework.integration.support.management.IntegrationManagement;
+import org.springframework.lang.Nullable;
+import org.springframework.messaging.Message;
+import org.springframework.messaging.MessageChannel;
+import org.springframework.messaging.MessageHandler;
+import org.springframework.messaging.support.ChannelInterceptor;
+import org.springframework.messaging.support.InterceptableChannel;
+import org.springframework.messaging.support.MessageBuilder;
+import org.springframework.util.Assert;
+import org.springframework.util.CollectionUtils;
+import org.springframework.util.ObjectUtils;
+import org.springframework.util.StringUtils;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.InvalidArgumentException;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.ShardIteratorType;
+import software.amazon.kinesis.common.InitialPositionInStream;
+import software.amazon.kinesis.common.InitialPositionInStreamExtended;
+import software.amazon.kinesis.producer.KinesisProducer;
+import software.amazon.kinesis.producer.KinesisProducerConfiguration;
+
+/**
+ *
+ * The Spring Cloud Stream Binder implementation for AWS Kinesis.
+ *
+ * @author Peter Oates
+ * @author Artem Bilan
+ * @author Arnaud Lecollaire
+ * @author Dirk Bonhomme
+ * @author Asiel Caballero
+ * @author Dmytro Danilenkov
+ * @author Minkyu Moon
+ *
+ * @since 4.0
+ */
+public class KinesisMessageChannelBinder extends
+ AbstractMessageChannelBinder, ExtendedProducerProperties, KinesisStreamProvisioner>
+ implements ExtendedPropertiesBinder {
+
+ private static final ErrorMessageStrategy ERROR_MESSAGE_STRATEGY = new KinesisMessageHeaderErrorMessageStrategy();
+
+ private final List streamsInUse = new ArrayList<>();
+
+ private final KinesisBinderConfigurationProperties configurationProperties;
+
+ private final KinesisAsyncClient amazonKinesis;
+
+ private final CloudWatchAsyncClient cloudWatchClient;
+
+ private final DynamoDbAsyncClient dynamoDBClient;
+
+ private final BytesMessageMapper embeddedHeadersMapper;
+
+ private final DynamoDbStreamsClient dynamoDBStreams;
+
+ private KinesisExtendedBindingProperties extendedBindingProperties = new KinesisExtendedBindingProperties();
+
+ @Nullable
+ private ConcurrentMetadataStore checkpointStore;
+
+ @Nullable
+ private LockRegistry> lockRegistry;
+
+ @Nullable
+ private KinesisProducerConfiguration kinesisProducerConfiguration;
+
+ private EvaluationContext evaluationContext;
+
+ private ObservationRegistry observationRegistry = ObservationRegistry.NOOP;
+
+ @SuppressWarnings("removal")
+ public KinesisMessageChannelBinder(KinesisBinderConfigurationProperties configurationProperties,
+ KinesisStreamProvisioner provisioningProvider, KinesisAsyncClient amazonKinesis,
+ @Nullable DynamoDbAsyncClient dynamoDBClient, @Nullable DynamoDbStreamsClient dynamoDBStreams,
+ @Nullable CloudWatchAsyncClient cloudWatchClient) {
+
+ super(new String[0], provisioningProvider);
+ Assert.notNull(amazonKinesis, "'amazonKinesis' must not be null");
+ this.configurationProperties = configurationProperties;
+ this.amazonKinesis = amazonKinesis;
+ this.cloudWatchClient = cloudWatchClient;
+ this.dynamoDBClient = dynamoDBClient;
+ this.dynamoDBStreams = dynamoDBStreams;
+ this.embeddedHeadersMapper = new org.springframework.integration.support.json.EmbeddedJsonHeadersMessageMapper(
+ headersToMap(configurationProperties));
+ }
+
+ public void setExtendedBindingProperties(KinesisExtendedBindingProperties extendedBindingProperties) {
+ this.extendedBindingProperties = extendedBindingProperties;
+ }
+
+ public void setCheckpointStore(ConcurrentMetadataStore checkpointStore) {
+ this.checkpointStore = checkpointStore;
+ }
+
+ public void setLockRegistry(LockRegistry> lockRegistry) {
+ this.lockRegistry = lockRegistry;
+ }
+
+ public void setKinesisProducerConfiguration(KinesisProducerConfiguration kinesisProducerConfiguration) {
+ this.kinesisProducerConfiguration = kinesisProducerConfiguration;
+ }
+
+ public void setObservationRegistry(@Nullable ObservationRegistry observationRegistry) {
+ this.observationRegistry = observationRegistry;
+ }
+
+ @Override
+ public KinesisConsumerProperties getExtendedConsumerProperties(String channelName) {
+ return this.extendedBindingProperties.getExtendedConsumerProperties(channelName);
+ }
+
+ @Override
+ public KinesisProducerProperties getExtendedProducerProperties(String channelName) {
+ return this.extendedBindingProperties.getExtendedProducerProperties(channelName);
+ }
+
+ @Override
+ public String getDefaultsPrefix() {
+ return this.extendedBindingProperties.getDefaultsPrefix();
+ }
+
+ @Override
+ public Class extends BinderSpecificPropertiesProvider> getExtendedPropertiesEntryClass() {
+ return this.extendedBindingProperties.getExtendedPropertiesEntryClass();
+ }
+
+ public KinesisAsyncClient getAmazonKinesis() {
+ return this.amazonKinesis;
+ }
+
+ public List getStreamsInUse() {
+ return this.streamsInUse;
+ }
+
+ @Override
+ protected void onInit() throws Exception {
+ super.onInit();
+ this.evaluationContext = ExpressionUtils.createStandardEvaluationContext(getBeanFactory());
+ }
+
+ @Override
+ public String getBinderIdentity() {
+ return "kinesis-" + super.getBinderIdentity();
+ }
+
+ @Override
+ protected MessageHandler createProducerMessageHandler(ProducerDestination destination,
+ ExtendedProducerProperties producerProperties, MessageChannel channel,
+ @Nullable MessageChannel errorChannel) {
+
+ FunctionExpression> partitionKeyExpression = new FunctionExpression<>(
+ (m) -> m.getHeaders().containsKey(BinderHeaders.PARTITION_HEADER)
+ ? m.getHeaders().get(BinderHeaders.PARTITION_HEADER)
+ : m.getPayload().hashCode());
+ AbstractMessageProducingHandler messageHandler;
+ KinesisProducerProperties kinesisProducerProperties = producerProperties.getExtension();
+ if (this.configurationProperties.isKplKclEnabled()) {
+ messageHandler = createKplMessageHandler(destination, partitionKeyExpression, kinesisProducerProperties,
+ kinesisProducerProperties.isEmbedHeaders() && !producerProperties.isUseNativeEncoding());
+ }
+ else {
+ messageHandler = createKinesisMessageHandler(destination, partitionKeyExpression,
+ kinesisProducerProperties.isEmbedHeaders() && !producerProperties.isUseNativeEncoding());
+ }
+ messageHandler.setAsync(!kinesisProducerProperties.isSync());
+ messageHandler.setSendTimeout(kinesisProducerProperties.getSendTimeout());
+ messageHandler.setBeanFactory(getBeanFactory());
+ String recordMetadataChannel = kinesisProducerProperties.getRecordMetadataChannel();
+ if (StringUtils.hasText(recordMetadataChannel)) {
+ messageHandler.setOutputChannelName(recordMetadataChannel);
+ }
+ else {
+ messageHandler.setOutputChannel(new NullChannel());
+ }
+
+ if (errorChannel != null) {
+ ((InterceptableChannel) channel).addInterceptor(new ChannelInterceptor() {
+
+ @Override
+ public Message> preSend(Message> message, MessageChannel channel) {
+ return MessageBuilder.fromMessage(message).setErrorChannel(errorChannel).build();
+ }
+
+ });
+ }
+
+ this.streamsInUse.add(destination.getName());
+
+ return messageHandler;
+ }
+
+ @Override
+ protected MessageHandler createProducerMessageHandler(ProducerDestination destination,
+ ExtendedProducerProperties producerProperties, MessageChannel errorChannel) {
+ return null;
+ }
+
+ private KinesisMessageHandler createKinesisMessageHandler(ProducerDestination destination,
+ FunctionExpression> partitionKeyExpression, boolean embedHeaders) {
+
+ KinesisMessageHandler messageHandler = new KinesisMessageHandler(this.amazonKinesis);
+ messageHandler.setStream(destination.getName());
+ messageHandler.setPartitionKeyExpression(partitionKeyExpression);
+ if (embedHeaders) {
+ messageHandler.setEmbeddedHeadersMapper(this.embeddedHeadersMapper);
+ }
+ return messageHandler;
+ }
+
+ private KplMessageHandler createKplMessageHandler(ProducerDestination destination,
+ FunctionExpression> partitionKeyExpression, KinesisProducerProperties kinesisProducerProperties,
+ boolean embedHeaders) {
+
+ KplMessageHandler messageHandler = new KplMessageHandler(
+ new KinesisProducer(this.kinesisProducerConfiguration));
+ messageHandler.setStream(destination.getName());
+ messageHandler.setPartitionKeyExpression(partitionKeyExpression);
+ messageHandler.setBackPressureThreshold(kinesisProducerProperties.getKplBackPressureThreshold());
+ if (embedHeaders) {
+ messageHandler.setEmbeddedHeadersMapper(this.embeddedHeadersMapper);
+ }
+ return messageHandler;
+ }
+
+ @Override
+ protected void postProcessOutputChannel(MessageChannel outputChannel,
+ ExtendedProducerProperties producerProperties) {
+
+ ((IntegrationManagement) outputChannel).registerObservationRegistry(this.observationRegistry);
+
+ if (outputChannel instanceof InterceptableChannel interceptableChannel && producerProperties.isPartitioned()) {
+ interceptableChannel.addInterceptor(0, new ChannelInterceptor() {
+
+ private final PartitionKeyExtractorStrategy partitionKeyExtractorStrategy;
+
+ {
+ if (StringUtils.hasText(producerProperties.getPartitionKeyExtractorName())) {
+ this.partitionKeyExtractorStrategy = getBeanFactory().getBean(
+ producerProperties.getPartitionKeyExtractorName(), PartitionKeyExtractorStrategy.class);
+ }
+ else {
+ this.partitionKeyExtractorStrategy = (message) -> producerProperties.getPartitionKeyExpression()
+ .getValue(KinesisMessageChannelBinder.this.evaluationContext, message);
+ }
+ }
+
+ @Override
+ public Message> preSend(Message> message, MessageChannel channel) {
+ Object partitionKey = this.partitionKeyExtractorStrategy.extractKey(message);
+ return MessageBuilder.fromMessage(message).setHeader(BinderHeaders.PARTITION_HEADER, partitionKey)
+ .build();
+ }
+
+ });
+ }
+ }
+
+ @Override
+ protected MessageProducer createConsumerEndpoint(ConsumerDestination destination, String group,
+ ExtendedConsumerProperties properties) {
+
+ this.streamsInUse.add(destination.getName());
+
+ MessageProducerSupport adapter;
+ if (this.configurationProperties.isKplKclEnabled()) {
+ adapter = createKclConsumerEndpoint(destination, group, properties);
+ }
+ else {
+ adapter = createKinesisConsumerEndpoint(destination, group, properties);
+ }
+
+ adapter.registerObservationRegistry(this.observationRegistry);
+ adapter.setBeanFactory(getBeanFactory());
+ adapter.setComponentName(String.format("Consumer for [%s]", destination.getName()));
+
+ return adapter;
+ }
+
+ private MessageProducerSupport createKclConsumerEndpoint(ConsumerDestination destination, String group,
+ ExtendedConsumerProperties properties) {
+
+ KinesisConsumerProperties kinesisConsumerProperties = properties.getExtension();
+
+ if (kinesisConsumerProperties.getShardId() != null) {
+ logger.warn("Kinesis Client Library doesn't does not support explicit shard configuration. "
+ + "Ignoring 'shardId' property");
+ }
+
+ String[] streams = Arrays.stream(StringUtils.commaDelimitedListToStringArray(destination.getName()))
+ .map(String::trim).toArray(String[]::new);
+
+ KclMessageDrivenChannelAdapter adapter = new KclMessageDrivenChannelAdapter(this.amazonKinesis,
+ this.cloudWatchClient, this.dynamoDBClient, streams);
+ if (kinesisConsumerProperties.isDynamoDbStreams()) {
+ adapter.setDynamoDBStreams(this.dynamoDBStreams);
+ }
+
+ boolean anonymous = !StringUtils.hasText(group);
+ String consumerGroup = anonymous ? "anonymous." + UUID.randomUUID() : group;
+
+ String workerId = kinesisConsumerProperties.getWorkerId() != null ? kinesisConsumerProperties.getWorkerId()
+ : UUID.randomUUID().toString();
+
+ String shardIteratorType = kinesisConsumerProperties.getShardIteratorType();
+
+ InitialPositionInStreamExtended kinesisShardOffset = InitialPositionInStreamExtended
+ .newInitialPosition(InitialPositionInStream.LATEST);
+
+ if (StringUtils.hasText(shardIteratorType)) {
+ String[] typeValue = shardIteratorType.split(":", 2);
+ ShardIteratorType iteratorType = ShardIteratorType.valueOf(typeValue[0]);
+ if (typeValue.length > 1) {
+ if (ShardIteratorType.AT_TIMESTAMP.equals(iteratorType)) {
+ kinesisShardOffset = InitialPositionInStreamExtended
+ .newInitialPositionAtTimestamp(new Date(Long.parseLong(typeValue[1])));
+ }
+ else {
+ throw new IllegalArgumentException("The KCL does not support 'AT_SEQUENCE_NUMBER' "
+ + "or 'AFTER_SEQUENCE_NUMBER' initial position in stream.");
+ }
+ }
+ else {
+ kinesisShardOffset = InitialPositionInStreamExtended
+ .newInitialPosition(InitialPositionInStream.valueOf(iteratorType.name()));
+ }
+ }
+
+ kinesisShardOffset = anonymous || StringUtils.hasText(shardIteratorType) ? kinesisShardOffset
+ : InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON);
+
+ adapter.setConsumerGroup(consumerGroup);
+ adapter.setWorkerId(workerId);
+ adapter.setStreamInitialSequence(kinesisShardOffset);
+ adapter.setCheckpointMode(kinesisConsumerProperties.getCheckpointMode());
+ adapter.setCheckpointsInterval(kinesisConsumerProperties.getCheckpointInterval());
+ adapter.setConsumerBackoff(kinesisConsumerProperties.getConsumerBackoff());
+ adapter.setListenerMode(kinesisConsumerProperties.getListenerMode());
+ adapter.setFanOut(kinesisConsumerProperties.isFanOut());
+ adapter.setMetricsLevel(kinesisConsumerProperties.getMetricsLevel());
+ adapter.setEmptyRecordList(kinesisConsumerProperties.isEmptyRecordList());
+ adapter.setLeaseTableName(kinesisConsumerProperties.getLeaseTableName());
+ adapter.setPollingMaxRecords(kinesisConsumerProperties.getPollingMaxRecords());
+ adapter.setPollingIdleTime(kinesisConsumerProperties.getPollingIdleTime());
+ adapter.setGracefulShutdownTimeout(kinesisConsumerProperties.getGracefulShutdownTimeout());
+ if (kinesisConsumerProperties.isEmbedHeaders() && !properties.isUseNativeDecoding()) {
+ adapter.setEmbeddedHeadersMapper(this.embeddedHeadersMapper);
+ }
+
+ if (properties.isUseNativeDecoding()) {
+ adapter.setConverter(null);
+ }
+ else {
+ // Defer byte[] conversion to the InboundContentTypeConvertingInterceptor
+ adapter.setConverter((bytes) -> bytes);
+ }
+
+ ErrorInfrastructure errorInfrastructure = registerErrorInfrastructure(destination, consumerGroup, properties);
+ adapter.setErrorMessageStrategy(ERROR_MESSAGE_STRATEGY);
+ adapter.setErrorChannel(errorInfrastructure.getErrorChannel());
+ adapter.setBindSourceRecord(true);
+ return adapter;
+ }
+
+ private MessageProducerSupport createKinesisConsumerEndpoint(ConsumerDestination destination, String group,
+ ExtendedConsumerProperties properties) {
+
+ KinesisConsumerProperties kinesisConsumerProperties = properties.getExtension();
+
+ if (properties.getInstanceCount() > 1 && properties.getExtension().getShardId() != null) {
+ throw InvalidArgumentException.builder()
+ .message("'instanceCount' more than 1 and 'shardId' cannot be provided together.").build();
+ }
+
+ Set shardOffsets = null;
+
+ String shardIteratorType = kinesisConsumerProperties.getShardIteratorType();
+
+ KinesisShardOffset kinesisShardOffset = KinesisShardOffset.latest();
+
+ if (StringUtils.hasText(shardIteratorType)) {
+ String[] typeValue = shardIteratorType.split(":", 2);
+ ShardIteratorType iteratorType = ShardIteratorType.valueOf(typeValue[0]);
+ kinesisShardOffset = new KinesisShardOffset(iteratorType);
+ if (typeValue.length > 1) {
+ if (ShardIteratorType.AT_TIMESTAMP.equals(iteratorType)) {
+ kinesisShardOffset.setTimestamp(Instant.ofEpochMilli(Long.parseLong(typeValue[1])));
+ }
+ else {
+ kinesisShardOffset.setSequenceNumber(typeValue[1]);
+ }
+ }
+ }
+
+ if (properties.getInstanceCount() > 1) {
+ Assert.state(!properties.isMultiplex(), "Cannot use multi-stream binding together with 'instance-index'");
+ shardOffsets = new HashSet<>();
+ KinesisConsumerDestination kinesisConsumerDestination = (KinesisConsumerDestination) destination;
+ List shards = kinesisConsumerDestination.getShards();
+ for (int i = 0; i < shards.size(); i++) {
+ // divide shards across instances
+ if ((i % properties.getInstanceCount()) == properties.getInstanceIndex()) {
+ KinesisShardOffset shardOffset = new KinesisShardOffset(kinesisShardOffset);
+ shardOffset.setStream(destination.getName());
+ shardOffset.setShard(shards.get(i).shardId());
+ shardOffsets.add(shardOffset);
+ }
+ }
+ }
+
+ KinesisAsyncClient amazonKinesisClient = kinesisConsumerProperties.isDynamoDbStreams()
+ ? new SpringDynamoDBStreamsAdapterClient(this.dynamoDBStreams)
+ : this.amazonKinesis;
+
+ KinesisMessageDrivenChannelAdapter adapter;
+
+ String shardId = kinesisConsumerProperties.getShardId();
+
+ if (CollectionUtils.isEmpty(shardOffsets) && shardId == null) {
+ String[] streams = Arrays.stream(StringUtils.commaDelimitedListToStringArray(destination.getName()))
+ .map(String::trim).toArray(String[]::new);
+ adapter = new KinesisMessageDrivenChannelAdapter(amazonKinesisClient, streams);
+ }
+ else if (shardId != null) {
+ Assert.state(!properties.isMultiplex(), "Cannot use multi-stream binding together with 'shard-id'");
+ KinesisShardOffset shardOffset = new KinesisShardOffset(kinesisShardOffset);
+ shardOffset.setStream(destination.getName());
+ shardOffset.setShard(shardId);
+ adapter = new KinesisMessageDrivenChannelAdapter(amazonKinesisClient, shardOffset);
+ }
+ else {
+ adapter = new KinesisMessageDrivenChannelAdapter(amazonKinesisClient,
+ shardOffsets.toArray(new KinesisShardOffset[0]));
+ }
+
+ boolean anonymous = !StringUtils.hasText(group);
+ String consumerGroup = anonymous ? "anonymous." + UUID.randomUUID() : group;
+ adapter.setConsumerGroup(consumerGroup);
+
+ KinesisShardOffset streamInitialSequence = anonymous || StringUtils.hasText(shardIteratorType)
+ ? kinesisShardOffset
+ : KinesisShardOffset.trimHorizon();
+ if (ShardIteratorType.TRIM_HORIZON.name().equals(shardIteratorType)) {
+ streamInitialSequence.setReset(true);
+ }
+ adapter.setStreamInitialSequence(streamInitialSequence);
+
+ adapter.setListenerMode(kinesisConsumerProperties.getListenerMode());
+ if (kinesisConsumerProperties.isEmbedHeaders() && !properties.isUseNativeDecoding()) {
+ adapter.setEmbeddedHeadersMapper(this.embeddedHeadersMapper);
+ }
+
+ if (properties.isUseNativeDecoding()) {
+ adapter.setConverter(null);
+ }
+ else {
+ // Defer byte[] conversion to the InboundContentTypeConvertingInterceptor
+ adapter.setConverter((bytes) -> bytes);
+ }
+
+ adapter.setCheckpointMode(kinesisConsumerProperties.getCheckpointMode());
+ adapter.setRecordsLimit(kinesisConsumerProperties.getRecordsLimit());
+ adapter.setIdleBetweenPolls(kinesisConsumerProperties.getIdleBetweenPolls());
+ adapter.setConsumerBackoff(kinesisConsumerProperties.getConsumerBackoff());
+ adapter.setCheckpointsInterval(kinesisConsumerProperties.getCheckpointInterval());
+
+ if (this.checkpointStore != null) {
+ adapter.setCheckpointStore(this.checkpointStore);
+ }
+
+ adapter.setLockRegistry(this.lockRegistry);
+
+ adapter.setConcurrency(properties.getConcurrency());
+ adapter.setStartTimeout(kinesisConsumerProperties.getStartTimeout());
+ adapter.setDescribeStreamBackoff(this.configurationProperties.getDescribeStreamBackoff());
+ adapter.setDescribeStreamRetries(this.configurationProperties.getDescribeStreamRetries());
+
+ ErrorInfrastructure errorInfrastructure = registerErrorInfrastructure(destination, consumerGroup, properties);
+ adapter.setErrorMessageStrategy(ERROR_MESSAGE_STRATEGY);
+ adapter.setErrorChannel(errorInfrastructure.getErrorChannel());
+ adapter.setBindSourceRecord(true);
+ return adapter;
+ }
+
+ @Override
+ protected ErrorMessageStrategy getErrorMessageStrategy() {
+ return ERROR_MESSAGE_STRATEGY;
+ }
+
+ private static String[] headersToMap(KinesisBinderConfigurationProperties configurationProperties) {
+ Assert.notNull(configurationProperties, "'configurationProperties' must not be null");
+ List headers = new ArrayList<>();
+ Collections.addAll(headers, BinderHeaders.STANDARD_HEADERS);
+ if (configurationProperties.isEnableObservation()) {
+ headers.add("traceparent");
+ headers.add("X-B3*");
+ headers.add("b3");
+ }
+ String[] additionalHeaders = configurationProperties.getHeaders();
+ if (!ObjectUtils.isEmpty(additionalHeaders)) {
+ Collections.addAll(headers, additionalHeaders);
+ }
+ return headers.toArray(new String[0]);
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/DynamoDbStreamsProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/DynamoDbStreamsProperties.java
new file mode 100644
index 000000000..17b467c1a
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/DynamoDbStreamsProperties.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.config;
+
+import io.awspring.cloud.autoconfigure.AwsClientProperties;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+
+/**
+ * {@link ConfigurationProperties} for configuring the DynamoDb Streams client.
+ *
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@ConfigurationProperties(prefix = "spring.cloud.aws.dynamodb-streams")
+public class DynamoDbStreamsProperties extends AwsClientProperties {
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/ExtendedBindingHandlerMappingsProviderConfiguration.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/ExtendedBindingHandlerMappingsProviderConfiguration.java
new file mode 100644
index 000000000..f8007b15d
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/ExtendedBindingHandlerMappingsProviderConfiguration.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.config;
+
+import java.util.Map;
+import org.springframework.boot.context.properties.source.ConfigurationPropertyName;
+import org.springframework.cloud.stream.config.BindingHandlerAdvise.MappingsProvider;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+
+/**
+ * Configuration for extended binding metadata.
+ *
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+
+@Configuration(proxyBeanMethods = false)
+public class ExtendedBindingHandlerMappingsProviderConfiguration {
+
+ @Bean
+ public MappingsProvider kinesisExtendedPropertiesDefaultMappingsProvider() {
+ return () -> Map.of(ConfigurationPropertyName.of("spring.cloud.stream.kinesis.bindings"),
+ ConfigurationPropertyName.of("spring.cloud.stream.kinesis.default"));
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/KinesisBinderConfiguration.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/KinesisBinderConfiguration.java
new file mode 100644
index 000000000..be4887b6e
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/KinesisBinderConfiguration.java
@@ -0,0 +1,283 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.config;
+
+import io.awspring.cloud.autoconfigure.AwsClientCustomizer;
+import io.awspring.cloud.autoconfigure.core.AwsClientBuilderConfigurer;
+import io.awspring.cloud.autoconfigure.core.CredentialsProviderAutoConfiguration;
+import io.awspring.cloud.autoconfigure.core.RegionProviderAutoConfiguration;
+import io.awspring.cloud.autoconfigure.dynamodb.DynamoDbProperties;
+import io.awspring.cloud.autoconfigure.metrics.CloudWatchProperties;
+import io.awspring.cloud.dynamodb.DynamoDbLockRegistry;
+import io.awspring.cloud.dynamodb.DynamoDbLockRepository;
+import io.awspring.cloud.dynamodb.DynamoDbMetadataStore;
+import io.awspring.cloud.kinesis.stream.binder.KinesisBinderHealthIndicator;
+import io.awspring.cloud.kinesis.stream.binder.KinesisMessageChannelBinder;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisBinderConfigurationProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisExtendedBindingProperties;
+import io.awspring.cloud.kinesis.stream.binder.provisioning.KinesisStreamProvisioner;
+import io.micrometer.observation.ObservationRegistry;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Stream;
+import org.springframework.beans.factory.ObjectProvider;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.AutoConfiguration;
+import org.springframework.boot.autoconfigure.AutoConfigureAfter;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnBean;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnClass;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
+import org.springframework.boot.context.properties.EnableConfigurationProperties;
+import org.springframework.boot.health.autoconfigure.contributor.ConditionalOnEnabledHealthIndicator;
+import org.springframework.boot.health.contributor.HealthIndicator;
+import org.springframework.cloud.stream.binder.Binder;
+import org.springframework.cloud.stream.binding.Bindable;
+import org.springframework.cloud.stream.config.ConsumerEndpointCustomizer;
+import org.springframework.cloud.stream.config.ProducerMessageHandlerCustomizer;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.integration.endpoint.MessageProducerSupport;
+import org.springframework.integration.handler.AbstractMessageProducingHandler;
+import org.springframework.integration.metadata.ConcurrentMetadataStore;
+import org.springframework.integration.support.locks.LockRegistry;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.regions.providers.AwsRegionProvider;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClientBuilder;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClientBuilder;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClientBuilder;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder;
+import software.amazon.kinesis.producer.KinesisProducerConfiguration;
+
+/**
+ * The auto-configuration for AWS components and Spring Cloud Stream Kinesis Binder.
+ *
+ * @author Peter Oates
+ * @author Artem Bilan
+ * @author Arnaud Lecollaire
+ * @author Asiel Caballero
+ *
+ * @since 4.0
+ */
+@AutoConfiguration
+@AutoConfigureAfter({ CredentialsProviderAutoConfiguration.class, RegionProviderAutoConfiguration.class })
+@ConditionalOnMissingBean(Binder.class)
+@EnableConfigurationProperties({ KinesisBinderConfigurationProperties.class, KinesisExtendedBindingProperties.class,
+ KinesisProperties.class, DynamoDbProperties.class, DynamoDbStreamsProperties.class,
+ CloudWatchProperties.class })
+public class KinesisBinderConfiguration {
+
+ private final KinesisBinderConfigurationProperties configurationProperties;
+
+ private final AwsCredentialsProvider awsCredentialsProvider;
+
+ private final AwsClientBuilderConfigurer awsClientBuilderConfigurer;
+
+ private final Region region;
+
+ private final boolean hasInputs;
+
+ public KinesisBinderConfiguration(KinesisBinderConfigurationProperties configurationProperties,
+ AwsCredentialsProvider awsCredentialsProvider, AwsRegionProvider regionProvider,
+ AwsClientBuilderConfigurer awsClientBuilderConfigurer, List bindables) {
+
+ this.configurationProperties = configurationProperties;
+ this.awsCredentialsProvider = awsCredentialsProvider;
+ this.awsClientBuilderConfigurer = awsClientBuilderConfigurer;
+ this.region = regionProvider.getRegion();
+ this.hasInputs = bindables.stream().map(Bindable::getInputs).flatMap(Set::stream).findFirst().isPresent();
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ public KinesisAsyncClient amazonKinesis(KinesisProperties properties,
+ ObjectProvider> configurer) {
+
+ return awsClientBuilderConfigurer
+ .configureAsyncClient(KinesisAsyncClient.builder(), properties, null, configurer.stream(), null)
+ .build();
+ }
+
+ @Bean
+ public KinesisStreamProvisioner provisioningProvider(KinesisAsyncClient amazonKinesis) {
+ return new KinesisStreamProvisioner(amazonKinesis, this.configurationProperties);
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ public DynamoDbAsyncClient dynamoDB(DynamoDbProperties properties,
+ ObjectProvider> configurer) {
+
+ if (this.hasInputs) {
+ return awsClientBuilderConfigurer
+ .configureAsyncClient(DynamoDbAsyncClient.builder(), properties, null, configurer.stream(), null)
+ .build();
+ }
+ else {
+ return null;
+ }
+ }
+
+ @Bean
+ @ConditionalOnMissingBean(LockRegistry.class)
+ @ConditionalOnBean(DynamoDbAsyncClient.class)
+ @ConditionalOnProperty(name = "spring.cloud.stream.kinesis.binder.kpl-kcl-enabled", havingValue = "false", matchIfMissing = true)
+ public DynamoDbLockRepository dynamoDbLockRepository(@Autowired(required = false) DynamoDbAsyncClient dynamoDB) {
+ if (dynamoDB != null) {
+ KinesisBinderConfigurationProperties.Locks locks = this.configurationProperties.getLocks();
+ DynamoDbLockRepository dynamoDbLockRepository = new DynamoDbLockRepository(dynamoDB, locks.getTable());
+ dynamoDbLockRepository.setBillingMode(locks.getBillingMode());
+ dynamoDbLockRepository.setReadCapacity(locks.getReadCapacity());
+ dynamoDbLockRepository.setWriteCapacity(locks.getWriteCapacity());
+ return dynamoDbLockRepository;
+ }
+ else {
+ return null;
+ }
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ @ConditionalOnBean(DynamoDbAsyncClient.class)
+ @ConditionalOnProperty(name = "spring.cloud.stream.kinesis.binder.kpl-kcl-enabled", havingValue = "false", matchIfMissing = true)
+ public LockRegistry> dynamoDBLockRegistry(
+ @Autowired(required = false) DynamoDbLockRepository dynamoDbLockRepository) {
+
+ if (dynamoDbLockRepository != null) {
+ KinesisBinderConfigurationProperties.Locks locks = this.configurationProperties.getLocks();
+ DynamoDbLockRegistry dynamoDbLockRegistry = new DynamoDbLockRegistry(dynamoDbLockRepository);
+ dynamoDbLockRegistry.setIdleBetweenTries(locks.getRefreshPeriod());
+ dynamoDbLockRegistry.setTimeToLive(locks.getLeaseDuration());
+ return dynamoDbLockRegistry;
+ }
+ else {
+ return null;
+ }
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ @ConditionalOnBean(DynamoDbAsyncClient.class)
+ @ConditionalOnProperty(name = "spring.cloud.stream.kinesis.binder.kpl-kcl-enabled", havingValue = "false", matchIfMissing = true)
+ public ConcurrentMetadataStore kinesisCheckpointStore(@Autowired(required = false) DynamoDbAsyncClient dynamoDB) {
+ if (dynamoDB != null) {
+ KinesisBinderConfigurationProperties.Checkpoint checkpoint = this.configurationProperties.getCheckpoint();
+ DynamoDbMetadataStore kinesisCheckpointStore = new DynamoDbMetadataStore(dynamoDB, checkpoint.getTable());
+ kinesisCheckpointStore.setBillingMode(checkpoint.getBillingMode());
+ kinesisCheckpointStore.setReadCapacity(checkpoint.getReadCapacity());
+ kinesisCheckpointStore.setWriteCapacity(checkpoint.getWriteCapacity());
+ kinesisCheckpointStore.setCreateTableDelay(checkpoint.getCreateDelay());
+ kinesisCheckpointStore.setCreateTableRetries(checkpoint.getCreateRetries());
+ Integer timeToLive = checkpoint.getTimeToLive();
+ if (timeToLive != null) {
+ kinesisCheckpointStore.setTimeToLive(timeToLive);
+ }
+ return kinesisCheckpointStore;
+ }
+ else {
+ return null;
+ }
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ @ConditionalOnProperty(name = "spring.cloud.stream.kinesis.binder.kpl-kcl-enabled")
+ public CloudWatchAsyncClient cloudWatch(CloudWatchProperties properties,
+ ObjectProvider> configurer) {
+
+ if (this.hasInputs) {
+ return awsClientBuilderConfigurer.configureAsyncClient(CloudWatchAsyncClient.builder(), properties, null,
+ Stream.of(configurer.getIfAvailable()), null).build();
+ }
+ else {
+ return null;
+ }
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ @ConditionalOnProperty(name = "spring.cloud.stream.kinesis.binder.kpl-kcl-enabled")
+ public KinesisProducerConfiguration kinesisProducerConfiguration() {
+ KinesisProducerConfiguration kinesisProducerConfiguration = new KinesisProducerConfiguration();
+ kinesisProducerConfiguration.setCredentialsProvider(this.awsCredentialsProvider);
+ kinesisProducerConfiguration.setRegion(this.region.id());
+ return kinesisProducerConfiguration;
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ public DynamoDbStreamsClient dynamoDBStreams(DynamoDbStreamsProperties properties,
+ ObjectProvider> configurer) {
+
+ if (this.hasInputs) {
+ return awsClientBuilderConfigurer
+ .configureAsyncClient(DynamoDbStreamsClient.builder(), properties, null, configurer.stream(), null)
+ .build();
+ }
+ else {
+ return null;
+ }
+ }
+
+ @Bean
+ public KinesisMessageChannelBinder kinesisMessageChannelBinder(KinesisStreamProvisioner provisioningProvider,
+ KinesisAsyncClient amazonKinesis, KinesisExtendedBindingProperties kinesisExtendedBindingProperties,
+ @Autowired(required = false) ConcurrentMetadataStore kinesisCheckpointStore,
+ @Autowired(required = false) LockRegistry> lockRegistry,
+ @Autowired(required = false) DynamoDbAsyncClient dynamoDBClient,
+ @Autowired(required = false) DynamoDbStreamsClient dynamoDBStreams,
+ @Autowired(required = false) CloudWatchAsyncClient cloudWatchClient,
+ @Autowired(required = false) KinesisProducerConfiguration kinesisProducerConfiguration,
+ @Autowired(required = false) ProducerMessageHandlerCustomizer extends AbstractMessageProducingHandler> producerMessageHandlerCustomizer,
+ @Autowired(required = false) ConsumerEndpointCustomizer extends MessageProducerSupport> consumerEndpointCustomizer,
+ @Autowired ObservationRegistry observationRegistry) {
+
+ KinesisMessageChannelBinder kinesisMessageChannelBinder = new KinesisMessageChannelBinder(
+ this.configurationProperties, provisioningProvider, amazonKinesis, dynamoDBClient, dynamoDBStreams,
+ cloudWatchClient);
+ kinesisMessageChannelBinder.setCheckpointStore(kinesisCheckpointStore);
+ kinesisMessageChannelBinder.setLockRegistry(lockRegistry);
+ kinesisMessageChannelBinder.setExtendedBindingProperties(kinesisExtendedBindingProperties);
+ kinesisMessageChannelBinder.setKinesisProducerConfiguration(kinesisProducerConfiguration);
+ kinesisMessageChannelBinder.setProducerMessageHandlerCustomizer(producerMessageHandlerCustomizer);
+ kinesisMessageChannelBinder.setConsumerEndpointCustomizer(consumerEndpointCustomizer);
+ if (this.configurationProperties.isEnableObservation()) {
+ kinesisMessageChannelBinder.setObservationRegistry(observationRegistry);
+ }
+ return kinesisMessageChannelBinder;
+ }
+
+ @Configuration(proxyBeanMethods = false)
+ @ConditionalOnClass(HealthIndicator.class)
+ @ConditionalOnEnabledHealthIndicator("binders")
+ protected static class KinesisBinderHealthIndicatorConfiguration {
+
+ @Bean
+ @ConditionalOnMissingBean(name = "kinesisBinderHealthIndicator")
+ public KinesisBinderHealthIndicator kinesisBinderHealthIndicator(
+ KinesisMessageChannelBinder kinesisMessageChannelBinder) {
+
+ return new KinesisBinderHealthIndicator(kinesisMessageChannelBinder);
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/KinesisProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/KinesisProperties.java
new file mode 100644
index 000000000..e15cc1787
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/config/KinesisProperties.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.config;
+
+import io.awspring.cloud.autoconfigure.AwsClientProperties;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+
+/**
+ * {@link ConfigurationProperties} for configuring the Kinesis client.
+ *
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@ConfigurationProperties(prefix = "spring.cloud.aws.kinesis")
+public class KinesisProperties extends AwsClientProperties {
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisBinderConfigurationProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisBinderConfigurationProperties.java
new file mode 100644
index 000000000..4a152707a
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisBinderConfigurationProperties.java
@@ -0,0 +1,281 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.properties;
+
+import io.awspring.cloud.dynamodb.DynamoDbLockRepository;
+import io.awspring.cloud.dynamodb.DynamoDbMetadataStore;
+import java.time.Duration;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+
+/**
+ * The Kinesis Binder specific configuration properties.
+ *
+ * @author Peter Oates
+ * @author Artem Bilan
+ * @author Jacob Severson
+ * @author Sergiu Pantiru
+ * @author Arnaud Lecollaire
+ * @author Asiel Caballero
+ *
+ * @since 4.0
+ */
+@ConfigurationProperties(prefix = "spring.cloud.stream.kinesis.binder")
+public class KinesisBinderConfigurationProperties {
+
+ private String[] headers = new String[] {};
+
+ private int describeStreamBackoff = 1000;
+
+ private int describeStreamRetries = 50;
+
+ private boolean autoCreateStream = true;
+
+ private boolean autoAddShards = false;
+
+ private int minShardCount = 1;
+
+ /**
+ * Enables the usage of Amazon KCL/KPL libraries for all message consumption and production.
+ */
+ private boolean kplKclEnabled;
+
+ /**
+ * Enable Micrometer observation registry across all the bindings in the binder.
+ */
+ private boolean enableObservation;
+
+ private final Checkpoint checkpoint = new Checkpoint();
+
+ private final Locks locks = new Locks();
+
+ public String[] getHeaders() {
+ return this.headers;
+ }
+
+ public void setHeaders(String... headers) {
+ this.headers = headers;
+ }
+
+ public int getDescribeStreamBackoff() {
+ return this.describeStreamBackoff;
+ }
+
+ public void setDescribeStreamBackoff(int describeStreamBackoff) {
+ this.describeStreamBackoff = describeStreamBackoff;
+ }
+
+ public int getDescribeStreamRetries() {
+ return this.describeStreamRetries;
+ }
+
+ public void setDescribeStreamRetries(int describeStreamRetries) {
+ this.describeStreamRetries = describeStreamRetries;
+ }
+
+ public boolean isAutoAddShards() {
+ return this.autoAddShards;
+ }
+
+ public void setAutoAddShards(boolean autoAddShards) {
+ this.autoAddShards = autoAddShards;
+ }
+
+ public int getMinShardCount() {
+ return this.minShardCount;
+ }
+
+ public void setMinShardCount(int minShardCount) {
+ this.minShardCount = minShardCount;
+ }
+
+ public Checkpoint getCheckpoint() {
+ return this.checkpoint;
+ }
+
+ public Locks getLocks() {
+ return this.locks;
+ }
+
+ public boolean isAutoCreateStream() {
+ return this.autoCreateStream;
+ }
+
+ public void setAutoCreateStream(boolean autoCreateStream) {
+ this.autoCreateStream = autoCreateStream;
+ }
+
+ public boolean isKplKclEnabled() {
+ return this.kplKclEnabled;
+ }
+
+ public void setKplKclEnabled(boolean kplKclEnabled) {
+ this.kplKclEnabled = kplKclEnabled;
+ }
+
+ public boolean isEnableObservation() {
+ return this.enableObservation;
+ }
+
+ public void setEnableObservation(boolean enableObservation) {
+ this.enableObservation = enableObservation;
+ }
+
+ /**
+ * The checkpoint DynamoDB table configuration properties.
+ */
+ public static class Checkpoint {
+
+ private String table = DynamoDbMetadataStore.DEFAULT_TABLE_NAME;
+
+ private BillingMode billingMode = BillingMode.PAY_PER_REQUEST;
+
+ private long readCapacity = 1L;
+
+ private long writeCapacity = 1L;
+
+ private int createDelay = 1;
+
+ private int createRetries = 25;
+
+ private Integer timeToLive;
+
+ public String getTable() {
+ return this.table;
+ }
+
+ public void setTable(String table) {
+ this.table = table;
+ }
+
+ public BillingMode getBillingMode() {
+ return billingMode;
+ }
+
+ public void setBillingMode(BillingMode billingMode) {
+ this.billingMode = billingMode;
+ }
+
+ public long getReadCapacity() {
+ return this.readCapacity;
+ }
+
+ public void setReadCapacity(long readCapacity) {
+ this.readCapacity = readCapacity;
+ }
+
+ public long getWriteCapacity() {
+ return this.writeCapacity;
+ }
+
+ public void setWriteCapacity(long writeCapacity) {
+ this.writeCapacity = writeCapacity;
+ }
+
+ public int getCreateDelay() {
+ return this.createDelay;
+ }
+
+ public void setCreateDelay(int createDelay) {
+ this.createDelay = createDelay;
+ }
+
+ public int getCreateRetries() {
+ return this.createRetries;
+ }
+
+ public void setCreateRetries(int createRetries) {
+ this.createRetries = createRetries;
+ }
+
+ public Integer getTimeToLive() {
+ return this.timeToLive;
+ }
+
+ public void setTimeToLive(Integer timeToLive) {
+ this.timeToLive = timeToLive;
+ }
+
+ }
+
+ /**
+ * The locks DynamoDB table configuration properties.
+ */
+ public static class Locks {
+
+ private String table = DynamoDbLockRepository.DEFAULT_TABLE_NAME;
+
+ private BillingMode billingMode = BillingMode.PAY_PER_REQUEST;
+
+ private long readCapacity = 1L;
+
+ private long writeCapacity = 1L;
+
+ private Duration refreshPeriod = Duration.ofSeconds(1);
+
+ private Duration leaseDuration = Duration.ofSeconds(60);
+
+ public String getTable() {
+ return this.table;
+ }
+
+ public void setTable(String table) {
+ this.table = table;
+ }
+
+ public BillingMode getBillingMode() {
+ return billingMode;
+ }
+
+ public void setBillingMode(BillingMode billingMode) {
+ this.billingMode = billingMode;
+ }
+
+ public long getReadCapacity() {
+ return this.readCapacity;
+ }
+
+ public void setReadCapacity(long readCapacity) {
+ this.readCapacity = readCapacity;
+ }
+
+ public long getWriteCapacity() {
+ return this.writeCapacity;
+ }
+
+ public void setWriteCapacity(long writeCapacity) {
+ this.writeCapacity = writeCapacity;
+ }
+
+ public Duration getRefreshPeriod() {
+ return this.refreshPeriod;
+ }
+
+ public void setRefreshPeriod(Duration refreshPeriod) {
+ this.refreshPeriod = refreshPeriod;
+ }
+
+ public Duration getLeaseDuration() {
+ return this.leaseDuration;
+ }
+
+ public void setLeaseDuration(Duration leaseDuration) {
+ this.leaseDuration = leaseDuration;
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisBindingProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisBindingProperties.java
new file mode 100644
index 000000000..fc51ed684
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisBindingProperties.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.properties;
+
+import org.springframework.cloud.stream.binder.BinderSpecificPropertiesProvider;
+
+/**
+ * The Kinesis-specific binding configuration properties.
+ *
+ * @author Peter Oates
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+public class KinesisBindingProperties implements BinderSpecificPropertiesProvider {
+
+ private KinesisConsumerProperties consumer = new KinesisConsumerProperties();
+
+ private KinesisProducerProperties producer = new KinesisProducerProperties();
+
+ public KinesisConsumerProperties getConsumer() {
+ return this.consumer;
+ }
+
+ public void setConsumer(KinesisConsumerProperties consumer) {
+ this.consumer = consumer;
+ }
+
+ public KinesisProducerProperties getProducer() {
+ return this.producer;
+ }
+
+ public void setProducer(KinesisProducerProperties producer) {
+ this.producer = producer;
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisConsumerProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisConsumerProperties.java
new file mode 100644
index 000000000..c30a86c06
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisConsumerProperties.java
@@ -0,0 +1,254 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.properties;
+
+import io.awspring.cloud.kinesis.integration.CheckpointMode;
+import io.awspring.cloud.kinesis.integration.ListenerMode;
+import software.amazon.kinesis.metrics.MetricsLevel;
+import software.amazon.kinesis.retrieval.polling.PollingConfig;
+
+/**
+ * The Kinesis-specific consumer binding configuration properties.
+ *
+ * @author Peter Oates
+ * @author Jacob Severson
+ * @author Artem Bilan
+ * @author Arnaud Lecollaire
+ * @author Dmytro Danilenkov
+ * @author Minkyu Moon
+ *
+ * @since 4.0
+ */
+public class KinesisConsumerProperties {
+
+ private int startTimeout = 60000;
+
+ private ListenerMode listenerMode = ListenerMode.record;
+
+ private CheckpointMode checkpointMode = CheckpointMode.batch;
+
+ /**
+ * Interval, in milliseconds, between two checkpoints when checkpoint mode is periodic.
+ */
+ private Long checkpointInterval = 5_000L;
+
+ private int recordsLimit = 10000;
+
+ private int idleBetweenPolls = 1000;
+
+ private int consumerBackoff = 1000;
+
+ private String shardIteratorType;
+
+ private String shardId;
+
+ /**
+ * Worker identifier used to distinguish different workers/processes (only used when KCL is enabled).
+ */
+ private String workerId;
+
+ /**
+ * The KCL fan-out or polling retrieval mode.
+ */
+ private boolean fanOut = true;
+
+ /**
+ * The KCL emptyRecordList option for batch listener mode.
+ */
+ private boolean emptyRecordList = false;
+
+ /**
+ * The KCL table name for leases.
+ */
+ private String leaseTableName;
+
+ /**
+ * The KCL max records for request in polling mode.
+ */
+ private int pollingMaxRecords = PollingConfig.DEFAULT_MAX_RECORDS;
+
+ /**
+ * The KCL idle between requests in polling mode.
+ */
+ private long pollingIdleTime = 1500L;
+
+ /**
+ * The KCL graceful shutdown timeout in milliseconds.
+ */
+ private long gracefulShutdownTimeout;
+
+ private boolean embedHeaders = true;
+
+ private boolean dynamoDbStreams;
+
+ /**
+ * The {@link MetricsLevel} for emitting (or not) metrics into Cloud Watch.
+ */
+ private MetricsLevel metricsLevel = MetricsLevel.DETAILED;
+
+ public int getStartTimeout() {
+ return this.startTimeout;
+ }
+
+ public void setStartTimeout(int startTimeout) {
+ this.startTimeout = startTimeout;
+ }
+
+ public ListenerMode getListenerMode() {
+ return this.listenerMode;
+ }
+
+ public void setListenerMode(ListenerMode listenerMode) {
+ this.listenerMode = listenerMode;
+ }
+
+ public CheckpointMode getCheckpointMode() {
+ return this.checkpointMode;
+ }
+
+ public void setCheckpointMode(CheckpointMode checkpointMode) {
+ this.checkpointMode = checkpointMode;
+ }
+
+ public Long getCheckpointInterval() {
+ return checkpointInterval;
+ }
+
+ public void setCheckpointInterval(Long checkpointInterval) {
+ this.checkpointInterval = checkpointInterval;
+ }
+
+ public int getRecordsLimit() {
+ return this.recordsLimit;
+ }
+
+ public void setRecordsLimit(int recordsLimit) {
+ this.recordsLimit = recordsLimit;
+ }
+
+ public int getIdleBetweenPolls() {
+ return this.idleBetweenPolls;
+ }
+
+ public void setIdleBetweenPolls(int idleBetweenPolls) {
+ this.idleBetweenPolls = idleBetweenPolls;
+ }
+
+ public int getConsumerBackoff() {
+ return this.consumerBackoff;
+ }
+
+ public void setConsumerBackoff(int consumerBackoff) {
+ this.consumerBackoff = consumerBackoff;
+ }
+
+ public String getShardIteratorType() {
+ return this.shardIteratorType;
+ }
+
+ public void setShardIteratorType(String shardIteratorType) {
+ this.shardIteratorType = shardIteratorType;
+ }
+
+ public String getWorkerId() {
+ return workerId;
+ }
+
+ public void setWorkerId(String workerId) {
+ this.workerId = workerId;
+ }
+
+ public String getShardId() {
+ return shardId;
+ }
+
+ public void setShardId(String shardId) {
+ this.shardId = shardId;
+ }
+
+ public boolean isEmbedHeaders() {
+ return this.embedHeaders;
+ }
+
+ public void setEmbedHeaders(boolean embedHeaders) {
+ this.embedHeaders = embedHeaders;
+ }
+
+ public boolean isFanOut() {
+ return this.fanOut;
+ }
+
+ public void setFanOut(boolean fanOut) {
+ this.fanOut = fanOut;
+ }
+
+ public MetricsLevel getMetricsLevel() {
+ return this.metricsLevel;
+ }
+
+ public void setMetricsLevel(MetricsLevel metricsLevel) {
+ this.metricsLevel = metricsLevel;
+ }
+
+ public boolean isEmptyRecordList() {
+ return this.emptyRecordList;
+ }
+
+ public void setEmptyRecordList(boolean emptyRecordList) {
+ this.emptyRecordList = emptyRecordList;
+ }
+
+ public String getLeaseTableName() {
+ return this.leaseTableName;
+ }
+
+ public void setLeaseTableName(String leaseTableName) {
+ this.leaseTableName = leaseTableName;
+ }
+
+ public int getPollingMaxRecords() {
+ return this.pollingMaxRecords;
+ }
+
+ public void setPollingMaxRecords(int pollingMaxRecords) {
+ this.pollingMaxRecords = pollingMaxRecords;
+ }
+
+ public long getPollingIdleTime() {
+ return this.pollingIdleTime;
+ }
+
+ public void setPollingIdleTime(long pollingIdleTime) {
+ this.pollingIdleTime = pollingIdleTime;
+ }
+
+ public long getGracefulShutdownTimeout() {
+ return this.gracefulShutdownTimeout;
+ }
+
+ public void setGracefulShutdownTimeout(long gracefulShutdownTimeout) {
+ this.gracefulShutdownTimeout = gracefulShutdownTimeout;
+ }
+
+ public boolean isDynamoDbStreams() {
+ return this.dynamoDbStreams;
+ }
+
+ public void setDynamoDbStreams(boolean dynamoDbStreams) {
+ this.dynamoDbStreams = dynamoDbStreams;
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisExtendedBindingProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisExtendedBindingProperties.java
new file mode 100644
index 000000000..03e57c2fc
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisExtendedBindingProperties.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.properties;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+import org.springframework.cloud.stream.binder.BinderSpecificPropertiesProvider;
+import org.springframework.cloud.stream.binder.ExtendedBindingProperties;
+
+/**
+ * The extended Kinesis-specific binding configuration properties.
+ *
+ * @author Peter Oates
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@ConfigurationProperties("spring.cloud.stream.kinesis")
+public class KinesisExtendedBindingProperties
+ implements ExtendedBindingProperties {
+
+ private static final String DEFAULTS_PREFIX = "spring.cloud.stream.kinesis.default";
+
+ private Map bindings = new HashMap<>();
+
+ public Map getBindings() {
+ return this.bindings;
+ }
+
+ public void setBindings(Map bindings) {
+ this.bindings = bindings;
+ }
+
+ @Override
+ public KinesisConsumerProperties getExtendedConsumerProperties(String channelName) {
+ if (this.bindings.containsKey(channelName) && this.bindings.get(channelName).getConsumer() != null) {
+ return this.bindings.get(channelName).getConsumer();
+ }
+ else {
+ return new KinesisConsumerProperties();
+ }
+ }
+
+ @Override
+ public KinesisProducerProperties getExtendedProducerProperties(String channelName) {
+ KinesisBindingProperties kinesisBindingProperties = this.bindings.get(channelName);
+ if (kinesisBindingProperties != null && kinesisBindingProperties.getProducer() != null) {
+ return kinesisBindingProperties.getProducer();
+ }
+ else {
+ return new KinesisProducerProperties();
+ }
+ }
+
+ @Override
+ public String getDefaultsPrefix() {
+ return DEFAULTS_PREFIX;
+ }
+
+ @Override
+ public Class extends BinderSpecificPropertiesProvider> getExtendedPropertiesEntryClass() {
+ return KinesisBindingProperties.class;
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisProducerProperties.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisProducerProperties.java
new file mode 100644
index 000000000..9fa87b888
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/properties/KinesisProducerProperties.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.properties;
+
+/**
+ * The Kinesis-specific producer binding configuration properties.
+ *
+ * @author Peter Oates
+ * @author Jacob Severson
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+public class KinesisProducerProperties {
+
+ /**
+ * Whether message handler produces in sync mode.
+ */
+ private boolean sync;
+
+ /**
+ * Timeout in milliseconds to wait for future completion in sync mode.
+ */
+ private long sendTimeout = 10000;
+
+ /**
+ * Whether to embed headers into Kinesis record.
+ */
+ private boolean embedHeaders = true;
+
+ /**
+ * The bean name of a MessageChannel to which successful send results should be sent. Works only for async mode.
+ */
+ private String recordMetadataChannel;
+
+ /**
+ * Maximum records in flight for handling backpressure. No backpressure by default. When backpressure handling is
+ * enabled and number of records in flight exceeds the threshold, a 'KplBackpressureException' would be thrown.
+ */
+ private long kplBackPressureThreshold;
+
+ public void setSync(boolean sync) {
+ this.sync = sync;
+ }
+
+ public boolean isSync() {
+ return this.sync;
+ }
+
+ public long getSendTimeout() {
+ return this.sendTimeout;
+ }
+
+ public void setSendTimeout(long sendTimeout) {
+ this.sendTimeout = sendTimeout;
+ }
+
+ public boolean isEmbedHeaders() {
+ return this.embedHeaders;
+ }
+
+ public void setEmbedHeaders(boolean embedHeaders) {
+ this.embedHeaders = embedHeaders;
+ }
+
+ public String getRecordMetadataChannel() {
+ return this.recordMetadataChannel;
+ }
+
+ public void setRecordMetadataChannel(String recordMetadataChannel) {
+ this.recordMetadataChannel = recordMetadataChannel;
+ }
+
+ public long getKplBackPressureThreshold() {
+ return this.kplBackPressureThreshold;
+ }
+
+ public void setKplBackPressureThreshold(long kplBackPressureThreshold) {
+ this.kplBackPressureThreshold = kplBackPressureThreshold;
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisConsumerDestination.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisConsumerDestination.java
new file mode 100644
index 000000000..ceae1a7f2
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisConsumerDestination.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.provisioning;
+
+import java.util.List;
+import org.springframework.cloud.stream.provisioning.ConsumerDestination;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+/**
+ * The Kinesis-specific {@link ConsumerDestination} implementation.
+ *
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+public final class KinesisConsumerDestination implements ConsumerDestination {
+
+ private final String streamName;
+
+ private final List shards;
+
+ private final String dlqName;
+
+ public KinesisConsumerDestination(String streamName, List shards) {
+ this(streamName, shards, null);
+ }
+
+ public KinesisConsumerDestination(String streamName, List shards, String dlqName) {
+ this.streamName = streamName;
+ this.shards = shards;
+ this.dlqName = dlqName;
+ }
+
+ @Override
+ public String getName() {
+ return this.streamName;
+ }
+
+ public List getShards() {
+ return this.shards;
+ }
+
+ public String getDlqName() {
+ return this.dlqName;
+ }
+
+ @Override
+ public String toString() {
+ return "KinesisConsumerDestination{" + "streamName='" + this.streamName + '\'' + ", shards=" + this.shards
+ + ", dlqName='" + this.dlqName + '\'' + '}';
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisProducerDestination.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisProducerDestination.java
new file mode 100644
index 000000000..2cdbd87b4
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisProducerDestination.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.provisioning;
+
+import java.util.List;
+import org.springframework.cloud.stream.provisioning.ProducerDestination;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+/**
+ * The Kinesis-specific {@link ProducerDestination} implementation.
+ *
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+public final class KinesisProducerDestination implements ProducerDestination {
+
+ private final String streamName;
+
+ private final List shards;
+
+ KinesisProducerDestination(String streamName, List shards) {
+ this.streamName = streamName;
+ this.shards = shards;
+ }
+
+ @Override
+ public String getName() {
+ return this.streamName;
+ }
+
+ @Override
+ public String getNameForPartition(int shard) {
+ return this.streamName;
+ }
+
+ public List getShards() {
+ return this.shards;
+ }
+
+ @Override
+ public String toString() {
+ return "KinesisProducerDestination{" + "streamName='" + this.streamName + '\'' + ", shards=" + this.shards
+ + '}';
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisStreamProvisioner.java b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisStreamProvisioner.java
new file mode 100644
index 000000000..e72db9667
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisStreamProvisioner.java
@@ -0,0 +1,192 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.provisioning;
+
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisBinderConfigurationProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisConsumerProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisProducerProperties;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.springframework.cloud.stream.binder.ExtendedConsumerProperties;
+import org.springframework.cloud.stream.binder.ExtendedProducerProperties;
+import org.springframework.cloud.stream.binder.HeaderMode;
+import org.springframework.cloud.stream.provisioning.ConsumerDestination;
+import org.springframework.cloud.stream.provisioning.ProducerDestination;
+import org.springframework.cloud.stream.provisioning.ProvisioningException;
+import org.springframework.cloud.stream.provisioning.ProvisioningProvider;
+import org.springframework.core.log.LogAccessor;
+import org.springframework.core.log.LogMessage;
+import org.springframework.util.Assert;
+import org.springframework.util.StringUtils;
+import software.amazon.awssdk.core.waiters.WaiterResponse;
+import software.amazon.awssdk.retries.api.BackoffStrategy;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.kinesis.model.ScalingType;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+/**
+ * The {@link ProvisioningProvider} implementation for Amazon Kinesis.
+ *
+ * @author Peter Oates
+ * @author Artem Bilan
+ * @author Jacob Severson
+ * @author Sergiu Pantiru
+ * @author Matthias Wesolowski
+ *
+ * @since 4.0
+ */
+public class KinesisStreamProvisioner implements
+ ProvisioningProvider, ExtendedProducerProperties> {
+
+ private static final LogAccessor logger = new LogAccessor(KinesisStreamProvisioner.class);
+
+ private final KinesisAsyncClient amazonKinesis;
+
+ private final KinesisBinderConfigurationProperties configurationProperties;
+
+ public KinesisStreamProvisioner(KinesisAsyncClient amazonKinesis,
+ KinesisBinderConfigurationProperties kinesisBinderConfigurationProperties) {
+
+ Assert.notNull(amazonKinesis, "'amazonKinesis' must not be null");
+ Assert.notNull(kinesisBinderConfigurationProperties, "'kinesisBinderConfigurationProperties' must not be null");
+ this.amazonKinesis = amazonKinesis;
+ this.configurationProperties = kinesisBinderConfigurationProperties;
+ }
+
+ @Override
+ public ProducerDestination provisionProducerDestination(String name,
+ ExtendedProducerProperties properties) throws ProvisioningException {
+
+ logger.info(() -> "Using Kinesis stream for outbound: " + name);
+
+ if (properties.getExtension().isEmbedHeaders()) {
+ properties.setHeaderMode(HeaderMode.none);
+ }
+
+ return new KinesisProducerDestination(name, createOrUpdate(name, properties.getPartitionCount()));
+ }
+
+ @Override
+ public ConsumerDestination provisionConsumerDestination(String name, String group,
+ ExtendedConsumerProperties properties) throws ProvisioningException {
+
+ KinesisConsumerProperties kinesisConsumerProperties = properties.getExtension();
+ if (kinesisConsumerProperties.isEmbedHeaders()) {
+ properties.setHeaderMode(HeaderMode.none);
+ }
+
+ if (kinesisConsumerProperties.isDynamoDbStreams()) {
+ logger.info(() -> "Using DynamoDB table in DynamoDB Streams support for inbound: " + name);
+ return new KinesisConsumerDestination(name, Collections.emptyList());
+ }
+
+ int shardCount = properties.getInstanceCount() * properties.getConcurrency();
+
+ if (!properties.isMultiplex()) {
+ List shardList = provisionKinesisConsumerDestination(name, shardCount);
+ return new KinesisConsumerDestination(name, shardList);
+ }
+ else {
+ String[] streams = StringUtils.commaDelimitedListToStringArray(name);
+ for (String stream : streams) {
+ provisionKinesisConsumerDestination(stream.trim(), shardCount);
+ }
+ return new KinesisConsumerDestination(name, List.of());
+ }
+ }
+
+ private List provisionKinesisConsumerDestination(String stream, int shards) {
+ logger.info(() -> "Using Kinesis stream for inbound: " + stream);
+
+ return createOrUpdate(stream, shards);
+ }
+
+ private List createOrUpdate(String stream, int shards) {
+ List shardList;
+ try {
+ shardList = getShardList(stream).join();
+ }
+ catch (CompletionException ex) {
+ Throwable cause = ex.getCause();
+ if (cause instanceof ResourceNotFoundException) {
+ if (!this.configurationProperties.isAutoCreateStream()) {
+ throw new ProvisioningException(
+ "The stream [" + stream + "] was not found and auto creation is disabled.", cause);
+ }
+ logger.info(() -> "Stream '" + stream + "' not found. Create one...");
+
+ shardList = createStream(stream, shards);
+ }
+ else {
+ throw new ProvisioningException("Cannot retrieve shards information for stream [" + stream + "].",
+ cause);
+ }
+ }
+
+ int effectiveShardCount = Math.max(this.configurationProperties.getMinShardCount(), shards);
+
+ if ((shardList.size() < effectiveShardCount) && this.configurationProperties.isAutoAddShards()) {
+ return updateShardCount(stream, shardList.size(), effectiveShardCount);
+ }
+
+ return shardList;
+ }
+
+ private CompletableFuture> getShardList(String stream) {
+ return this.amazonKinesis.describeStreamSummary(request -> request.streamName(stream))
+ .thenCompose(reply -> this.amazonKinesis.listShards(request -> request.streamName(stream)))
+ .thenApply(ListShardsResponse::shards);
+ }
+
+ private List createStream(String streamName, int shards) {
+ try {
+ return this.amazonKinesis
+ .createStream(request -> request.streamName(streamName)
+ .shardCount(Math.max(this.configurationProperties.getMinShardCount(), shards)))
+ .thenCompose(reply -> waitForStreamToBecomeActive(streamName))
+ .thenCompose(reply -> getShardList(streamName)).join();
+ }
+ catch (Exception ex) {
+ throw new ProvisioningException("Cannot create stream [" + streamName + "].", ex);
+ }
+ }
+
+ private CompletableFuture> waitForStreamToBecomeActive(String streamName) {
+ return this.amazonKinesis.waiter().waitUntilStreamExists(request -> request.streamName(streamName),
+ waiter -> waiter.maxAttempts(this.configurationProperties.getDescribeStreamRetries())
+ .backoffStrategyV2(BackoffStrategy.fixedDelay(
+ Duration.ofMillis(this.configurationProperties.getDescribeStreamBackoff()))));
+ }
+
+ private List updateShardCount(String streamName, int shardCount, int targetCount) {
+ logger.info(LogMessage.format(
+ "Stream [%s] has [%d] shards compared to a target configuration of [%d], creating shards...",
+ streamName, shardCount, targetCount));
+
+ return this.amazonKinesis
+ .updateShardCount(request -> request.streamName(streamName).targetShardCount(targetCount)
+ .scalingType(ScalingType.UNIFORM_SCALING))
+ .thenCompose(reply -> waitForStreamToBecomeActive(streamName))
+ .thenCompose(reply -> getShardList(streamName)).join();
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/resources/META-INF/spring.binders b/spring-cloud-aws-kinesis-stream-binder/src/main/resources/META-INF/spring.binders
new file mode 100644
index 000000000..7d27a2720
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/resources/META-INF/spring.binders
@@ -0,0 +1 @@
+kinesis: io.awspring.cloud.kinesis.stream.binder.config.KinesisBinderConfiguration
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports b/spring-cloud-aws-kinesis-stream-binder/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports
new file mode 100644
index 000000000..77e2e4542
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports
@@ -0,0 +1 @@
+io.awspring.cloud.kinesis.stream.binder.config.ExtendedBindingHandlerMappingsProviderConfiguration
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderFunctionalTests.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderFunctionalTests.java
new file mode 100644
index 000000000..f1bd3eff8
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderFunctionalTests.java
@@ -0,0 +1,193 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.awspring.cloud.kinesis.integration.KinesisHeaders;
+import io.awspring.cloud.kinesis.integration.KinesisShardOffset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.assertj.core.api.Condition;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.cloud.stream.binder.Binding;
+import org.springframework.cloud.stream.binding.BindingService;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.integration.metadata.ConcurrentMetadataStore;
+import org.springframework.integration.metadata.SimpleMetadataStore;
+import org.springframework.integration.support.MessageBuilder;
+import org.springframework.integration.support.locks.DefaultLockRegistry;
+import org.springframework.integration.support.locks.LockRegistry;
+import org.springframework.integration.test.util.TestUtils;
+import org.springframework.messaging.Message;
+import org.springframework.messaging.support.GenericMessage;
+import org.springframework.test.annotation.DirtiesContext;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+
+/**
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@SpringBootTest(webEnvironment = SpringBootTest.WebEnvironment.NONE, properties = {
+ "spring.cloud.stream.bindings.eventConsumerBatchProcessingWithHeaders-in-0.consumer.multiplex = true",
+ "spring.cloud.stream.bindings.eventConsumerBatchProcessingWithHeaders-in-0.destination = some_other_stream,"
+ + KinesisBinderFunctionalTests.KINESIS_STREAM,
+ "spring.cloud.stream.kinesis.bindings.eventConsumerBatchProcessingWithHeaders-in-0.consumer.idleBetweenPolls = 1",
+ "spring.cloud.stream.kinesis.bindings.eventConsumerBatchProcessingWithHeaders-in-0.consumer.listenerMode = batch",
+ "spring.cloud.stream.kinesis.bindings.eventConsumerBatchProcessingWithHeaders-in-0.consumer.checkpointMode = manual",
+ "spring.cloud.stream.kinesis.binder.headers = event.eventType",
+ "spring.cloud.stream.kinesis.binder.autoAddShards = true" })
+@DirtiesContext
+public class KinesisBinderFunctionalTests implements LocalstackContainerTest {
+
+ static final String KINESIS_STREAM = "test_stream";
+
+ @Autowired
+ private ObjectMapper objectMapper;
+
+ @Autowired
+ private CountDownLatch messageBarrier;
+
+ @Autowired
+ private AtomicReference>> messageHolder;
+
+ @Autowired
+ private BindingService bindingService;
+
+ @SuppressWarnings("unchecked")
+ @Test
+ void testKinesisFunction() throws JsonProcessingException, InterruptedException {
+ List> consumerBindings = this.bindingService
+ .getConsumerBindings("eventConsumerBatchProcessingWithHeaders-in-0");
+
+ assertThat(consumerBindings).hasSize(1);
+
+ Binding> binding = consumerBindings.get(0);
+
+ Map shardConsumers = TestUtils.getPropertyValue(binding, "lifecycle.shardConsumers",
+ Map.class);
+ assertThat(shardConsumers).hasSize(2).hasKeySatisfying(keySatisfyingCondition(KINESIS_STREAM))
+ .hasKeySatisfying(keySatisfyingCondition("some_other_stream"));
+
+ Object shardConsumer = shardConsumers.values().iterator().next();
+
+ await().untilAsserted(
+ () -> assertThat(TestUtils.getPropertyValue(shardConsumer, "state").toString()).isEqualTo("CONSUME"));
+
+ PutRecordsRequest.Builder putRecordsRequest = PutRecordsRequest.builder().streamName(KINESIS_STREAM);
+
+ List putRecordsRequestEntryList = new ArrayList<>();
+
+ for (int i = 0; i < 10; i++) {
+ Message eventMessages = MessageBuilder.withPayload("Message" + i)
+ .setHeader("event.eventType", "createEvent").build();
+ PutRecordsRequestEntry putRecordsRequestEntry = PutRecordsRequestEntry.builder().partitionKey("1")
+ .data(SdkBytes.fromByteArray(objectMapper.writeValueAsBytes(eventMessages))).build();
+ putRecordsRequestEntryList.add(putRecordsRequestEntry);
+ }
+ putRecordsRequest.records(putRecordsRequestEntryList);
+
+ LocalstackContainerTest.kinesisClient().putRecords(putRecordsRequest.build());
+
+ assertThat(this.messageBarrier.await(60, TimeUnit.SECONDS)).isTrue();
+
+ Message> message = this.messageHolder.get();
+ assertThat(message.getHeaders())
+ .containsKeys(KinesisHeaders.CHECKPOINTER, KinesisHeaders.SHARD, KinesisHeaders.RECEIVED_STREAM)
+ .doesNotContainKeys(KinesisHeaders.STREAM, KinesisHeaders.PARTITION_KEY);
+
+ List> payload = message.getPayload();
+ assertThat(payload).hasSize(10);
+
+ Object item = payload.get(0);
+
+ assertThat(item).isInstanceOf(GenericMessage.class);
+
+ Message> messageFromBatch = (Message>) item;
+
+ assertThat(messageFromBatch.getPayload()).isEqualTo("Message0");
+ assertThat(messageFromBatch.getHeaders()).containsEntry("event.eventType", "createEvent");
+ }
+
+ private static Condition keySatisfyingCondition(String streamName) {
+ return new Condition<>() {
+
+ @Override
+ public boolean matches(KinesisShardOffset value) {
+ return value.getStream().equals(streamName);
+ }
+
+ };
+ }
+
+ @Configuration(proxyBeanMethods = false)
+ @EnableAutoConfiguration
+ static class TestConfiguration {
+
+ @Bean
+ public LockRegistry> lockRegistry() {
+ return new DefaultLockRegistry();
+ }
+
+ @Bean
+ public ConcurrentMetadataStore checkpointStore() {
+ return new SimpleMetadataStore();
+ }
+
+ @Bean
+ @SuppressWarnings("removal")
+ public ObjectMapper objectMapper() {
+ return org.springframework.integration.support.json.JacksonJsonUtils.messagingAwareMapper();
+ }
+
+ @Bean
+ public AtomicReference> messageHolder() {
+ return new AtomicReference<>();
+ }
+
+ @Bean
+ public CountDownLatch messageBarrier() {
+ return new CountDownLatch(1);
+ }
+
+ @Bean
+ public Consumer> eventConsumerBatchProcessingWithHeaders(AtomicReference> messageHolder,
+ CountDownLatch messageBarrier) {
+
+ return eventMessages -> {
+ messageHolder.set(eventMessages);
+ messageBarrier.countDown();
+ };
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderTests.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderTests.java
new file mode 100644
index 000000000..12dcd0115
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisBinderTests.java
@@ -0,0 +1,457 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+
+import io.awspring.cloud.kinesis.integration.KclMessageDrivenChannelAdapter;
+import io.awspring.cloud.kinesis.integration.KinesisShardOffset;
+import io.awspring.cloud.kinesis.integration.ListenerMode;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisBinderConfigurationProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisConsumerProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisProducerProperties;
+import java.util.Date;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.mockito.BDDMockito;
+import org.springframework.beans.DirectFieldAccessor;
+import org.springframework.cloud.stream.binder.Binding;
+import org.springframework.cloud.stream.binder.ExtendedConsumerProperties;
+import org.springframework.cloud.stream.binder.ExtendedProducerProperties;
+import org.springframework.cloud.stream.binder.HeaderMode;
+import org.springframework.cloud.stream.binder.PartitionCapableBinderTests;
+import org.springframework.cloud.stream.binder.Spy;
+import org.springframework.cloud.stream.binder.TestUtils;
+import org.springframework.cloud.stream.config.BindingProperties;
+import org.springframework.context.ApplicationContext;
+import org.springframework.context.Lifecycle;
+import org.springframework.expression.common.LiteralExpression;
+import org.springframework.integration.channel.DirectChannel;
+import org.springframework.integration.channel.QueueChannel;
+import org.springframework.integration.support.MessageBuilder;
+import org.springframework.messaging.Message;
+import org.springframework.messaging.MessageChannel;
+import org.springframework.messaging.MessageHandlingException;
+import org.springframework.messaging.MessageHeaders;
+import org.springframework.messaging.SubscribableChannel;
+import org.springframework.messaging.support.ErrorMessage;
+import org.springframework.messaging.support.GenericMessage;
+import org.springframework.util.MimeTypeUtils;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse;
+import software.amazon.awssdk.services.kinesis.model.PutRecordRequest;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.ShardIteratorType;
+import software.amazon.awssdk.services.kinesis.model.StreamStatus;
+import software.amazon.kinesis.common.InitialPositionInStream;
+import software.amazon.kinesis.common.InitialPositionInStreamExtended;
+import software.amazon.kinesis.metrics.MetricsLevel;
+
+/**
+ * The tests for Kinesis Binder.
+ *
+ * @author Artem Bilan
+ * @author Jacob Severson
+ * @author Arnaud Lecollaire
+ * @author Minkyu Moon
+ *
+ * @since 4.0
+ */
+public class KinesisBinderTests extends
+ PartitionCapableBinderTests, ExtendedProducerProperties>
+ implements LocalstackContainerTest {
+
+ private static final String CLASS_UNDER_TEST_NAME = KinesisBinderTests.class.getSimpleName();
+
+ private static KinesisAsyncClient AMAZON_KINESIS;
+
+ private static DynamoDbAsyncClient DYNAMO_DB;
+
+ private static CloudWatchAsyncClient CLOUD_WATCH;
+
+ public KinesisBinderTests() {
+ this.timeoutMultiplier = 10D;
+ }
+
+ @BeforeAll
+ public static void setup() {
+ AMAZON_KINESIS = LocalstackContainerTest.kinesisClient();
+ DYNAMO_DB = LocalstackContainerTest.dynamoDbClient();
+ CLOUD_WATCH = LocalstackContainerTest.cloudWatchClient();
+ }
+
+ @Test
+ @Override
+ public void testClean(TestInfo testInfo) {
+ }
+
+ @Test
+ @Override
+ public void testPartitionedModuleSpEL(TestInfo testInfo) {
+
+ }
+
+ @Test
+ public void testAutoCreateStreamForNonExistingStream() throws Exception {
+ KinesisTestBinder binder = getBinder();
+ DirectChannel output = createBindableChannel("output", new BindingProperties());
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ Date testDate = new Date();
+ consumerProperties.getExtension()
+ .setShardIteratorType(ShardIteratorType.AT_TIMESTAMP.name() + ":" + testDate.getTime());
+ String testStreamName = "nonexisting" + System.currentTimeMillis();
+ Binding> binding = binder.bindConsumer(testStreamName, "test", output, consumerProperties);
+ binding.unbind();
+
+ DescribeStreamResponse streamResult = AMAZON_KINESIS
+ .describeStream(request -> request.streamName(testStreamName)).join();
+ String createdStreamName = streamResult.streamDescription().streamName();
+ int createdShards = streamResult.streamDescription().shards().size();
+ StreamStatus createdStreamStatus = streamResult.streamDescription().streamStatus();
+
+ assertThat(createdStreamName).isEqualTo(testStreamName);
+ assertThat(createdShards)
+ .isEqualTo(consumerProperties.getInstanceCount() * consumerProperties.getConcurrency());
+ assertThat(createdStreamStatus).isEqualTo(StreamStatus.ACTIVE);
+
+ KinesisShardOffset shardOffset = TestUtils.getPropertyValue(binding, "lifecycle.streamInitialSequence",
+ KinesisShardOffset.class);
+ assertThat(shardOffset.getIteratorType()).isEqualTo(ShardIteratorType.AT_TIMESTAMP);
+ assertThat(Date.from(shardOffset.getTimestamp())).isEqualTo(testDate);
+ }
+
+ @Test
+ @Override
+ @SuppressWarnings("unchecked")
+ public void testAnonymousGroup(TestInfo testInfo) throws Exception {
+ KinesisTestBinder binder = getBinder();
+ ExtendedProducerProperties producerProperties = createProducerProperties();
+ producerProperties.setHeaderMode(HeaderMode.none);
+ DirectChannel output = createBindableChannel("output", createProducerBindingProperties(producerProperties));
+
+ Binding producerBinding = binder.bindProducer(
+ String.format("defaultGroup%s0", getDestinationNameDelimiter()), output, producerProperties);
+
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ consumerProperties.setConcurrency(2);
+ consumerProperties.setInstanceCount(3);
+ consumerProperties.setInstanceIndex(0);
+
+ QueueChannel input1 = new QueueChannel();
+ Binding binding1 = binder.bindConsumer(
+ String.format("defaultGroup%s0", getDestinationNameDelimiter()), null, input1, consumerProperties);
+
+ consumerProperties.setInstanceIndex(1);
+
+ QueueChannel input2 = new QueueChannel();
+ Binding binding2 = binder.bindConsumer(
+ String.format("defaultGroup%s0", getDestinationNameDelimiter()), null, input2, consumerProperties);
+
+ String testPayload1 = "foo-" + UUID.randomUUID().toString();
+ output.send(MessageBuilder.withPayload(testPayload1)
+ .setHeader(MessageHeaders.CONTENT_TYPE, MimeTypeUtils.TEXT_PLAIN).build());
+
+ Message receivedMessage1 = (Message) receive(input1, 10);
+ assertThat(receivedMessage1).isNotNull();
+ assertThat(new String(receivedMessage1.getPayload())).isEqualTo(testPayload1);
+
+ Message receivedMessage2 = (Message) receive(input2, 10);
+ assertThat(receivedMessage2).isNotNull();
+ assertThat(new String(receivedMessage2.getPayload())).isEqualTo(testPayload1);
+
+ binding2.unbind();
+
+ String testPayload2 = "foo-" + UUID.randomUUID();
+ output.send(MessageBuilder.withPayload(testPayload2)
+ .setHeader(MessageHeaders.CONTENT_TYPE, MimeTypeUtils.TEXT_PLAIN).build());
+
+ binding2 = binder.bindConsumer(String.format("defaultGroup%s0", getDestinationNameDelimiter()), null, input2,
+ consumerProperties);
+ String testPayload3 = "foo-" + UUID.randomUUID();
+ output.send(MessageBuilder.withPayload(testPayload3)
+ .setHeader(MessageHeaders.CONTENT_TYPE, MimeTypeUtils.TEXT_PLAIN).build());
+
+ receivedMessage1 = (Message) receive(input1, 10);
+ assertThat(receivedMessage1).isNotNull();
+ assertThat(new String(receivedMessage1.getPayload())).isEqualTo(testPayload2);
+ receivedMessage1 = (Message) receive(input1, 10);
+ assertThat(receivedMessage1).isNotNull();
+ assertThat(new String(receivedMessage1.getPayload())).isNotNull();
+
+ receivedMessage2 = (Message) receive(input2, 10);
+ assertThat(receivedMessage2).isNotNull();
+ assertThat(new String(receivedMessage2.getPayload())).isEqualTo(testPayload1);
+
+ receivedMessage2 = (Message) receive(input2, 10);
+ assertThat(receivedMessage2).isNotNull();
+ assertThat(new String(receivedMessage2.getPayload())).isEqualTo(testPayload2);
+
+ receivedMessage2 = (Message) receive(input2, 10);
+ assertThat(receivedMessage2).isNotNull();
+ assertThat(new String(receivedMessage2.getPayload())).isEqualTo(testPayload3);
+
+ producerBinding.unbind();
+ binding1.unbind();
+ binding2.unbind();
+ }
+
+ @Test
+ // @Disabled
+ public void testProducerErrorChannel() throws Exception {
+ KinesisTestBinder binder = getBinder();
+
+ final RuntimeException putRecordException = new RuntimeException("putRecordRequestEx");
+ KinesisAsyncClient amazonKinesisMock = mock(KinesisAsyncClient.class);
+ BDDMockito.given(amazonKinesisMock.putRecord(any(PutRecordRequest.class)))
+ .willAnswer((invocation) -> CompletableFuture.failedFuture(putRecordException));
+
+ new DirectFieldAccessor(binder.getBinder()).setPropertyValue("amazonKinesis", amazonKinesisMock);
+
+ ExtendedProducerProperties producerProps = createProducerProperties();
+ producerProps.getExtension().setSync(false);
+ producerProps.setErrorChannelEnabled(true);
+ producerProps.populateBindingName("foobar");
+ DirectChannel moduleOutputChannel = createBindableChannel("output",
+ createProducerBindingProperties(producerProps));
+ Binding producerBinding = binder.bindProducer("ec.0", moduleOutputChannel, producerProps);
+
+ ApplicationContext applicationContext = TestUtils.getPropertyValue(binder.getBinder(), "applicationContext",
+ ApplicationContext.class);
+ String errorChannelName = testBinder.getBinder().getBinderIdentity() + "." + producerProps.getBindingName()
+ + ".errors";
+ SubscribableChannel ec = applicationContext.getBean(errorChannelName, SubscribableChannel.class);
+ final AtomicReference> errorMessage = new AtomicReference<>();
+ final CountDownLatch latch = new CountDownLatch(1);
+ ec.subscribe((message) -> {
+ errorMessage.set(message);
+ latch.countDown();
+ });
+
+ String messagePayload = "oops";
+ moduleOutputChannel.send(new GenericMessage<>(messagePayload.getBytes()));
+
+ assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue();
+ assertThat(errorMessage.get()).isInstanceOf(ErrorMessage.class);
+ assertThat(errorMessage.get().getPayload()).isInstanceOf(MessageHandlingException.class);
+ MessageHandlingException exception = (MessageHandlingException) errorMessage.get().getPayload();
+ assertThat(exception.getCause()).isSameAs(putRecordException);
+ producerBinding.unbind();
+ }
+
+ @Test
+ @SuppressWarnings("unchecked")
+ public void testBatchListener() throws Exception {
+ KinesisTestBinder binder = getBinder();
+ ExtendedProducerProperties producerProperties = createProducerProperties();
+ producerProperties.setHeaderMode(HeaderMode.none);
+ DirectChannel output = createBindableChannel("output", createProducerBindingProperties(producerProperties));
+
+ Binding outputBinding = binder.bindProducer("testBatchListener", output, producerProperties);
+
+ for (int i = 0; i < 3; i++) {
+ output.send(new GenericMessage<>(i));
+ }
+
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ consumerProperties.getExtension().setListenerMode(ListenerMode.batch);
+ consumerProperties.setUseNativeDecoding(true);
+ consumerProperties.setHeaderMode(HeaderMode.none);
+
+ QueueChannel input = new QueueChannel();
+ Binding inputBinding = binder.bindConsumer("testBatchListener", null, input,
+ consumerProperties);
+
+ Message> receivedMessage = (Message>) receive(input, 10);
+ assertThat(receivedMessage).isNotNull();
+ assertThat(receivedMessage.getPayload().size()).isEqualTo(3);
+
+ receivedMessage.getPayload().forEach((r) -> {
+ assertThat(r).isInstanceOf(Record.class);
+ });
+
+ outputBinding.unbind();
+ inputBinding.unbind();
+ }
+
+ @Test
+ public void testKclWithTimestampAtInitialPositionInStream() throws Exception {
+ KinesisBinderConfigurationProperties configurationProperties = new KinesisBinderConfigurationProperties();
+ configurationProperties.setKplKclEnabled(true);
+ KinesisTestBinder binder = getBinder(configurationProperties);
+ DirectChannel output = createBindableChannel("output", new BindingProperties());
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ consumerProperties.setAutoStartup(false);
+ Date testDate = new Date();
+ consumerProperties.getExtension()
+ .setShardIteratorType(ShardIteratorType.AT_TIMESTAMP.name() + ":" + testDate.getTime());
+ Binding> binding = binder.bindConsumer("testKclStream", "test", output, consumerProperties);
+
+ Lifecycle lifecycle = TestUtils.getPropertyValue(binding, "lifecycle", Lifecycle.class);
+ assertThat(lifecycle).isInstanceOf(KclMessageDrivenChannelAdapter.class);
+
+ InitialPositionInStreamExtended initialSequence = TestUtils.getPropertyValue(lifecycle, "streamInitialSequence",
+ InitialPositionInStreamExtended.class);
+
+ assertThat(initialSequence).isEqualTo(InitialPositionInStreamExtended.newInitialPositionAtTimestamp(testDate));
+
+ binding.unbind();
+ }
+
+ @Test
+ public void testKclWithTrimHorizonInitialPositionInStream() throws Exception {
+ KinesisBinderConfigurationProperties configurationProperties = new KinesisBinderConfigurationProperties();
+ configurationProperties.setKplKclEnabled(true);
+ KinesisTestBinder binder = getBinder(configurationProperties);
+ DirectChannel output = createBindableChannel("output", new BindingProperties());
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ consumerProperties.setAutoStartup(false);
+ consumerProperties.getExtension().setFanOut(false);
+ Binding> binding = binder.bindConsumer("testKclStream", null, output, consumerProperties);
+
+ Lifecycle lifecycle = TestUtils.getPropertyValue(binding, "lifecycle", Lifecycle.class);
+ assertThat(lifecycle).isInstanceOf(KclMessageDrivenChannelAdapter.class);
+
+ InitialPositionInStreamExtended initialSequence = TestUtils.getPropertyValue(lifecycle, "streamInitialSequence",
+ InitialPositionInStreamExtended.class);
+
+ assertThat(initialSequence)
+ .isEqualTo(InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON));
+
+ assertThat(TestUtils.getPropertyValue(lifecycle, "fanOut", Boolean.class)).isFalse();
+
+ binding.unbind();
+ }
+
+ @Test
+ public void testMetricsLevelOfKcl() throws Exception {
+ KinesisBinderConfigurationProperties configurationProperties = new KinesisBinderConfigurationProperties();
+ configurationProperties.setKplKclEnabled(true);
+ KinesisTestBinder binder = getBinder(configurationProperties);
+ DirectChannel output = createBindableChannel("output", new BindingProperties());
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ consumerProperties.setAutoStartup(false);
+ consumerProperties.getExtension().setMetricsLevel(MetricsLevel.NONE);
+ Binding> binding = binder.bindConsumer("testKclStream", null, output, consumerProperties);
+
+ Lifecycle lifecycle = TestUtils.getPropertyValue(binding, "lifecycle", Lifecycle.class);
+ assertThat(lifecycle).isInstanceOf(KclMessageDrivenChannelAdapter.class);
+
+ assertThat(TestUtils.getPropertyValue(lifecycle, "metricsLevel", MetricsLevel.class))
+ .isEqualTo(MetricsLevel.NONE);
+
+ binding.unbind();
+ }
+
+ @Test
+ public void testPartitionCountIncreasedIfAutoAddPartitionsSet() throws Exception {
+ KinesisBinderConfigurationProperties configurationProperties = new KinesisBinderConfigurationProperties();
+
+ String stream = "existing" + System.currentTimeMillis();
+
+ AMAZON_KINESIS.createStream(request -> request.streamName(stream).shardCount(2)).join();
+
+ List shards = describeStream(stream);
+
+ assertThat(shards.size()).isEqualTo(2);
+
+ configurationProperties.setMinShardCount(4);
+ configurationProperties.setAutoAddShards(true);
+ KinesisTestBinder binder = getBinder(configurationProperties);
+
+ ExtendedConsumerProperties consumerProperties = createConsumerProperties();
+ DirectChannel output = createBindableChannel("output", new BindingProperties());
+ Binding> binding = binder.bindConsumer(stream, "test", output, consumerProperties);
+ binding.unbind();
+
+ shards = describeStream(stream);
+
+ assertThat(shards.size()).isEqualTo(6);
+ }
+
+ private List describeStream(String stream) {
+ return AMAZON_KINESIS.describeStream(request -> request.streamName(stream))
+ .thenCompose(
+ reply -> AMAZON_KINESIS.waiter().waitUntilStreamExists(request -> request.streamName(stream)))
+ .join().matched().response().get().streamDescription().shards();
+ }
+
+ @Override
+ protected boolean usesExplicitRouting() {
+ return false;
+ }
+
+ @Override
+ protected String getClassUnderTestName() {
+ return CLASS_UNDER_TEST_NAME;
+ }
+
+ @Override
+ protected KinesisTestBinder getBinder() {
+ return getBinder(new KinesisBinderConfigurationProperties());
+ }
+
+ private KinesisTestBinder getBinder(KinesisBinderConfigurationProperties kinesisBinderConfigurationProperties) {
+ if (this.testBinder == null) {
+ this.testBinder = new KinesisTestBinder(AMAZON_KINESIS, DYNAMO_DB, CLOUD_WATCH,
+ kinesisBinderConfigurationProperties);
+ this.timeoutMultiplier = 20;
+ }
+ return this.testBinder;
+ }
+
+ @Override
+ protected ExtendedConsumerProperties createConsumerProperties() {
+ ExtendedConsumerProperties kinesisConsumerProperties = new ExtendedConsumerProperties<>(
+ new KinesisConsumerProperties());
+ // set the default values that would normally be propagated by Spring Cloud Stream
+ kinesisConsumerProperties.setInstanceCount(1);
+ kinesisConsumerProperties.setInstanceIndex(0);
+ kinesisConsumerProperties.getExtension().setShardIteratorType(ShardIteratorType.TRIM_HORIZON.name());
+ kinesisConsumerProperties.getExtension().setIdleBetweenPolls(10);
+ return kinesisConsumerProperties;
+ }
+
+ private ExtendedProducerProperties createProducerProperties() {
+ return createProducerProperties(null);
+ }
+
+ @Override
+ protected ExtendedProducerProperties createProducerProperties(TestInfo testInto) {
+ ExtendedProducerProperties producerProperties = new ExtendedProducerProperties<>(
+ new KinesisProducerProperties());
+ producerProperties.setPartitionKeyExpression(new LiteralExpression("1"));
+ producerProperties.getExtension().setSync(true);
+ return producerProperties;
+ }
+
+ @Override
+ public Spy spyOn(String name) {
+ throw new UnsupportedOperationException("'spyOn' is not used by Kinesis tests");
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisTestBinder.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisTestBinder.java
new file mode 100644
index 000000000..519a8b4ad
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/KinesisTestBinder.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder;
+
+import io.awspring.cloud.kinesis.integration.KinesisMessageDrivenChannelAdapter;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisBinderConfigurationProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisConsumerProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisProducerProperties;
+import io.awspring.cloud.kinesis.stream.binder.provisioning.KinesisStreamProvisioner;
+import java.util.concurrent.CompletableFuture;
+import org.springframework.beans.DirectFieldAccessor;
+import org.springframework.cloud.stream.binder.AbstractTestBinder;
+import org.springframework.cloud.stream.binder.ExtendedConsumerProperties;
+import org.springframework.cloud.stream.binder.ExtendedProducerProperties;
+import org.springframework.cloud.stream.binder.PartitionTestSupport;
+import org.springframework.cloud.stream.provisioning.ConsumerDestination;
+import org.springframework.context.annotation.AnnotationConfigApplicationContext;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.context.support.GenericApplicationContext;
+import org.springframework.integration.config.EnableIntegration;
+import org.springframework.integration.core.MessageProducer;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+
+/**
+ * An {@link AbstractTestBinder} implementation for the {@link KinesisMessageChannelBinder}.
+ *
+ * @author Artem Bilan
+ * @author Arnaud Lecollaire
+ *
+ * @since 4.0
+ */
+public class KinesisTestBinder extends
+ AbstractTestBinder, ExtendedProducerProperties> {
+
+ private final KinesisAsyncClient amazonKinesis;
+
+ private final GenericApplicationContext applicationContext;
+
+ public KinesisTestBinder(KinesisAsyncClient amazonKinesis, DynamoDbAsyncClient dynamoDbClient,
+ CloudWatchAsyncClient cloudWatchClient,
+ KinesisBinderConfigurationProperties kinesisBinderConfigurationProperties) {
+
+ this.applicationContext = new AnnotationConfigApplicationContext(Config.class);
+
+ this.amazonKinesis = amazonKinesis;
+
+ KinesisStreamProvisioner provisioningProvider = new KinesisStreamProvisioner(amazonKinesis,
+ kinesisBinderConfigurationProperties);
+
+ KinesisMessageChannelBinder binder = new TestKinesisMessageChannelBinder(amazonKinesis, dynamoDbClient,
+ cloudWatchClient, kinesisBinderConfigurationProperties, provisioningProvider);
+
+ binder.setApplicationContext(this.applicationContext);
+
+ setBinder(binder);
+ }
+
+ public GenericApplicationContext getApplicationContext() {
+ return this.applicationContext;
+ }
+
+ @Override
+ public void cleanup() {
+ this.amazonKinesis.listStreams()
+ .thenCompose(reply -> CompletableFuture.allOf(reply.streamNames().stream()
+ .map(streamName -> this.amazonKinesis.deleteStream(request -> request.streamName(streamName))
+ .thenCompose(result -> this.amazonKinesis.waiter()
+ .waitUntilStreamNotExists(request -> request.streamName(streamName))))
+ .toArray(CompletableFuture[]::new)))
+ .join();
+ }
+
+ /**
+ * Test configuration.
+ */
+ @Configuration
+ @EnableIntegration
+ static class Config {
+
+ @Bean
+ public PartitionTestSupport partitionSupport() {
+ return new PartitionTestSupport();
+ }
+
+ }
+
+ private static class TestKinesisMessageChannelBinder extends KinesisMessageChannelBinder {
+
+ TestKinesisMessageChannelBinder(KinesisAsyncClient amazonKinesis, DynamoDbAsyncClient dynamoDbClient,
+ CloudWatchAsyncClient cloudWatchClient,
+ KinesisBinderConfigurationProperties kinesisBinderConfigurationProperties,
+ KinesisStreamProvisioner provisioningProvider) {
+
+ super(kinesisBinderConfigurationProperties, provisioningProvider, amazonKinesis, dynamoDbClient, null,
+ cloudWatchClient);
+ }
+
+ /*
+ * Some tests use multiple instance indexes for the same topic; we need to make the error infrastructure beans
+ * unique.
+ */
+ @Override
+ protected String errorsBaseName(ConsumerDestination destination, String group,
+ ExtendedConsumerProperties consumerProperties) {
+ return super.errorsBaseName(destination, group, consumerProperties) + "-"
+ + consumerProperties.getInstanceIndex();
+ }
+
+ @Override
+ protected MessageProducer createConsumerEndpoint(ConsumerDestination destination, String group,
+ ExtendedConsumerProperties properties) {
+
+ MessageProducer messageProducer = super.createConsumerEndpoint(destination, group, properties);
+ if (messageProducer instanceof KinesisMessageDrivenChannelAdapter) {
+ DirectFieldAccessor dfa = new DirectFieldAccessor(messageProducer);
+ dfa.setPropertyValue("describeStreamBackoff", 10);
+ dfa.setPropertyValue("consumerBackoff", 10);
+ dfa.setPropertyValue("idleBetweenPolls", 1);
+ }
+ return messageProducer;
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/LocalstackContainerTest.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/LocalstackContainerTest.java
new file mode 100644
index 000000000..df1438a66
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/LocalstackContainerTest.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder;
+
+import org.junit.jupiter.api.BeforeAll;
+import org.testcontainers.containers.localstack.LocalStackContainer;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+
+/**
+ * The base contract for JUnit tests based on the container for Localstack. The Testcontainers 'reuse' option must be
+ * disabled, so, Ryuk container is started and will clean all the containers up from this test suite after JVM exit.
+ * Since the Localstack container instance is shared via static property, it is going to be started only once per JVM;
+ * therefore, the target Docker container is reused automatically.
+ *
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@Testcontainers(disabledWithoutDocker = true)
+public interface LocalstackContainerTest {
+
+ LocalStackContainer LOCAL_STACK_CONTAINER = new LocalStackContainer(
+ DockerImageName.parse("localstack/localstack:4.4.0"));
+
+ @BeforeAll
+ static void startContainer() {
+ LOCAL_STACK_CONTAINER.start();
+ System.setProperty("spring.cloud.aws.region.static", LOCAL_STACK_CONTAINER.getRegion());
+ System.setProperty("spring.cloud.aws.endpoint", LOCAL_STACK_CONTAINER.getEndpoint().toString());
+ System.setProperty("spring.cloud.aws.credentials.access-key", LOCAL_STACK_CONTAINER.getAccessKey());
+ System.setProperty("spring.cloud.aws.credentials.secret-key", LOCAL_STACK_CONTAINER.getSecretKey());
+ }
+
+ static KinesisAsyncClient kinesisClient() {
+ return applyAwsClientOptions(KinesisAsyncClient.builder().httpClientBuilder(NettyNioAsyncHttpClient.builder()));
+ }
+
+ static DynamoDbAsyncClient dynamoDbClient() {
+ return applyAwsClientOptions(DynamoDbAsyncClient.builder());
+ }
+
+ static DynamoDbStreamsClient dynamoDbStreamsClient() {
+ return applyAwsClientOptions(DynamoDbStreamsClient.builder());
+ }
+
+ static CloudWatchAsyncClient cloudWatchClient() {
+ return applyAwsClientOptions(CloudWatchAsyncClient.builder());
+ }
+
+ static AwsCredentialsProvider credentialsProvider() {
+ return StaticCredentialsProvider.create(
+ AwsBasicCredentials.create(LOCAL_STACK_CONTAINER.getAccessKey(), LOCAL_STACK_CONTAINER.getSecretKey()));
+ }
+
+ private static , T> T applyAwsClientOptions(B clientBuilder) {
+ return clientBuilder.region(Region.of(LOCAL_STACK_CONTAINER.getRegion()))
+ .credentialsProvider(credentialsProvider()).endpointOverride(LOCAL_STACK_CONTAINER.getEndpoint())
+ .build();
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/dynamodb/DynamoDbStreamsIntegrationTests.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/dynamodb/DynamoDbStreamsIntegrationTests.java
new file mode 100644
index 000000000..884f04278
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/dynamodb/DynamoDbStreamsIntegrationTests.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.dynamodb;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
+import io.awspring.cloud.kinesis.stream.binder.LocalstackContainerTest;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.cloud.stream.binder.Binding;
+import org.springframework.cloud.stream.binding.BindingService;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.integration.metadata.ConcurrentMetadataStore;
+import org.springframework.integration.metadata.SimpleMetadataStore;
+import org.springframework.integration.support.locks.DefaultLockRegistry;
+import org.springframework.integration.support.locks.LockRegistry;
+import org.springframework.integration.test.util.TestUtils;
+import org.springframework.test.annotation.DirtiesContext;
+import software.amazon.awssdk.retries.api.BackoffStrategy;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
+import software.amazon.awssdk.services.dynamodb.model.StreamViewType;
+
+/**
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@SpringBootTest(webEnvironment = SpringBootTest.WebEnvironment.NONE, properties = "spring.cloud.stream.kinesis.bindings.dynamoDbConsumer-in-0.consumer.dynamoDbStreams=true")
+@DirtiesContext
+public class DynamoDbStreamsIntegrationTests implements LocalstackContainerTest {
+
+ static final String TEST_TABLE = "StreamsBinderDemoTable";
+
+ static final String TABLE_KEY = "id";
+
+ static DynamoDbAsyncClient DYNAMODB;
+
+ static String DYNAMODB_STREAM_ARN;
+
+ @Autowired
+ private BindingService bindingService;
+
+ @Autowired
+ private CountDownLatch messageBarrier;
+
+ @Autowired
+ private AtomicReference messageHolder;
+
+ @BeforeAll
+ static void setup() {
+ DYNAMODB = LocalstackContainerTest.dynamoDbClient();
+ DYNAMODB_STREAM_ARN = createDemoTable();
+ AmazonDynamoDBStreamsAdapterClient streamsAdapterClient = new AmazonDynamoDBStreamsAdapterClient(
+ LocalstackContainerTest.dynamoDbStreamsClient(), null);
+
+ await().atMost(Duration.ofMinutes(2))
+ .untilAsserted(() -> assertThat(
+ streamsAdapterClient.describeStream(builder -> builder.streamName(DYNAMODB_STREAM_ARN)))
+ .succeedsWithin(Duration.ofSeconds(60))
+ .extracting(describeStreamResponse -> describeStreamResponse.streamDescription()
+ .streamStatusAsString())
+ .isEqualTo("ENABLED"));
+
+ System.setProperty("spring.cloud.stream.bindings.dynamoDbConsumer-in-0.destination", DYNAMODB_STREAM_ARN);
+ }
+
+ private static String createDemoTable() {
+ CreateTableRequest createTableRequest = CreateTableRequest.builder()
+ .attributeDefinitions(AttributeDefinition.builder().attributeName(TABLE_KEY)
+ .attributeType(ScalarAttributeType.S).build())
+ .keySchema(KeySchemaElement.builder().attributeName(TABLE_KEY).keyType(KeyType.HASH).build())
+ .billingMode(BillingMode.PAY_PER_REQUEST).tableName(TEST_TABLE)
+ .streamSpecification(
+ builder -> builder.streamEnabled(true).streamViewType(StreamViewType.NEW_AND_OLD_IMAGES))
+ .build();
+
+ return DYNAMODB.createTable(createTableRequest)
+ .thenCompose(result -> DYNAMODB.waiter().waitUntilTableExists(request -> request.tableName(TEST_TABLE),
+ waiter -> waiter.maxAttempts(10)
+ .backoffStrategyV2(BackoffStrategy.fixedDelay(Duration.ofSeconds(1)))))
+ .thenApply(waiterResponse -> waiterResponse.matched().response()
+ .map(describeTableResponse -> describeTableResponse.table().latestStreamArn()))
+ .join().get();
+ }
+
+ @Test
+ void fromDynamoDbStreamToKinesisBinder() throws InterruptedException {
+ List> consumerBindings = this.bindingService.getConsumerBindings("dynamoDbConsumer-in-0");
+
+ assertThat(consumerBindings).hasSize(1);
+
+ Binding> binding = consumerBindings.get(0);
+
+ Map, ?> shardConsumers = TestUtils.getPropertyValue(binding, "lifecycle.shardConsumers", Map.class);
+ assertThat(shardConsumers).hasSize(1);
+
+ Object shardConsumer = shardConsumers.values().iterator().next();
+
+ await().untilAsserted(
+ () -> assertThat(TestUtils.getPropertyValue(shardConsumer, "state").toString()).isEqualTo("CONSUME"));
+
+ DYNAMODB.putItem(builder -> builder.tableName(TEST_TABLE).item(
+ Map.of(TABLE_KEY, AttributeValue.fromS("some_id"), "Message", AttributeValue.fromS("some value"))))
+ .join();
+
+ assertThat(this.messageBarrier.await(60, TimeUnit.SECONDS)).isTrue();
+ assertThat(this.messageHolder.get()).contains("some_id", "some value", "\"eventName\":\"INSERT\"",
+ "\"eventSource\":\"aws:dynamodb\"");
+ }
+
+ @Configuration(proxyBeanMethods = false)
+ @EnableAutoConfiguration
+ static class TestConfiguration {
+
+ @Bean
+ public LockRegistry> lockRegistry() {
+ return new DefaultLockRegistry();
+ }
+
+ @Bean
+ public ConcurrentMetadataStore checkpointStore() {
+ return new SimpleMetadataStore();
+ }
+
+ @Bean
+ public AtomicReference messageHolder() {
+ return new AtomicReference<>();
+ }
+
+ @Bean
+ public CountDownLatch messageBarrier() {
+ return new CountDownLatch(1);
+ }
+
+ @Bean
+ public Consumer dynamoDbConsumer(AtomicReference messageHolder, CountDownLatch messageBarrier) {
+ return eventMessages -> {
+ messageHolder.set(eventMessages);
+ messageBarrier.countDown();
+ };
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/observation/KinesisBinderObservationTests.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/observation/KinesisBinderObservationTests.java
new file mode 100644
index 000000000..665367897
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/observation/KinesisBinderObservationTests.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.observation;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import brave.handler.SpanHandler;
+import brave.test.TestSpanHandler;
+import io.awspring.cloud.kinesis.integration.KinesisHeaders;
+import io.awspring.cloud.kinesis.stream.binder.LocalstackContainerTest;
+import io.micrometer.tracing.Span;
+import io.micrometer.tracing.brave.bridge.BraveFinishedSpan;
+import io.micrometer.tracing.test.simple.SpansAssert;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
+import org.springframework.boot.micrometer.tracing.test.autoconfigure.AutoConfigureTracing;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.integration.channel.QueueChannel;
+import org.springframework.integration.config.GlobalChannelInterceptor;
+import org.springframework.integration.metadata.ConcurrentMetadataStore;
+import org.springframework.integration.metadata.SimpleMetadataStore;
+import org.springframework.integration.support.locks.DefaultLockRegistry;
+import org.springframework.integration.support.locks.LockRegistry;
+import org.springframework.integration.support.management.observation.IntegrationObservation;
+import org.springframework.messaging.Message;
+import org.springframework.messaging.MessageChannel;
+import org.springframework.messaging.support.ChannelInterceptor;
+import org.springframework.messaging.support.GenericMessage;
+import org.springframework.test.annotation.DirtiesContext;
+import reactor.core.publisher.Flux;
+
+/**
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@SpringBootTest(webEnvironment = SpringBootTest.WebEnvironment.NONE, properties = {
+ "spring.cloud.function.definition=kinesisSupplier;kinesisConsumer",
+ "spring.cloud.stream.bindings.kinesisSupplier-out-0.destination="
+ + KinesisBinderObservationTests.KINESIS_STREAM,
+ "spring.cloud.stream.kinesis.bindings.kinesisSupplier-out-0.producer.recordMetadataChannel=producerResultsChannel",
+ "spring.cloud.stream.bindings.kinesisConsumer-in-0.destination=" + KinesisBinderObservationTests.KINESIS_STREAM,
+ "spring.cloud.stream.bindings.kinesisConsumer-in-0.group=observation-group",
+ "spring.cloud.stream.kinesis.binder.enable-observation=true",
+ "logging.level.org.springframework.cloud.stream.binder.kinesis.observation=debug",
+ "management.tracing.sampling.probability=1.0" })
+@AutoConfigureTracing
+@DirtiesContext
+public class KinesisBinderObservationTests implements LocalstackContainerTest {
+
+ private static final Log LOGGER = LogFactory.getLog(KinesisBinderObservationTests.class);
+
+ static final String KINESIS_STREAM = "test_observation_stream";
+
+ private static final TestSpanHandler SPANS = new TestSpanHandler();
+
+ @Autowired
+ private CountDownLatch messageBarrier;
+
+ @Autowired
+ private AtomicReference> messageHolder;
+
+ @Autowired
+ QueueChannel producerResultsChannel;
+
+ @Test
+ void observationIsPropagatedFromSupplierToConsumer() throws InterruptedException {
+ assertThat(this.messageBarrier.await(30, TimeUnit.SECONDS)).isTrue();
+ Message message = this.messageHolder.get();
+ assertThat(message.getHeaders()).containsKeys("traceparent");
+ assertThat(message.getPayload()).isEqualTo("test data");
+
+ await().until(() -> SPANS.spans().size() == 3);
+ SpansAssert.assertThat(SPANS.spans().stream().map(BraveFinishedSpan::fromBrave).toList()).haveSameTraceId()
+ .hasASpanWithName("kinesisSupplier-out-0 send",
+ spanAssert -> spanAssert.hasKindEqualTo(Span.Kind.PRODUCER)
+ .hasTag(IntegrationObservation.ProducerTags.COMPONENT_TYPE, "producer"))
+ .hasASpanWithName(String.format("Consumer for [%s] receive", KINESIS_STREAM),
+ spanAssert -> spanAssert.hasKindEqualTo(Span.Kind.CONSUMER)
+ .hasTag(IntegrationObservation.HandlerTags.COMPONENT_TYPE, "message-producer"))
+ .hasASpanWithName("kinesisConsumer process"); // Function part
+
+ Message> producerResultMessage = producerResultsChannel.receive(10_000);
+ assertThat(producerResultMessage.getPayload()).isEqualTo("test data".getBytes());
+ assertThat(producerResultMessage.getHeaders()).containsKeys(KinesisHeaders.SHARD,
+ KinesisHeaders.SEQUENCE_NUMBER);
+ }
+
+ @Configuration(proxyBeanMethods = false)
+ @EnableAutoConfiguration
+ static class TestConfiguration {
+
+ @Bean
+ SpanHandler spanHandler() {
+ return SPANS;
+ }
+
+ @Bean
+ public LockRegistry> lockRegistry() {
+ return new DefaultLockRegistry();
+ }
+
+ @Bean
+ public ConcurrentMetadataStore checkpointStore() {
+ return new SimpleMetadataStore();
+ }
+
+ @Bean
+ public AtomicReference> messageHolder() {
+ return new AtomicReference<>();
+ }
+
+ @Bean
+ public CountDownLatch messageBarrier() {
+ return new CountDownLatch(1);
+ }
+
+ @Bean
+ public Supplier>> kinesisSupplier() {
+ return () -> Flux.just("test data").map(GenericMessage::new);
+ }
+
+ @Bean
+ QueueChannel producerResultsChannel() {
+ return new QueueChannel();
+ }
+
+ @Bean
+ @GlobalChannelInterceptor(patterns = "kinesisSupplier-out-0")
+ ChannelInterceptor loggingChannelInterceptor() {
+ return new ChannelInterceptor() {
+
+ @Override
+ public Message> preSend(Message> message, MessageChannel channel) {
+ LOGGER.debug("Send message: " + message);
+ return message;
+ }
+
+ };
+ }
+
+ @Bean
+ public Consumer> kinesisConsumer(AtomicReference> messageHolder,
+ CountDownLatch messageBarrier) {
+
+ return message -> {
+ LOGGER.debug("Received message: " + message);
+ messageHolder.set(message);
+ messageBarrier.countDown();
+ };
+ }
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisStreamProvisionerTests.java b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisStreamProvisionerTests.java
new file mode 100644
index 000000000..55fbe2d3c
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/java/io/awspring/cloud/kinesis/stream/binder/provisioning/KinesisStreamProvisionerTests.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.stream.binder.provisioning;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+
+import io.awspring.cloud.kinesis.stream.binder.LocalstackContainerTest;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisBinderConfigurationProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisConsumerProperties;
+import io.awspring.cloud.kinesis.stream.binder.properties.KinesisProducerProperties;
+import java.time.Duration;
+import org.assertj.core.api.InstanceOfAssertFactories;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.springframework.cloud.stream.binder.ExtendedConsumerProperties;
+import org.springframework.cloud.stream.binder.ExtendedProducerProperties;
+import org.springframework.cloud.stream.binder.HeaderMode;
+import org.springframework.cloud.stream.provisioning.ConsumerDestination;
+import org.springframework.cloud.stream.provisioning.ProducerDestination;
+import org.springframework.cloud.stream.provisioning.ProvisioningException;
+import software.amazon.awssdk.retries.api.BackoffStrategy;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+
+/**
+ * Tests for the {@link KinesisStreamProvisioner}.
+ *
+ * @author Jacob Severson
+ * @author Artem Bilan
+ * @author Sergiu Pantiru
+ *
+ * @since 4.0
+ */
+class KinesisStreamProvisionerTests implements LocalstackContainerTest {
+
+ private static KinesisAsyncClient AMAZON_KINESIS;
+
+ @BeforeAll
+ static void setup() {
+ AMAZON_KINESIS = LocalstackContainerTest.kinesisClient();
+ }
+
+ private void createStream(String streamName) {
+ AMAZON_KINESIS.createStream(request -> request.streamName(streamName).shardCount(1))
+ .thenCompose(reply -> AMAZON_KINESIS.waiter()
+ .waitUntilStreamExists(request -> request.streamName(streamName), waiter -> waiter
+ .maxAttempts(60).backoffStrategyV2(BackoffStrategy.fixedDelay(Duration.ofSeconds(1)))))
+ .join();
+ }
+
+ @Test
+ void testProvisionProducerSuccessfulWithExistingStream() {
+ String streamName = "provisioning1";
+ createStream(streamName);
+
+ KinesisBinderConfigurationProperties binderProperties = new KinesisBinderConfigurationProperties();
+ KinesisStreamProvisioner provisioner = new KinesisStreamProvisioner(AMAZON_KINESIS, binderProperties);
+ ExtendedProducerProperties extendedProducerProperties = new ExtendedProducerProperties<>(
+ new KinesisProducerProperties());
+
+ ProducerDestination destination = provisioner.provisionProducerDestination(streamName,
+ extendedProducerProperties);
+
+ assertThat(destination.getName()).isEqualTo(streamName);
+ assertThat(destination).isInstanceOf(KinesisProducerDestination.class).extracting("shards")
+ .asInstanceOf(InstanceOfAssertFactories.LIST).hasSize(1);
+ }
+
+ @Test
+ void testProvisionConsumerSuccessfulWithExistingStream() {
+ String streamName = "provisioning2";
+ createStream(streamName);
+
+ KinesisBinderConfigurationProperties binderProperties = new KinesisBinderConfigurationProperties();
+ KinesisStreamProvisioner provisioner = new KinesisStreamProvisioner(AMAZON_KINESIS, binderProperties);
+
+ ExtendedConsumerProperties extendedConsumerProperties = new ExtendedConsumerProperties<>(
+ new KinesisConsumerProperties());
+ extendedConsumerProperties.setHeaderMode(HeaderMode.embeddedHeaders);
+
+ String group = "test-group";
+ ConsumerDestination destination = provisioner.provisionConsumerDestination(streamName, group,
+ extendedConsumerProperties);
+
+ assertThat(destination.getName()).isEqualTo(streamName);
+ assertThat(destination).isInstanceOf(KinesisConsumerDestination.class).extracting("shards")
+ .asInstanceOf(InstanceOfAssertFactories.LIST).hasSize(1);
+ assertThat(extendedConsumerProperties.getExtension().isEmbedHeaders()).isTrue();
+ assertThat(extendedConsumerProperties.getHeaderMode()).isEqualTo(HeaderMode.none);
+ }
+
+ @Test
+ void testProvisionConsumerExistingStreamUpdateShards() {
+ String streamName = "provisioning3";
+ createStream(streamName);
+
+ KinesisBinderConfigurationProperties binderProperties = new KinesisBinderConfigurationProperties();
+ int targetShardCount = 2;
+ binderProperties.setMinShardCount(targetShardCount);
+ binderProperties.setAutoAddShards(true);
+ KinesisStreamProvisioner provisioner = new KinesisStreamProvisioner(AMAZON_KINESIS, binderProperties);
+
+ ExtendedConsumerProperties extendedConsumerProperties = new ExtendedConsumerProperties<>(
+ new KinesisConsumerProperties());
+
+ String group = "test-group";
+ ConsumerDestination destination = provisioner.provisionConsumerDestination(streamName, group,
+ extendedConsumerProperties);
+
+ assertThat(destination.getName()).isEqualTo(streamName);
+ assertThat(destination).isInstanceOf(KinesisConsumerDestination.class).extracting("shards")
+ .asInstanceOf(InstanceOfAssertFactories.LIST).hasSizeGreaterThanOrEqualTo(2);
+ }
+
+ @Test
+ void testProvisionProducerSuccessfulWithNewStream() {
+ String streamName = "provisioning4";
+ KinesisBinderConfigurationProperties binderProperties = new KinesisBinderConfigurationProperties();
+ KinesisStreamProvisioner provisioner = new KinesisStreamProvisioner(AMAZON_KINESIS, binderProperties);
+ ExtendedProducerProperties extendedProducerProperties = new ExtendedProducerProperties<>(
+ new KinesisProducerProperties());
+
+ ProducerDestination destination = provisioner.provisionProducerDestination(streamName,
+ extendedProducerProperties);
+
+ assertThat(destination.getName()).isEqualTo(streamName);
+ assertThat(destination).isInstanceOf(KinesisProducerDestination.class).extracting("shards")
+ .asInstanceOf(InstanceOfAssertFactories.LIST).hasSize(1);
+ }
+
+ @Test
+ void testProvisionConsumerResourceNotFoundException() {
+ KinesisBinderConfigurationProperties binderProperties = new KinesisBinderConfigurationProperties();
+ binderProperties.setAutoCreateStream(false);
+ KinesisStreamProvisioner provisioner = new KinesisStreamProvisioner(AMAZON_KINESIS, binderProperties);
+ int instanceCount = 1;
+ int concurrency = 1;
+
+ ExtendedConsumerProperties extendedConsumerProperties = new ExtendedConsumerProperties<>(
+ new KinesisConsumerProperties());
+ extendedConsumerProperties.setInstanceCount(instanceCount);
+ extendedConsumerProperties.setConcurrency(concurrency);
+
+ String name = "provisioning5";
+ String group = "test-group";
+
+ assertThatExceptionOfType(ProvisioningException.class)
+ .isThrownBy(() -> provisioner.provisionConsumerDestination(name, group, extendedConsumerProperties))
+ .withMessageContaining("The stream [provisioning5] was not found and auto creation is disabled.")
+ .withCauseInstanceOf(ResourceNotFoundException.class);
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis-stream-binder/src/test/resources/logback.xml b/spring-cloud-aws-kinesis-stream-binder/src/test/resources/logback.xml
new file mode 100644
index 000000000..5566e44d3
--- /dev/null
+++ b/spring-cloud-aws-kinesis-stream-binder/src/test/resources/logback.xml
@@ -0,0 +1,16 @@
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger - %msg%n
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/spring-cloud-aws-kinesis/pom.xml b/spring-cloud-aws-kinesis/pom.xml
index 4eca3c1c1..3a648385c 100644
--- a/spring-cloud-aws-kinesis/pom.xml
+++ b/spring-cloud-aws-kinesis/pom.xml
@@ -28,19 +28,24 @@
amazon-kinesis-client
true
+
+ com.amazonaws
+ dynamodb-streams-kinesis-adapter
+ true
+
org.springframework.integration
spring-integration-core
true
- org.testcontainers
- testcontainers-localstack
+ org.testcontainers
+ testcontainers-localstack
test
-org.testcontainers
- testcontainers-junit-jupiter
+ org.testcontainers
+ testcontainers-junit-jupiter
test
@@ -51,5 +56,4 @@
-
diff --git a/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapter.java b/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapter.java
index b6914256c..34f424dfc 100644
--- a/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapter.java
+++ b/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapter.java
@@ -15,6 +15,11 @@
*/
package io.awspring.cloud.kinesis.integration;
+import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
+import com.amazonaws.services.dynamodbv2.streamsadapter.StreamsSchedulerFactory;
+import com.amazonaws.services.dynamodbv2.streamsadapter.model.DynamoDBStreamsProcessRecordsInput;
+import com.amazonaws.services.dynamodbv2.streamsadapter.polling.DynamoDBStreamsPollingConfig;
+import com.amazonaws.services.dynamodbv2.streamsadapter.processor.DynamoDBStreamsShardRecordProcessor;
import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryDeserializer;
import java.time.Duration;
import java.util.ArrayList;
@@ -50,6 +55,7 @@
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse;
import software.amazon.awssdk.utils.BinaryUtils;
@@ -114,6 +120,8 @@ public class KclMessageDrivenChannelAdapter extends MessageProducerSupport imple
private final DynamoDbAsyncClient dynamoDBClient;
+ private DynamoDbStreamsClient dynamoDBStreams;
+
private TaskExecutor executor = new SimpleAsyncTaskExecutor();
private String consumerGroup = "SpringIntegration";
@@ -385,6 +393,11 @@ public void setGracefulShutdownTimeout(long gracefulShutdownTimeout) {
this.gracefulShutdownTimeout = gracefulShutdownTimeout;
}
+ public void setDynamoDBStreams(DynamoDbStreamsClient dynamoDBStreams) {
+ this.dynamoDBStreams = dynamoDBStreams;
+ this.fanOut = false;
+ }
+
@Override
protected void onInit() {
super.onInit();
@@ -402,11 +415,22 @@ protected void onInit() {
private StreamTracker buildStreamTracker() {
if (this.streams.length == 1) {
- return new SingleStreamTracker(StreamIdentifier.singleStreamInstance(this.streams[0]),
+ String singleStream = this.streams[0];
+ if (this.dynamoDBStreams != null) {
+ return StreamsSchedulerFactory.createSingleStreamTracker(singleStream, this.streamInitialSequence);
+ }
+
+ return new SingleStreamTracker(StreamIdentifier.singleStreamInstance(singleStream),
this.streamInitialSequence);
}
else {
- return new StreamsTracker();
+ StreamsTracker streamsTracker = new StreamsTracker();
+ if (this.dynamoDBStreams != null) {
+ return StreamsSchedulerFactory.createMultiStreamTracker(Arrays.asList(this.streams),
+ this.streamInitialSequence, streamsTracker.formerStreamsLeasesDeletionStrategy);
+ }
+
+ return streamsTracker;
}
}
@@ -427,12 +451,16 @@ protected void doStart() {
RetrievalSpecificConfig retrievalSpecificConfig;
String singleStreamName = this.streams.length == 1 ? this.streams[0] : null;
if (this.fanOut) {
+ Assert.state(this.dynamoDBStreams == null, "'DynamoDBStreams' consumption does not support 'fan-out'.");
retrievalSpecificConfig = new FanOutConfig(this.kinesisClient).applicationName(this.consumerGroup)
.streamName(singleStreamName);
}
else {
- retrievalSpecificConfig = new PollingConfig(this.kinesisClient).streamName(singleStreamName)
- .maxRecords(this.pollingMaxRecords).idleTimeBetweenReadsInMillis(this.pollingIdleTime);
+ PollingConfig pollingConfig = this.dynamoDBStreams == null ? new PollingConfig(this.kinesisClient)
+ : new DynamoDBStreamsPollingConfig(this.kinesisClient);
+
+ retrievalSpecificConfig = pollingConfig.streamName(singleStreamName).maxRecords(this.pollingMaxRecords)
+ .idleTimeBetweenReadsInMillis(this.pollingIdleTime);
}
RetrievalConfig retrievalConfig = this.config.retrievalConfig()
@@ -455,8 +483,17 @@ protected void doStart() {
ProcessorConfig processorConfig = this.config.processorConfig()
.callProcessRecordsEvenForEmptyRecordList(this.emptyRecordList);
- this.scheduler = new Scheduler(this.config.checkpointConfig(), coordinatorConfig, leaseManagementConfig,
- lifecycleConfig, metricsConfig, processorConfig, retrievalConfig);
+ if (this.dynamoDBStreams != null) {
+ this.scheduler = StreamsSchedulerFactory.createScheduler(this.config.checkpointConfig(), coordinatorConfig,
+ leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig,
+ new AmazonDynamoDBStreamsAdapterClient(this.dynamoDBStreams) {
+
+ });
+ }
+ else {
+ this.scheduler = new Scheduler(this.config.checkpointConfig(), coordinatorConfig, leaseManagementConfig,
+ lifecycleConfig, metricsConfig, processorConfig, retrievalConfig);
+ }
this.executor.execute(this.scheduler);
}
@@ -513,7 +550,7 @@ private final class RecordProcessorFactory implements ShardRecordProcessorFactor
@Override
public ShardRecordProcessor shardRecordProcessor() {
- throw new UnsupportedOperationException();
+ return new RecordProcessor(null);
}
@Override
@@ -562,7 +599,7 @@ public FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy()
/**
* Processes records and checkpoints progress.
*/
- private final class RecordProcessor implements ShardRecordProcessor {
+ private final class RecordProcessor implements DynamoDBStreamsShardRecordProcessor {
private final String stream;
@@ -612,6 +649,22 @@ public void shutdownRequested(ShutdownRequestedInput shutdownRequestedInput) {
}
}
+ @Override
+ public void processRecords(DynamoDBStreamsProcessRecordsInput dynamoDBStreamsProcessRecordsInput) {
+ ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
+ .cacheEntryTime(dynamoDBStreamsProcessRecordsInput.cacheEntryTime())
+ .checkpointer(dynamoDBStreamsProcessRecordsInput.checkpointer())
+ .cacheExitTime(dynamoDBStreamsProcessRecordsInput.cacheExitTime())
+ .isAtShardEnd(dynamoDBStreamsProcessRecordsInput.isAtShardEnd())
+ .millisBehindLatest(dynamoDBStreamsProcessRecordsInput.millisBehindLatest())
+ .childShards(dynamoDBStreamsProcessRecordsInput.childShards())
+ .records(dynamoDBStreamsProcessRecordsInput.records().stream().map(KinesisClientRecord.class::cast)
+ .toList())
+ .build();
+
+ processRecords(processRecordsInput);
+ }
+
@Override
public void processRecords(ProcessRecordsInput processRecordsInput) {
List records = processRecordsInput.records();
diff --git a/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/SpringDynamoDBStreamsAdapterClient.java b/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/SpringDynamoDBStreamsAdapterClient.java
new file mode 100644
index 000000000..e1d78c3b8
--- /dev/null
+++ b/spring-cloud-aws-kinesis/src/main/java/io/awspring/cloud/kinesis/integration/SpringDynamoDBStreamsAdapterClient.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.integration;
+
+import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
+import com.amazonaws.services.dynamodbv2.streamsadapter.adapter.DynamoDBStreamsGetRecordsResponseAdapter;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.services.dynamodb.model.GetRecordsRequest;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
+import software.amazon.awssdk.services.kinesis.model.InvalidArgumentException;
+import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.StreamDescription;
+
+/**
+ * This is Spring Cloud DynamoDB Adapter to be able to support {@code ListShards} operations. Also, adapts
+ * {@link #getDynamoDBStreamsRecords(software.amazon.awssdk.services.dynamodb.model.GetRecordsRequest)} to
+ * {@link #getRecords(software.amazon.awssdk.services.kinesis.model.GetRecordsRequest)} for regular
+ * {@link KinesisMessageDrivenChannelAdapter}. The KCL bindings uses {@link KclMessageDrivenChannelAdapter}'s native
+ * support for {@link AmazonDynamoDBStreamsAdapterClient} API.
+ *
+ * @author Asiel Caballero
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ *
+ * @see ListShards
+ * @see ListShards GH issue
+ */
+public class SpringDynamoDBStreamsAdapterClient extends AmazonDynamoDBStreamsAdapterClient {
+
+ private static final String SEPARATOR = "!!##%%";
+
+ public SpringDynamoDBStreamsAdapterClient(DynamoDbStreamsClient dynamoDbStreamsClient) {
+ super(dynamoDbStreamsClient);
+ }
+
+ /**
+ * List shards for a DynamoDB Stream using its {@code DescribeStream} API, as they don't support {@code ListShards}
+ * operations. Returns the result adapted to use the AmazonKinesis model.
+ * @param request Container for the necessary parameters to execute the ListShards service method
+ * @return The response from the DescribeStream service method, adapted for use with the AmazonKinesis model
+ */
+ @Override
+ public CompletableFuture listShards(ListShardsRequest request) {
+ String nextToken = request.nextToken();
+ if (nextToken != null && request.streamName() != null) {
+ return CompletableFuture.failedFuture(InvalidArgumentException.builder()
+ .message("NextToken and StreamName cannot be provided together.").build());
+ }
+
+ String streamName = request.streamName();
+ String exclusiveStartShardId = request.exclusiveStartShardId();
+
+ if (nextToken != null) {
+ String[] split = nextToken.split(SEPARATOR);
+
+ if (split.length != 2) {
+ return CompletableFuture
+ .failedFuture(InvalidArgumentException.builder().message("Invalid ShardIterator").build());
+ }
+
+ streamName = split[0];
+ exclusiveStartShardId = split[1];
+ }
+
+ DescribeStreamRequest dsr = DescribeStreamRequest.builder().streamName(streamName)
+ .exclusiveStartShardId(exclusiveStartShardId).limit(request.maxResults()).build();
+
+ StreamDescription streamDescription;
+ try {
+ streamDescription = describeStream(dsr).join().streamDescription();
+ }
+ catch (CompletionException ex) {
+ ResourceNotFoundException resourceEx = ResourceNotFoundException.builder().message(ex.getMessage()).build();
+ resourceEx.setStackTrace(ex.getStackTrace());
+ return CompletableFuture.failedFuture(resourceEx);
+ }
+
+ List shards = streamDescription.shards();
+ ListShardsResponse.Builder result = ListShardsResponse.builder().shards(shards);
+
+ if (streamDescription.hasMoreShards()) {
+ result.nextToken(buildFakeNextToken(streamName, shards.get(shards.size() - 1).shardId()));
+ }
+
+ return CompletableFuture.completedFuture(result.build());
+ }
+
+ @Override
+ public CompletableFuture getRecords(
+ software.amazon.awssdk.services.kinesis.model.GetRecordsRequest getRecordsRequest)
+ throws AwsServiceException, SdkClientException {
+
+ GetRecordsRequest dynamoDbStreamsRecordsRequest = GetRecordsRequest.builder().limit(getRecordsRequest.limit())
+ .shardIterator(getRecordsRequest.shardIterator()).build();
+
+ return getDynamoDBStreamsRecords(dynamoDbStreamsRecordsRequest).thenApply(this::toGetRecordsResponse);
+ }
+
+ private GetRecordsResponse toGetRecordsResponse(DynamoDBStreamsGetRecordsResponseAdapter recordsResponseAdapter) {
+ return GetRecordsResponse.builder().childShards(recordsResponseAdapter.childShards())
+ .millisBehindLatest(recordsResponseAdapter.millisBehindLatest())
+ .nextShardIterator(recordsResponseAdapter.nextShardIterator())
+ .records(recordsResponseAdapter.records().stream()
+ .map((kinesisClientRecord) -> Record.builder()
+ .approximateArrivalTimestamp(kinesisClientRecord.approximateArrivalTimestamp())
+ .encryptionType(kinesisClientRecord.encryptionType())
+ .partitionKey(kinesisClientRecord.partitionKey())
+ .sequenceNumber(kinesisClientRecord.sequenceNumber())
+ .data(SdkBytes.fromByteBuffer(kinesisClientRecord.data())).build())
+ .toList())
+ .build();
+ }
+
+ private static String buildFakeNextToken(String streamName, String lastShard) {
+ return lastShard != null ? streamName + SEPARATOR + lastShard : null;
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/LocalstackContainerTest.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/LocalstackContainerTest.java
index e61c21f85..ca0127628 100644
--- a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/LocalstackContainerTest.java
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/LocalstackContainerTest.java
@@ -27,6 +27,7 @@
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
/**
@@ -58,6 +59,10 @@ static DynamoDbAsyncClient dynamoDbClient() {
return applyAwsClientOptions(DynamoDbAsyncClient.builder());
}
+ static DynamoDbStreamsClient dynamoDbStreamsClient() {
+ return applyAwsClientOptions(DynamoDbStreamsClient.builder());
+ }
+
static CloudWatchAsyncClient cloudWatchClient() {
return applyAwsClientOptions(CloudWatchAsyncClient.builder());
}
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/DynamoDbStreamToKclIntegrationTests.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/DynamoDbStreamToKclIntegrationTests.java
new file mode 100644
index 000000000..199674130
--- /dev/null
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/DynamoDbStreamToKclIntegrationTests.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.integration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
+import io.awspring.cloud.kinesis.LocalstackContainerTest;
+import java.time.Duration;
+import java.util.Map;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.integration.channel.QueueChannel;
+import org.springframework.integration.config.EnableIntegration;
+import org.springframework.messaging.Message;
+import org.springframework.messaging.PollableChannel;
+import org.springframework.test.annotation.DirtiesContext;
+import org.springframework.test.context.junit.jupiter.SpringJUnitConfig;
+import software.amazon.awssdk.retries.api.BackoffStrategy;
+import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeAction;
+import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
+import software.amazon.awssdk.services.dynamodb.model.StreamViewType;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.kinesis.common.InitialPositionInStream;
+import software.amazon.kinesis.common.InitialPositionInStreamExtended;
+import software.amazon.kinesis.metrics.MetricsLevel;
+
+/**
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@SpringJUnitConfig
+@DirtiesContext
+class DynamoDbStreamToKclIntegrationTests implements LocalstackContainerTest {
+
+ static final String TEST_TABLE = "StreamsKclTable";
+
+ static final String LEASE_TABLE_NAME = "lease_table";
+
+ static final String TABLE_KEY = "id";
+
+ static KinesisAsyncClient AMAZON_KINESIS;
+
+ static CloudWatchAsyncClient CLOUD_WATCH;
+
+ static DynamoDbAsyncClient DYNAMODB;
+
+ static DynamoDbStreamsClient DYNAMODB_STREAMS;
+
+ static String DYNAMODB_STREAM_ARN;
+
+ @Autowired
+ PollableChannel kinesisReceiveChannel;
+
+ @BeforeAll
+ static void setup() {
+ AMAZON_KINESIS = LocalstackContainerTest.kinesisClient();
+ CLOUD_WATCH = LocalstackContainerTest.cloudWatchClient();
+ DYNAMODB = LocalstackContainerTest.dynamoDbClient();
+ DYNAMODB_STREAMS = LocalstackContainerTest.dynamoDbStreamsClient();
+ DYNAMODB_STREAM_ARN = createDemoTable();
+
+ AmazonDynamoDBStreamsAdapterClient streamsAdapterClient = new AmazonDynamoDBStreamsAdapterClient(
+ DYNAMODB_STREAMS, null);
+
+ await().atMost(Duration.ofMinutes(2))
+ .untilAsserted(() -> assertThat(
+ streamsAdapterClient.describeStream(builder -> builder.streamName(DYNAMODB_STREAM_ARN)))
+ .succeedsWithin(Duration.ofSeconds(60))
+ .extracting(describeStreamResponse -> describeStreamResponse.streamDescription()
+ .streamStatusAsString())
+ .isEqualTo("ENABLED"));
+ }
+
+ private static String createDemoTable() {
+ CreateTableRequest createTableRequest = CreateTableRequest.builder()
+ .attributeDefinitions(AttributeDefinition.builder().attributeName(TABLE_KEY)
+ .attributeType(ScalarAttributeType.S).build())
+ .keySchema(KeySchemaElement.builder().attributeName(TABLE_KEY).keyType(KeyType.HASH).build())
+ .billingMode(BillingMode.PAY_PER_REQUEST).tableName(TEST_TABLE)
+ .streamSpecification(
+ builder -> builder.streamEnabled(true).streamViewType(StreamViewType.NEW_AND_OLD_IMAGES))
+ .build();
+
+ return DYNAMODB.createTable(createTableRequest)
+ .thenCompose(result -> DYNAMODB.waiter().waitUntilTableExists(request -> request.tableName(TEST_TABLE),
+ waiter -> waiter.maxAttempts(10)
+ .backoffStrategyV2(BackoffStrategy.fixedDelay(Duration.ofSeconds(1)))))
+ .thenApply(waiterResponse -> waiterResponse.matched().response()
+ .map(describeTableResponse -> describeTableResponse.table().latestStreamArn()))
+ .join().get();
+ }
+
+ @Test
+ void fromDynamoDbStreamToKcl() {
+ DYNAMODB.putItem(builder -> builder.tableName(TEST_TABLE).item(
+ Map.of(TABLE_KEY, AttributeValue.fromS("some_id"), "Message", AttributeValue.fromS("some value"))))
+ .join();
+
+ Message> receive = this.kinesisReceiveChannel.receive(120_000);
+ assertThat(receive).extracting(Message::getPayload).asString().contains("some_id", "some value",
+ "\"eventName\":\"INSERT\"", "\"eventSource\":\"aws:dynamodb\"");
+
+ DYNAMODB.updateItem(
+ builder -> builder.tableName(TEST_TABLE).key(Map.of(TABLE_KEY, AttributeValue.fromS("some_id")))
+ .attributeUpdates(Map.of("Message", AttributeValueUpdate.builder().action(AttributeAction.PUT)
+ .value(AttributeValue.fromS("updated value")).build())))
+ .join();
+
+ receive = this.kinesisReceiveChannel.receive(30_000);
+ assertThat(receive).extracting(Message::getPayload).asString().contains("some_id", "some value",
+ "updated value", "\"eventName\":\"MODIFY\"");
+
+ DYNAMODB.deleteItem(
+ builder -> builder.tableName(TEST_TABLE).key(Map.of(TABLE_KEY, AttributeValue.fromS("some_id"))))
+ .join();
+
+ receive = this.kinesisReceiveChannel.receive(30_000);
+ assertThat(receive).extracting(Message::getPayload).asString().contains("some_id", "\"eventName\":\"REMOVE\"");
+ }
+
+ @Configuration(proxyBeanMethods = false)
+ @EnableIntegration
+ static class TestConfiguration {
+
+ @Bean
+ PollableChannel kinesisReceiveChannel() {
+ return new QueueChannel();
+ }
+
+ @Bean
+ KclMessageDrivenChannelAdapter kclMessageDrivenChannelAdapter(PollableChannel kinesisReceiveChannel) {
+ KclMessageDrivenChannelAdapter adapter = new KclMessageDrivenChannelAdapter(AMAZON_KINESIS, CLOUD_WATCH,
+ DYNAMODB, DYNAMODB_STREAM_ARN);
+ adapter.setOutputChannel(kinesisReceiveChannel);
+ adapter.setStreamInitialSequence(
+ InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON));
+ adapter.setLeaseTableName(LEASE_TABLE_NAME);
+ adapter.setConverter(String::new);
+ adapter.setDynamoDBStreams(DYNAMODB_STREAMS);
+ adapter.setMetricsLevel(MetricsLevel.NONE);
+ adapter.setLeaseManagementConfigCustomizer(
+ leaseManagementConfig -> leaseManagementConfig.maxLeasesForWorker(10).shardSyncIntervalMillis(0)
+ .workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
+ adapter.setCoordinatorConfigCustomizer(
+ coordinatorConfig -> coordinatorConfig.shardConsumerDispatchPollIntervalMillis(500L));
+ adapter.setPollingMaxRecords(3);
+ adapter.setGracefulShutdownTimeout(100);
+ return adapter;
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/DynamoDbStreamToKinesisIntegrationTests.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/DynamoDbStreamToKinesisIntegrationTests.java
new file mode 100644
index 000000000..1548b4dd2
--- /dev/null
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/DynamoDbStreamToKinesisIntegrationTests.java
@@ -0,0 +1,192 @@
+/*
+ * Copyright 2013-2025 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.integration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
+import io.awspring.cloud.kinesis.LocalstackContainerTest;
+import java.time.Duration;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.springframework.beans.DirectFieldAccessor;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.integration.channel.QueueChannel;
+import org.springframework.integration.config.EnableIntegration;
+import org.springframework.integration.test.util.TestUtils;
+import org.springframework.messaging.Message;
+import org.springframework.messaging.PollableChannel;
+import org.springframework.test.annotation.DirtiesContext;
+import org.springframework.test.context.junit.jupiter.SpringJUnitConfig;
+import software.amazon.awssdk.retries.api.BackoffStrategy;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeAction;
+import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate;
+import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
+import software.amazon.awssdk.services.dynamodb.model.StreamViewType;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+
+/**
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+@SpringJUnitConfig
+@DirtiesContext
+class DynamoDbStreamToKinesisIntegrationTests implements LocalstackContainerTest {
+
+ static final Log LOGGER = LogFactory.getLog(DynamoDbStreamToKinesisIntegrationTests.class);
+
+ static final String TEST_TABLE = "StreamsDemoTable";
+
+ static final String TABLE_KEY = "id";
+
+ static DynamoDbAsyncClient DYNAMODB;
+
+ static DynamoDbStreamsClient DYNAMODB_STREAMS;
+
+ static String DYNAMODB_STREAM_ARN;
+
+ @Autowired
+ KinesisMessageDrivenChannelAdapter kinesisMessageDrivenChannelAdapter;
+
+ @Autowired
+ PollableChannel kinesisReceiveChannel;
+
+ @BeforeAll
+ static void setup() {
+ DYNAMODB = LocalstackContainerTest.dynamoDbClient();
+ DYNAMODB_STREAMS = LocalstackContainerTest.dynamoDbStreamsClient();
+ DYNAMODB_STREAM_ARN = createDemoTable();
+
+ AmazonDynamoDBStreamsAdapterClient streamsAdapterClient = new AmazonDynamoDBStreamsAdapterClient(
+ DYNAMODB_STREAMS, null);
+
+ await().atMost(Duration.ofMinutes(2))
+ .untilAsserted(() -> assertThat(
+ streamsAdapterClient.describeStream(builder -> builder.streamName(DYNAMODB_STREAM_ARN)))
+ .succeedsWithin(Duration.ofSeconds(60))
+ .extracting(describeStreamResponse -> describeStreamResponse.streamDescription()
+ .streamStatusAsString())
+ .isEqualTo("ENABLED"));
+ }
+
+ private static String createDemoTable() {
+ CreateTableRequest createTableRequest = CreateTableRequest.builder()
+ .attributeDefinitions(AttributeDefinition.builder().attributeName(TABLE_KEY)
+ .attributeType(ScalarAttributeType.S).build())
+ .keySchema(KeySchemaElement.builder().attributeName(TABLE_KEY).keyType(KeyType.HASH).build())
+ .billingMode(BillingMode.PAY_PER_REQUEST).tableName(TEST_TABLE)
+ .streamSpecification(
+ builder -> builder.streamEnabled(true).streamViewType(StreamViewType.NEW_AND_OLD_IMAGES))
+ .build();
+
+ return DYNAMODB.createTable(createTableRequest)
+ .thenCompose(result -> DYNAMODB.waiter().waitUntilTableExists(request -> request.tableName(TEST_TABLE),
+ waiter -> waiter.maxAttempts(10)
+ .backoffStrategyV2(BackoffStrategy.fixedDelay(Duration.ofSeconds(1)))))
+ .thenApply(waiterResponse -> waiterResponse.matched().response()
+ .map(describeTableResponse -> describeTableResponse.table().latestStreamArn()))
+ .join().get();
+ }
+
+ @Test
+ void fromDynamoDbStreamToKinesis() {
+ Map, ?> shardConsumers = TestUtils.getPropertyValue(this.kinesisMessageDrivenChannelAdapter, "shardConsumers",
+ Map.class);
+
+ await().untilAsserted(() -> assertThat(shardConsumers).hasSize(1));
+
+ Object shardConsumer = shardConsumers.values().iterator().next();
+
+ await().untilAsserted(
+ () -> assertThat(TestUtils.getPropertyValue(shardConsumer, "state").toString()).isEqualTo("CONSUME"));
+
+ DYNAMODB.putItem(builder -> builder.tableName(TEST_TABLE).item(
+ Map.of(TABLE_KEY, AttributeValue.fromS("some_id"), "Message", AttributeValue.fromS("some value"))))
+ .join();
+
+ Message> receive = this.kinesisReceiveChannel.receive(120_000);
+ assertThat(receive)
+ .satisfies(LOGGER::warn)
+ .extracting(Message::getPayload)
+ .asString()
+ .contains("some_id", "some value", "\"eventName\":\"INSERT\"", "\"eventSource\":\"aws:dynamodb\"");
+
+ DYNAMODB.updateItem(
+ builder -> builder.tableName(TEST_TABLE).key(Map.of(TABLE_KEY, AttributeValue.fromS("some_id")))
+ .attributeUpdates(Map.of("Message", AttributeValueUpdate.builder().action(AttributeAction.PUT)
+ .value(AttributeValue.fromS("updated value")).build())))
+ .join();
+
+ receive = this.kinesisReceiveChannel.receive(30_000);
+ assertThat(receive)
+ .satisfies(LOGGER::warn)
+ .extracting(Message::getPayload)
+ .asString()
+ .contains("some_id", "some value", "updated value", "\"eventName\":\"MODIFY\"");
+
+ DYNAMODB.deleteItem(
+ builder -> builder.tableName(TEST_TABLE).key(Map.of(TABLE_KEY, AttributeValue.fromS("some_id"))))
+ .join();
+
+ receive = this.kinesisReceiveChannel.receive(30_000);
+ assertThat(receive)
+ .satisfies(LOGGER::warn)
+ .extracting(Message::getPayload)
+ .asString()
+ .contains("some_id", "\"eventName\":\"REMOVE\"");
+ }
+
+ @Configuration(proxyBeanMethods = false)
+ @EnableIntegration
+ static class TestConfiguration {
+
+ @Bean
+ PollableChannel kinesisReceiveChannel() {
+ return new QueueChannel();
+ }
+
+ @Bean
+ KinesisMessageDrivenChannelAdapter kinesisMessageDrivenChannelAdapter(PollableChannel kinesisReceiveChannel) {
+ KinesisMessageDrivenChannelAdapter adapter = new KinesisMessageDrivenChannelAdapter(
+ new SpringDynamoDBStreamsAdapterClient(DYNAMODB_STREAMS), DYNAMODB_STREAM_ARN);
+ adapter.setStreamInitialSequence(KinesisShardOffset.trimHorizon());
+ adapter.setOutputChannel(kinesisReceiveChannel);
+ adapter.setConverter(String::new);
+ DirectFieldAccessor dfa = new DirectFieldAccessor(adapter);
+ dfa.setPropertyValue("describeStreamBackoff", 10);
+ dfa.setPropertyValue("consumerBackoff", 10);
+ dfa.setPropertyValue("idleBetweenPolls", 1);
+ return adapter;
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterMultiStreamTests.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterMultiStreamTests.java
index 81aa4d1f3..ac6761abe 100644
--- a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterMultiStreamTests.java
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterMultiStreamTests.java
@@ -130,8 +130,9 @@ public KclMessageDrivenChannelAdapter kclMessageDrivenChannelAdapter(PollableCha
adapter.setConverter(String::new);
adapter.setConsumerGroup("multi_stream_group");
adapter.setMetricsLevel(MetricsLevel.NONE);
- adapter.setLeaseManagementConfigCustomizer(leaseManagementConfig -> leaseManagementConfig
- .workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
+ adapter.setLeaseManagementConfigCustomizer(
+ leaseManagementConfig -> leaseManagementConfig.maxLeasesForWorker(10).shardSyncIntervalMillis(0)
+ .workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
return adapter;
}
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterTests.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterTests.java
index 4161a6072..1ebf13cd2 100644
--- a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterTests.java
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KclMessageDrivenChannelAdapterTests.java
@@ -182,11 +182,12 @@ public KclMessageDrivenChannelAdapter kclMessageDrivenChannelAdapter(PollableCha
adapter.setOutputChannel(kinesisReceiveChannel);
adapter.setConverter(String::new);
adapter.setConsumerGroup("single_stream_group");
- adapter.setLeaseTableName("test_table");
+ adapter.setLeaseTableName(LEASE_TABLE_NAME);
adapter.setFanOut(false);
adapter.setMetricsLevel(MetricsLevel.NONE);
- adapter.setLeaseManagementConfigCustomizer(leaseManagementConfig -> leaseManagementConfig
- .maxLeasesForWorker(10).workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
+ adapter.setLeaseManagementConfigCustomizer(
+ leaseManagementConfig -> leaseManagementConfig.maxLeasesForWorker(10).shardSyncIntervalMillis(0)
+ .workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
adapter.setCoordinatorConfigCustomizer(
coordinatorConfig -> coordinatorConfig.shardConsumerDispatchPollIntervalMillis(500L));
adapter.setBindSourceRecord(true);
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KplKclIntegrationTests.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KplKclIntegrationTests.java
index 3a186fcd7..9c64cc6b8 100644
--- a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KplKclIntegrationTests.java
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/KplKclIntegrationTests.java
@@ -20,6 +20,7 @@
import io.awspring.cloud.kinesis.LocalstackContainerTest;
import java.net.URI;
import java.util.Date;
+import java.util.Map;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
@@ -41,6 +42,14 @@
import org.springframework.test.context.junit.jupiter.SpringJUnitConfig;
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughput;
+import software.amazon.awssdk.services.dynamodb.model.PutItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
import software.amazon.kinesis.common.InitialPositionInStream;
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
@@ -60,6 +69,8 @@ class KplKclIntegrationTests implements LocalstackContainerTest {
private static final String TEST_STREAM = "TestStreamKplKcl";
+ static final String LEASE_TABLE_NAME = "kcl_lease_table";
+
private static KinesisAsyncClient AMAZON_KINESIS;
private static DynamoDbAsyncClient DYNAMO_DB;
@@ -84,6 +95,32 @@ static void setup() {
AMAZON_KINESIS.createStream(request -> request.streamName(TEST_STREAM).shardCount(1)).thenCompose(
result -> AMAZON_KINESIS.waiter().waitUntilStreamExists(request -> request.streamName(TEST_STREAM)))
.join();
+
+ initializeLeaseTable();
+ }
+
+ /**
+ * Initialize the lease table to improve KCL initialisation time
+ */
+ private static void initializeLeaseTable() {
+ DYNAMO_DB
+ .createTable(CreateTableRequest.builder().tableName(LEASE_TABLE_NAME)
+ .attributeDefinitions(AttributeDefinition.builder().attributeName("leaseKey")
+ .attributeType(ScalarAttributeType.S).build())
+ .keySchema(KeySchemaElement.builder().attributeName("leaseKey").keyType(KeyType.HASH).build())
+ .provisionedThroughput(
+ ProvisionedThroughput.builder().readCapacityUnits(1L).writeCapacityUnits(1L).build())
+ .build())
+ .thenCompose(result -> DYNAMO_DB.waiter()
+ .waitUntilTableExists(request -> request.tableName(LEASE_TABLE_NAME)))
+ .thenCompose(describeTableResponseWaiterResponse -> DYNAMO_DB.putItem(PutItemRequest.builder()
+ .tableName(LEASE_TABLE_NAME)
+ .item(Map.of("leaseKey", AttributeValue.fromS("shardId-000000000000"), "checkpoint",
+ AttributeValue.fromS("TRIM_HORIZON"), "leaseCounter", AttributeValue.fromN("1"),
+ "startingHashKey", AttributeValue.fromS("0"), "ownerSwitchesSinceCheckpoint",
+ AttributeValue.fromN("0"), "checkpointSubSequenceNumber", AttributeValue.fromN("0")))
+ .build()))
+ .join();
}
@Test
@@ -164,8 +201,9 @@ public KclMessageDrivenChannelAdapter kclMessageDrivenChannelAdapter(PollableCha
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON));
adapter.setBindSourceRecord(true);
adapter.setMetricsLevel(MetricsLevel.NONE);
- adapter.setLeaseManagementConfigCustomizer(leaseManagementConfig -> leaseManagementConfig
- .workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
+ adapter.setLeaseManagementConfigCustomizer(
+ leaseManagementConfig -> leaseManagementConfig.maxLeasesForWorker(10).shardSyncIntervalMillis(0)
+ .workerUtilizationAwareAssignmentConfig().disableWorkerMetrics(true));
return adapter;
}
diff --git a/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/SpringDynamoDBAdapterClientTests.java b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/SpringDynamoDBAdapterClientTests.java
new file mode 100644
index 000000000..b0bcd27ff
--- /dev/null
+++ b/spring-cloud-aws-kinesis/src/test/java/io/awspring/cloud/kinesis/integration/SpringDynamoDBAdapterClientTests.java
@@ -0,0 +1,199 @@
+/*
+ * Copyright 2013-2019 the original author or authors.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.awspring.cloud.kinesis.integration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.List;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.awscore.exception.AwsErrorDetails;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse;
+import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement;
+import software.amazon.awssdk.services.dynamodb.model.KeyType;
+import software.amazon.awssdk.services.dynamodb.model.Shard;
+import software.amazon.awssdk.services.dynamodb.model.StreamDescription;
+import software.amazon.awssdk.services.dynamodb.model.StreamStatus;
+import software.amazon.awssdk.services.dynamodb.model.StreamViewType;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import software.amazon.awssdk.services.kinesis.model.InvalidArgumentException;
+import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+
+/**
+ * @author Asiel Caballero
+ * @author Artem Bilan
+ *
+ * @since 4.0
+ */
+public class SpringDynamoDBAdapterClientTests {
+
+ private final DynamoDbStreamsClient amazonDynamoDBStreams = new InMemoryDynamoDbStreamsClient();
+
+ private final SpringDynamoDBStreamsAdapterClient springDynamoDBAdapterClient = new SpringDynamoDBStreamsAdapterClient(
+ amazonDynamoDBStreams);
+
+ @Test
+ public void listShardsForNotFoundStream() {
+ assertThat(springDynamoDBAdapterClient.listShards(builder -> builder.streamName("not-found")))
+ .completesExceptionallyWithin(Duration.ofSeconds(10)).withThrowableThat()
+ .withCauseInstanceOf(ResourceNotFoundException.class);
+ }
+
+ @Test
+ public void listShardsWithInvalidToken() {
+ assertThat(springDynamoDBAdapterClient.listShards(builder -> builder.nextToken("invalid-token")))
+ .completesExceptionallyWithin(Duration.ofSeconds(10)).withThrowableThat()
+ .withCauseInstanceOf(InvalidArgumentException.class);
+ }
+
+ @Test
+ public void listShardsWithTokenAndStreamName() {
+ assertThat(springDynamoDBAdapterClient.listShards(
+ builder -> builder.streamName(InMemoryDynamoDbStreamsClient.STREAM_ARN).nextToken("valid!!##%%token")))
+ .completesExceptionallyWithin(Duration.ofSeconds(10)).withThrowableThat()
+ .withCauseInstanceOf(InvalidArgumentException.class);
+ }
+
+ @Test
+ public void listShardsNoPagination() {
+ ListShardsResponse listShards = springDynamoDBAdapterClient
+ .listShards(builder -> builder.streamName(InMemoryDynamoDbStreamsClient.STREAM_ARN)).join();
+
+ assertThat(listShards.nextToken()).isNull();
+ assertThat(listShards.shards()).hasSize(InMemoryDynamoDbStreamsClient.SHARDS.size());
+ }
+
+ @Test
+ public void listShardWithTokenPagination() {
+ int maxResults = (InMemoryDynamoDbStreamsClient.SHARDS.size() / 2)
+ + (InMemoryDynamoDbStreamsClient.SHARDS.size() % 2);
+ ListShardsResponse listShards = springDynamoDBAdapterClient
+ .listShards(
+ builder -> builder.streamName(InMemoryDynamoDbStreamsClient.STREAM_ARN).maxResults(maxResults))
+ .join();
+
+ assertThat(listShards.nextToken()).isNotNull();
+ assertThat(listShards.shards()).hasSize(maxResults);
+
+ listShards = springDynamoDBAdapterClient
+ .listShards(ListShardsRequest.builder().nextToken(listShards.nextToken()).build()).join();
+
+ assertThat(listShards.nextToken()).isNull();
+ assertThat(listShards.shards()).hasSize(InMemoryDynamoDbStreamsClient.SHARDS.size() - maxResults);
+ }
+
+ @Test
+ public void listShardsWithShardIdPagination() {
+ ListShardsResponse listShards = springDynamoDBAdapterClient
+ .listShards(builder -> builder.streamName(InMemoryDynamoDbStreamsClient.STREAM_ARN)
+ .exclusiveStartShardId(InMemoryDynamoDbStreamsClient.SHARDS.get(2).shardId()).maxResults(1))
+ .join();
+
+ assertThat(listShards.nextToken()).isNotNull();
+ assertThat(listShards.shards()).hasSize(1);
+ assertThat(listShards.shards().get(0).shardId())
+ .isEqualTo(InMemoryDynamoDbStreamsClient.SHARDS.get(3).shardId());
+ }
+
+ private static class InMemoryDynamoDbStreamsClient implements DynamoDbStreamsClient {
+
+ private static final String TABLE_NAME = "test-streams";
+
+ private static final String STREAM_LABEL = "2020-10-21T11:49:13.355";
+
+ private static final String STREAM_ARN = String.format("arn:aws:dynamodb:%s:%s:table/%s/stream/%s",
+ Region.US_EAST_1.id(), "000000000000", TABLE_NAME, STREAM_LABEL);
+
+ private static final List SHARDS = Arrays.asList(
+ buildShard("shardId-00000001603195033866-c5d0c2b1", "51100000000002515059163",
+ "51300000000002521847055").build(),
+ buildShard("shardId-00000001603208699318-b15c42af", "804300000000026046960744",
+ "804300000000026046960744").parentShardId("shardId-00000001603195033866-c5d0c2b1").build(),
+ buildShard("shardId-00000001603223404428-90b80e6c", "1613900000000033324335703",
+ "1613900000000033324335703").parentShardId("shardId-00000001603208699318-b15c42af").build(),
+ buildShard("shardId-00000001603237029376-bd9c40dd", "2364600000000001701561758",
+ "2364600000000001701561758").parentShardId("shardId-00000001603223404428-90b80e6c").build(),
+ buildShard("shardId-00000001603249855034-b917a47f", "3071400000000035046301998",
+ "3071400000000035046301998").parentShardId("shardId-00000001603237029376-bd9c40dd").build());
+
+ private final StreamDescription.Builder streamDescription = StreamDescription.builder().streamArn(STREAM_ARN)
+ .streamLabel(STREAM_LABEL).streamViewType(StreamViewType.KEYS_ONLY)
+ .creationRequestDateTime(Instant.now()).tableName(TABLE_NAME)
+ .keySchema(KeySchemaElement.builder().attributeName("name").keyType(KeyType.HASH).build())
+ .streamStatus(StreamStatus.ENABLED);
+
+ private static Shard.Builder buildShard(String parentShardIdString, String startingSequenceNumber,
+ String endingSequenceNumber) {
+
+ return Shard.builder().shardId(parentShardIdString).sequenceNumberRange(builder -> builder
+ .startingSequenceNumber(startingSequenceNumber).endingSequenceNumber(endingSequenceNumber));
+ }
+
+ @Override
+ public DescribeStreamResponse describeStream(DescribeStreamRequest request) {
+ // Invalid StreamArn (Service: AmazonDynamoDBStreams; Status Code: 400;
+ // Error Code: ValidationException; Request ID: ; Proxy: null)
+ if (!STREAM_ARN.equals(request.streamArn())) {
+ throw software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException.builder()
+ .awsErrorDetails(
+ AwsErrorDetails.builder().errorCode("404").errorMessage("Invalid StreamArn").build())
+ .build();
+ }
+
+ int limit = request.limit() != null ? request.limit() : 100;
+
+ int shardIndex = request.exclusiveStartShardId() != null ? findShardIndex(request.exclusiveStartShardId())
+ : 0;
+ int lastShardIndex = Math.min(shardIndex + limit, SHARDS.size());
+ streamDescription.shards(SHARDS.subList(shardIndex, lastShardIndex)).lastEvaluatedShardId(
+ lastShardIndex != SHARDS.size() ? SHARDS.get(lastShardIndex).shardId() : null);
+
+ return DescribeStreamResponse.builder().streamDescription(streamDescription.build()).build();
+ }
+
+ private static int findShardIndex(String shardId) {
+ int i = 0;
+ // checkstyle forced this way of writing it
+ while (i < SHARDS.size() && !SHARDS.get(i).shardId().equals(shardId)) {
+ i++;
+ }
+
+ if (i + 1 >= SHARDS.size()) {
+ throw new RuntimeException("ShardId not found");
+ }
+
+ return i + 1;
+ }
+
+ @Override
+ public String serviceName() {
+ return DynamoDbStreamsClient.SERVICE_NAME;
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ }
+
+}
diff --git a/spring-cloud-aws-kinesis/src/test/resources/logback.xml b/spring-cloud-aws-kinesis/src/test/resources/logback.xml
index 5566e44d3..ccfdafa21 100644
--- a/spring-cloud-aws-kinesis/src/test/resources/logback.xml
+++ b/spring-cloud-aws-kinesis/src/test/resources/logback.xml
@@ -13,4 +13,5 @@
+
diff --git a/spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis-client/pom.xml b/spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis-client/pom.xml
index 6cccc6cfe..199895b00 100644
--- a/spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis-client/pom.xml
+++ b/spring-cloud-aws-starters/spring-cloud-aws-starter-integration-kinesis-client/pom.xml
@@ -26,6 +26,10 @@
software.amazon.kinesis
amazon-kinesis-client
+
+ com.amazonaws
+ dynamodb-streams-kinesis-adapter
+
org.springframework.integration
spring-integration-core