records = deduplicateRecords(event.records());
+ if (records.isEmpty()) {
+ requestNext();
+ return;
+ }
+
+ final ShardFetchResult result = createFetchResult(shardId, records, millisBehind);
+ lastQueuedSequenceNumber = result.lastSequenceNumber();
+ resultSink.accept(result);
+ if (paused.compareAndSet(false, true)) {
+ pausedConsumers.add(ShardConsumer.this);
+ }
+ }
+
+ @Override
+ public void onError(final Throwable t) {
+ logSubscriptionError(t);
+ endSubscriptionIfCurrent(generation);
+ }
+
+ @Override
+ public void onComplete() {
+ consumerLogger.debug("Enhanced Fan-Out subscription completed normally for shard [{}]", shardId);
+ endSubscriptionIfCurrent(generation);
+ }
+ }
+ }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisConsumerClient.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisConsumerClient.java
new file mode 100644
index 000000000000..ff1e8a0e4fa6
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisConsumerClient.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import org.apache.nifi.logging.ComponentLog;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Abstract base for Kinesis consumer clients. Provides per-shard result queues and
+ * shard-claim infrastructure used by both polling (shared-throughput) and EFO
+ * (enhanced-fan-out) implementations.
+ *
+ * Results are stored in per-shard FIFO queues rather than a single shared queue.
+ * This guarantees that results for the same shard are always consumed in enqueue order,
+ * preventing out-of-order delivery when concurrent tasks cannot claim the same shard.
+ */
+abstract class KinesisConsumerClient {
+
+ private static final long DIAGNOSTIC_INTERVAL_NANOS = TimeUnit.SECONDS.toNanos(30);
+
+ protected final KinesisClient kinesisClient;
+ protected final ComponentLog logger;
+ private final Map> shardQueues = new ConcurrentHashMap<>();
+ private final Map shardLocks = new ConcurrentHashMap<>();
+ private final Semaphore resultNotification = new Semaphore(0);
+ protected final Set shardsInFlight = ConcurrentHashMap.newKeySet();
+
+ private volatile long lastDiagnosticLogNanos;
+ private volatile Instant timestampForInitialPosition;
+
+ KinesisConsumerClient(final KinesisClient kinesisClient, final ComponentLog logger) {
+ this.kinesisClient = kinesisClient;
+ this.logger = logger;
+ }
+
+ void setTimestampForInitialPosition(final Instant timestamp) {
+ this.timestampForInitialPosition = timestamp;
+ }
+
+ Instant getTimestampForInitialPosition() {
+ return timestampForInitialPosition;
+ }
+
+ void initialize(final KinesisAsyncClient asyncClient, final String streamName, final String consumerName) {
+ }
+
+ abstract void startFetches(List shards, String streamName, int batchSize,
+ String initialStreamPosition, KinesisShardManager shardManager);
+
+ abstract boolean hasPendingFetches();
+
+ abstract void acknowledgeResults(List results);
+
+ abstract void rollbackResults(List results);
+
+ abstract void removeUnownedShards(Set ownedShards);
+
+ abstract void logDiagnostics(int ownedCount, int cachedShardCount);
+
+ Object getShardLock(final String shardId) {
+ return shardLocks.computeIfAbsent(shardId, k -> new Object());
+ }
+
+ void close() {
+ shardQueues.clear();
+ shardLocks.clear();
+ resultNotification.drainPermits();
+ shardsInFlight.clear();
+ }
+
+ void enqueueResult(final ShardFetchResult result) {
+ shardQueues.computeIfAbsent(result.shardId(), k -> new ConcurrentLinkedQueue<>()).add(result);
+ resultNotification.release();
+ }
+
+ ShardFetchResult pollShardResult(final String shardId) {
+ final Queue queue = shardQueues.get(shardId);
+ final ShardFetchResult result = queue == null ? null : queue.poll();
+ if (result != null) {
+ onResultPolled();
+ }
+ return result;
+ }
+
+ protected void onResultPolled() {
+ }
+
+ int drainShardQueue(final String shardId) {
+ final Queue queue = shardQueues.get(shardId);
+ if (queue == null) {
+ return 0;
+ }
+ int drained = 0;
+ while (queue.poll() != null) {
+ drained++;
+ }
+ return drained;
+ }
+
+ ShardFetchResult pollAnyResult(final long timeout, final TimeUnit unit) throws InterruptedException {
+ final long deadlineNanos = System.nanoTime() + unit.toNanos(timeout);
+ while (System.nanoTime() < deadlineNanos) {
+ for (final Queue queue : shardQueues.values()) {
+ final ShardFetchResult result = queue.poll();
+ if (result != null) {
+ onResultPolled();
+ return result;
+ }
+ }
+ final long remainingMs = TimeUnit.NANOSECONDS.toMillis(deadlineNanos - System.nanoTime());
+ if (remainingMs <= 0) {
+ break;
+ }
+ resultNotification.drainPermits();
+ resultNotification.tryAcquire(Math.min(remainingMs, 100), TimeUnit.MILLISECONDS);
+ }
+ return null;
+ }
+
+ List getShardIdsWithResults() {
+ final List ids = new ArrayList<>();
+ for (final Map.Entry> entry : shardQueues.entrySet()) {
+ if (!entry.getValue().isEmpty()) {
+ ids.add(entry.getKey());
+ }
+ }
+ return ids;
+ }
+
+ boolean awaitResults(final long timeout, final TimeUnit unit) throws InterruptedException {
+ resultNotification.drainPermits();
+ return resultNotification.tryAcquire(timeout, unit);
+ }
+
+ int totalQueuedResults() {
+ int total = 0;
+ for (final Queue queue : shardQueues.values()) {
+ total += queue.size();
+ }
+ return total;
+ }
+
+ boolean hasQueuedResults() {
+ for (final Queue queue : shardQueues.values()) {
+ if (!queue.isEmpty()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ boolean claimShard(final String shardId) {
+ return shardsInFlight.add(shardId);
+ }
+
+ void releaseShards(final Collection shardIds) {
+ shardsInFlight.removeAll(shardIds);
+ }
+
+ protected static ShardFetchResult createFetchResult(final String shardId, final List records, final long millisBehindLatest) {
+ return new ShardFetchResult(shardId, ProducerLibraryDeaggregator.deaggregate(shardId, records), millisBehindLatest);
+ }
+
+ long drainDeduplicatedEventCount() {
+ return 0;
+ }
+
+ protected boolean shouldLogDiagnostics() {
+ final long now = System.nanoTime();
+ if (now < DIAGNOSTIC_INTERVAL_NANOS + lastDiagnosticLogNanos) {
+ return false;
+ }
+ lastDiagnosticLogNanos = now;
+ return true;
+ }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/KinesisRecordMetadata.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisRecordMetadata.java
similarity index 70%
rename from nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/KinesisRecordMetadata.java
rename to nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisRecordMetadata.java
index 9fdc480727f1..bdbeb697ec5e 100644
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/KinesisRecordMetadata.java
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisRecordMetadata.java
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.nifi.processors.aws.kinesis.converter;
+package org.apache.nifi.processors.aws.kinesis;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.MapRecord;
@@ -22,16 +22,15 @@
import org.apache.nifi.serialization.record.RecordField;
import org.apache.nifi.serialization.record.RecordFieldType;
import org.apache.nifi.serialization.record.RecordSchema;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-final class KinesisRecordMetadata {
+public final class KinesisRecordMetadata {
- static final String METADATA = "kinesisMetadata";
- static final String APPROX_ARRIVAL_TIMESTAMP = "approximateArrival";
+ public static final String METADATA = "kinesisMetadata";
+ public static final String APPROX_ARRIVAL_TIMESTAMP = "approximateArrival";
private static final String STREAM = "stream";
private static final String SHARD_ID = "shardId";
@@ -49,28 +48,28 @@ final class KinesisRecordMetadata {
private static final RecordField FIELD_APPROX_ARRIVAL_TIMESTAMP = new RecordField(APPROX_ARRIVAL_TIMESTAMP, RecordFieldType.TIMESTAMP.getDataType());
private static final RecordSchema SCHEMA_METADATA = new SimpleRecordSchema(List.of(
- FIELD_STREAM,
- FIELD_SHARD_ID,
- FIELD_SEQUENCE_NUMBER,
- FIELD_SUB_SEQUENCE_NUMBER,
- FIELD_SHARDED_SEQUENCE_NUMBER,
- FIELD_PARTITION_KEY,
- FIELD_APPROX_ARRIVAL_TIMESTAMP));
+ FIELD_STREAM,
+ FIELD_SHARD_ID,
+ FIELD_SEQUENCE_NUMBER,
+ FIELD_SUB_SEQUENCE_NUMBER,
+ FIELD_SHARDED_SEQUENCE_NUMBER,
+ FIELD_PARTITION_KEY,
+ FIELD_APPROX_ARRIVAL_TIMESTAMP));
- static final RecordField FIELD_METADATA = new RecordField(METADATA, RecordFieldType.RECORD.getRecordDataType(SCHEMA_METADATA));
+ public static final RecordField FIELD_METADATA = new RecordField(METADATA, RecordFieldType.RECORD.getRecordDataType(SCHEMA_METADATA));
- static Record composeMetadataObject(final KinesisClientRecord kinesisRecord, final String streamName, final String shardId) {
+ public static Record composeMetadataObject(final UserRecord record, final String streamName, final String shardId) {
final Map metadata = new HashMap<>(7, 1.0f);
metadata.put(STREAM, streamName);
metadata.put(SHARD_ID, shardId);
- metadata.put(SEQUENCE_NUMBER, kinesisRecord.sequenceNumber());
- metadata.put(SUB_SEQUENCE_NUMBER, kinesisRecord.subSequenceNumber());
- metadata.put(SHARDED_SEQUENCE_NUMBER, "%s%020d".formatted(kinesisRecord.sequenceNumber(), kinesisRecord.subSequenceNumber()));
- metadata.put(PARTITION_KEY, kinesisRecord.partitionKey());
+ metadata.put(SEQUENCE_NUMBER, record.sequenceNumber());
+ metadata.put(SUB_SEQUENCE_NUMBER, record.subSequenceNumber());
+ metadata.put(SHARDED_SEQUENCE_NUMBER, "%s%020d".formatted(record.sequenceNumber(), record.subSequenceNumber()));
+ metadata.put(PARTITION_KEY, record.partitionKey());
- if (kinesisRecord.approximateArrivalTimestamp() != null) {
- metadata.put(APPROX_ARRIVAL_TIMESTAMP, kinesisRecord.approximateArrivalTimestamp().toEpochMilli());
+ if (record.approximateArrivalTimestamp() != null) {
+ metadata.put(APPROX_ARRIVAL_TIMESTAMP, record.approximateArrivalTimestamp().toEpochMilli());
}
return new MapRecord(SCHEMA_METADATA, metadata);
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisShardManager.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisShardManager.java
new file mode 100644
index 000000000000..53fbfd41bd3c
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/KinesisShardManager.java
@@ -0,0 +1,568 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
+import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException;
+import software.amazon.awssdk.services.dynamodb.model.GetItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.GetItemResponse;
+import software.amazon.awssdk.services.dynamodb.model.QueryRequest;
+import software.amazon.awssdk.services.dynamodb.model.QueryResponse;
+import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import java.math.BigInteger;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Coordinates shard ownership and checkpoints across clustered processor instances using
+ * a single DynamoDB table.
+ *
+ * The table stores two record types under the same stream hash key:
+ *
+ * - Shard rows: key {@code (streamName, shardId)} with lease/checkpoint fields
+ * - Node heartbeat rows: key {@code (streamName, "__node__#")}
+ *
+ *
+ * Lease lifecycle used during refresh:
+ *
+ * - Discover active shard leases and identify currently available shards
+ * - Compute fair-share target from active node heartbeats
+ * - If this node is over target, mark excess shards for graceful relinquish
+ * - Continue renewing leases for owned shards and draining relinquishing shards
+ * - After drain deadline, explicitly release relinquishing shards
+ * - Acquire available shards until fair-share target is reached
+ *
+ *
+ * Graceful relinquish is designed to reduce duplicate replay at rebalance boundaries by:
+ * (a) stopping new fetches immediately (shard removed from {@code ownedShards}),
+ * (b) briefly retaining lease ownership to allow in-flight work to finish,
+ * then (c) explicitly releasing the lease for fast handoff.
+ *
+ *
Shard split/merge: this implementation does not enforce parent-before-child
+ * ordering. When a shard is split or shards are merged, child shards become eligible for
+ * consumption immediately alongside any still-active parent shards. Callers that require strict
+ * ordering across split/merge boundaries would need to defer child shard assignment until the
+ * parent shard's {@code SHARD_END} has been reached and checkpointed.
+ */
+final class KinesisShardManager {
+
+ private static final long DEFAULT_SHARD_CACHE_MILLIS = 60_000;
+ private static final long DEFAULT_LEASE_DURATION_MILLIS = 30_000;
+ private static final long DEFAULT_LEASE_REFRESH_INTERVAL_MILLIS = 10_000;
+ private static final long DEFAULT_NODE_HEARTBEAT_EXPIRATION_MILLIS =
+ DEFAULT_LEASE_DURATION_MILLIS + DEFAULT_LEASE_REFRESH_INTERVAL_MILLIS;
+
+ private final KinesisClient kinesisClient;
+ private final DynamoDbClient dynamoDbClient;
+ private final ComponentLog logger;
+ private final String nodeId;
+ private final String checkpointTableName;
+ private final String streamName;
+ private final long shardCacheMillis;
+ private final long leaseDurationMillis;
+ private final long leaseRefreshIntervalMillis;
+ private final long nodeHeartbeatExpirationMillis;
+ private final long relinquishDrainMillis;
+
+ private volatile ShardCache shardCache = new ShardCache(List.of(), Instant.EPOCH);
+ private final Set ownedShards = ConcurrentHashMap.newKeySet();
+ private final Map pendingRelinquishDeadlines = new ConcurrentHashMap<>();
+ private final AtomicBoolean leaseRefreshInProgress = new AtomicBoolean(false);
+ private final Map highestWrittenCheckpoints = new ConcurrentHashMap<>();
+ private volatile Instant lastLeaseRefresh = Instant.EPOCH;
+ private volatile String activeCheckpointTableName;
+
+ KinesisShardManager(final KinesisClient kinesisClient, final DynamoDbClient dynamoDbClient, final ComponentLog logger,
+ final String checkpointTableName, final String streamName) {
+ this(kinesisClient, dynamoDbClient, logger, checkpointTableName, streamName,
+ DEFAULT_SHARD_CACHE_MILLIS, DEFAULT_LEASE_DURATION_MILLIS,
+ DEFAULT_LEASE_REFRESH_INTERVAL_MILLIS, DEFAULT_NODE_HEARTBEAT_EXPIRATION_MILLIS);
+ }
+
+ KinesisShardManager(final KinesisClient kinesisClient, final DynamoDbClient dynamoDbClient, final ComponentLog logger,
+ final String checkpointTableName, final String streamName, final long shardCacheMillis,
+ final long leaseDurationMillis, final long leaseRefreshIntervalMillis, final long nodeHeartbeatExpirationMillis) {
+ this.kinesisClient = kinesisClient;
+ this.dynamoDbClient = dynamoDbClient;
+ this.logger = logger;
+ this.nodeId = UUID.randomUUID().toString();
+ this.checkpointTableName = checkpointTableName;
+ this.streamName = streamName;
+ this.shardCacheMillis = shardCacheMillis;
+ this.leaseDurationMillis = leaseDurationMillis;
+ this.leaseRefreshIntervalMillis = leaseRefreshIntervalMillis;
+ this.nodeHeartbeatExpirationMillis = nodeHeartbeatExpirationMillis;
+ this.relinquishDrainMillis = Math.max(2_000L, leaseRefreshIntervalMillis);
+ this.activeCheckpointTableName = checkpointTableName;
+ }
+
+ void ensureCheckpointTableExists() {
+ final CheckpointTableUtils.TableSchema currentSchema =
+ CheckpointTableUtils.getTableSchema(dynamoDbClient, checkpointTableName);
+ logger.debug("Checkpoint table [{}] detected as {}", checkpointTableName, currentSchema);
+
+ final LegacyCheckpointMigrator migrator =
+ new LegacyCheckpointMigrator(dynamoDbClient, checkpointTableName, streamName, nodeId, logger);
+
+ switch (currentSchema) {
+ case NOT_FOUND -> {
+ final String orphanedMigration = migrator.findMigrationTable();
+ if (orphanedMigration == null) {
+ CheckpointTableUtils.createNewSchemaTable(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.waitForTableActive(dynamoDbClient, logger, checkpointTableName);
+ } else {
+ logger.info("Found orphaned migration table [{}]; renaming to [{}]",
+ orphanedMigration, checkpointTableName);
+ migrator.renameMigrationTable(orphanedMigration);
+ }
+ }
+ case NEW -> {
+ CheckpointTableUtils.waitForTableActive(dynamoDbClient, logger, checkpointTableName);
+ migrator.cleanupLingeringMigration();
+ }
+ case LEGACY -> {
+ migrator.migrateAndRename();
+ }
+ default -> throw new ProcessException(
+ "Unsupported DynamoDB schema for checkpoint table [%s]".formatted(checkpointTableName));
+ }
+
+ activeCheckpointTableName = checkpointTableName;
+ logger.info("Using checkpoint table [{}] for stream [{}]", activeCheckpointTableName, streamName);
+ }
+
+ List getShards() {
+ final ShardCache current = shardCache;
+ if (!current.shards().isEmpty() && Instant.now().toEpochMilli() < current.refreshTime().toEpochMilli() + shardCacheMillis) {
+ return current.shards();
+ }
+ return refreshShards();
+ }
+
+ int getCachedShardCount() {
+ return shardCache.shards().size();
+ }
+
+ Set getOwnedShardIds() {
+ return Collections.unmodifiableSet(ownedShards);
+ }
+
+ List getOwnedShards() {
+ final List result = new ArrayList<>();
+ for (final Shard shard : getShards()) {
+ if (ownedShards.contains(shard.shardId())) {
+ result.add(shard);
+ }
+ }
+ return result;
+ }
+
+ boolean shouldProcessFetchedResult(final String shardId) {
+ return ownedShards.contains(shardId) || pendingRelinquishDeadlines.containsKey(shardId);
+ }
+
+ void refreshLeasesIfNecessary(final int clusterMemberCount) {
+ if (Instant.now().toEpochMilli() < leaseRefreshIntervalMillis + lastLeaseRefresh.toEpochMilli()) {
+ return;
+ }
+
+ if (!leaseRefreshInProgress.compareAndSet(false, true)) {
+ return;
+ }
+
+ try {
+ final List allShards = getShards();
+ final Set currentShardIds = new HashSet<>();
+ for (final Shard shard : allShards) {
+ currentShardIds.add(shard.shardId());
+ }
+
+ final long now = Instant.now().toEpochMilli();
+ updateNodeHeartbeat(now);
+ final Map> ownerToShards = new HashMap<>();
+ final List availableShardIds = new ArrayList<>();
+
+ final Map> leaseItemsByShardId = queryAllLeaseItems();
+ for (final String shardId : currentShardIds) {
+ final Map item = leaseItemsByShardId.get(shardId);
+ if (item != null && item.containsKey("leaseOwner")) {
+ final String owner = item.get("leaseOwner").s();
+ final AttributeValue expiryAttr = item.get("leaseExpiry");
+ final long expiry = expiryAttr == null ? 0 : Long.parseLong(expiryAttr.n());
+ if (expiry >= now) {
+ ownerToShards.computeIfAbsent(owner, k -> new ArrayList<>()).add(shardId);
+ } else {
+ availableShardIds.add(shardId);
+ }
+ } else {
+ availableShardIds.add(shardId);
+ }
+ }
+
+ final Set stillOwned = new HashSet<>(ownerToShards.getOrDefault(nodeId, List.of()));
+ ownedShards.retainAll(stillOwned);
+ pendingRelinquishDeadlines.keySet().removeIf(shardId -> !currentShardIds.contains(shardId) || !stillOwned.contains(shardId));
+
+ final int heartbeatNodes = countActiveNodes(now);
+ final int totalOwners = Math.max(heartbeatNodes, Math.max(1, clusterMemberCount));
+ final int targetCount = (allShards.size() + totalOwners - 1) / totalOwners;
+
+ final List currentlyOwned = new ArrayList<>(ownerToShards.getOrDefault(nodeId, List.of()));
+ final int excessCount = Math.max(0, currentlyOwned.size() - targetCount);
+ if (excessCount > 0) {
+ for (int index = 0; index < excessCount && index < currentlyOwned.size(); index++) {
+ final String shardToRelinquish = currentlyOwned.get(index);
+ if (!pendingRelinquishDeadlines.containsKey(shardToRelinquish)) {
+ pendingRelinquishDeadlines.put(shardToRelinquish, now + relinquishDrainMillis);
+ ownedShards.remove(shardToRelinquish);
+ logger.info("Starting graceful relinquish for shard {}", shardToRelinquish);
+ }
+ }
+ }
+
+ for (final String shardId : ownedShards) {
+ tryAcquireLease(shardId);
+ }
+
+ for (final Map.Entry pendingRelinquish : new ArrayList<>(pendingRelinquishDeadlines.entrySet())) {
+ final String shardId = pendingRelinquish.getKey();
+ final long relinquishDeadline = pendingRelinquish.getValue();
+ if (!stillOwned.contains(shardId)) {
+ pendingRelinquishDeadlines.remove(shardId);
+ continue;
+ }
+
+ if (now < relinquishDeadline) {
+ tryAcquireLease(shardId);
+ } else {
+ try {
+ releaseLease(shardId);
+ pendingRelinquishDeadlines.remove(shardId);
+ logger.info("Completed graceful relinquish for shard {}", shardId);
+ } catch (final Exception e) {
+ logger.warn("Failed to complete graceful relinquish for shard {}", shardId, e);
+ }
+ }
+ }
+
+ for (final String shardId : availableShardIds) {
+ if (ownedShards.size() >= targetCount) {
+ break;
+ }
+ if (pendingRelinquishDeadlines.containsKey(shardId)) {
+ continue;
+ }
+ if (tryAcquireLease(shardId)) {
+ ownedShards.add(shardId);
+ }
+ }
+
+ ownedShards.removeIf(id -> !currentShardIds.contains(id));
+ lastLeaseRefresh = Instant.now();
+ } finally {
+ leaseRefreshInProgress.set(false);
+ }
+ }
+
+ String readCheckpoint(final String shardId) {
+ final GetItemRequest getItemRequest = GetItemRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .key(checkpointKey(shardId))
+ .consistentRead(true)
+ .build();
+ final GetItemResponse response = dynamoDbClient.getItem(getItemRequest);
+
+ if (response.hasItem() && response.item().containsKey("sequenceNumber")) {
+ final String value = response.item().get("sequenceNumber").s();
+ if (isValidSequenceNumber(value)) {
+ logger.debug("Read checkpoint for shard {}: {}", shardId, value);
+ return value;
+ }
+ logger.warn("Ignoring non-numeric checkpoint [{}] for shard {} in table [{}]", value, shardId, activeCheckpointTableName);
+ } else {
+ logger.debug("No checkpoint found for shard {} in table [{}]", shardId, activeCheckpointTableName);
+ }
+ return null;
+ }
+
+ void writeCheckpoints(final Map checkpoints) {
+ for (final Map.Entry entry : checkpoints.entrySet()) {
+ writeCheckpoint(entry.getKey(), entry.getValue());
+ }
+ }
+
+ void releaseAllLeases() {
+ final Set shardsToRelease = new HashSet<>(ownedShards);
+ shardsToRelease.addAll(pendingRelinquishDeadlines.keySet());
+ for (final String shardId : shardsToRelease) {
+ try {
+ releaseLease(shardId);
+ } catch (final Exception e) {
+ logger.warn("Failed to release lease for shard {}", shardId, e);
+ }
+ }
+
+ try {
+ final UpdateItemRequest heartbeatReleaseRequest = UpdateItemRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .key(nodeHeartbeatKey())
+ .updateExpression("REMOVE nodeHeartbeat, lastUpdateTimestamp")
+ .build();
+ dynamoDbClient.updateItem(heartbeatReleaseRequest);
+ } catch (final Exception e) {
+ logger.debug("Failed to clear node heartbeat record for node {}", nodeId, e);
+ }
+ }
+
+ void close() {
+ ownedShards.clear();
+ pendingRelinquishDeadlines.clear();
+ highestWrittenCheckpoints.clear();
+ leaseRefreshInProgress.set(false);
+ lastLeaseRefresh = Instant.EPOCH;
+ shardCache = new ShardCache(List.of(), Instant.EPOCH);
+ }
+
+ private synchronized List refreshShards() {
+ final ShardCache current = shardCache;
+ if (!current.shards().isEmpty() && Instant.now().toEpochMilli() < current.refreshTime().toEpochMilli() + shardCacheMillis) {
+ return current.shards();
+ }
+
+ final List allShards = new ArrayList<>();
+ String nextToken = null;
+ do {
+ final ListShardsRequest request = nextToken != null
+ ? ListShardsRequest.builder().nextToken(nextToken).build()
+ : ListShardsRequest.builder().streamName(streamName).build();
+
+ final ListShardsResponse response = kinesisClient.listShards(request);
+ allShards.addAll(response.shards());
+ nextToken = response.nextToken();
+ } while (nextToken != null);
+
+ logger.debug("ListShards returned {} shards for stream {}", allShards.size(), streamName);
+ shardCache = new ShardCache(allShards, Instant.now());
+ return allShards;
+ }
+
+ private static boolean isValidSequenceNumber(final String value) {
+ if (value == null || value.isEmpty()) {
+ return false;
+ }
+ for (int idx = 0; idx < value.length(); idx++) {
+ if (!Character.isDigit(value.charAt(idx))) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private boolean tryAcquireLease(final String shardId) {
+ final long now = Instant.now().toEpochMilli();
+ final long expiry = now + leaseDurationMillis;
+ final AttributeValue ownerVal = AttributeValue.builder().s(nodeId).build();
+ final AttributeValue expiryVal = AttributeValue.builder().n(String.valueOf(expiry)).build();
+ final AttributeValue nowVal = AttributeValue.builder().n(String.valueOf(now)).build();
+
+ try {
+ final UpdateItemRequest leaseRequest = UpdateItemRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .key(checkpointKey(shardId))
+ .updateExpression("SET leaseOwner = :owner, leaseExpiry = :exp, lastUpdateTimestamp = :ts")
+ .conditionExpression("attribute_not_exists(leaseOwner) OR leaseOwner = :owner OR leaseExpiry < :now")
+ .expressionAttributeValues(Map.of(
+ ":owner", ownerVal,
+ ":exp", expiryVal,
+ ":now", nowVal,
+ ":ts", nowVal))
+ .build();
+ dynamoDbClient.updateItem(leaseRequest);
+ return true;
+ } catch (final ConditionalCheckFailedException e) {
+ return false;
+ } catch (final Exception e) {
+ logger.warn("Failed to acquire lease for shard {}", shardId, e);
+ return false;
+ }
+ }
+
+ private void updateNodeHeartbeat(final long now) {
+ final UpdateItemRequest heartbeatRequest = UpdateItemRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .key(nodeHeartbeatKey())
+ .updateExpression("SET nodeHeartbeat = :heartbeat, lastUpdateTimestamp = :ts")
+ .expressionAttributeValues(Map.of(
+ ":heartbeat", AttributeValue.builder().n(String.valueOf(now)).build(),
+ ":ts", AttributeValue.builder().n(String.valueOf(now)).build()))
+ .build();
+ dynamoDbClient.updateItem(heartbeatRequest);
+ }
+
+ private Map> queryAllLeaseItems() {
+ final Map> itemsByShardId = new HashMap<>();
+ final QueryRequest.Builder queryBuilder = QueryRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .consistentRead(true)
+ .keyConditionExpression("streamName = :streamName")
+ .expressionAttributeValues(Map.of(
+ ":streamName", AttributeValue.builder().s(streamName).build()));
+
+ Map exclusiveStartKey = null;
+ do {
+ final QueryRequest queryRequest = exclusiveStartKey == null
+ ? queryBuilder.build()
+ : queryBuilder.exclusiveStartKey(exclusiveStartKey).build();
+ final QueryResponse queryResponse = dynamoDbClient.query(queryRequest);
+ for (final Map item : queryResponse.items()) {
+ final AttributeValue shardIdAttr = item.get(CheckpointTableUtils.ATTR_SHARD_ID);
+ if (shardIdAttr == null) {
+ continue;
+ }
+ final String shardId = shardIdAttr.s();
+ if (shardId.startsWith(CheckpointTableUtils.NODE_HEARTBEAT_PREFIX)
+ || CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID.equals(shardId)) {
+ continue;
+ }
+ itemsByShardId.put(shardId, item);
+ }
+
+ exclusiveStartKey = queryResponse.lastEvaluatedKey();
+ } while (exclusiveStartKey != null && !exclusiveStartKey.isEmpty());
+
+ return itemsByShardId;
+ }
+
+ private int countActiveNodes(final long now) {
+ final QueryRequest.Builder queryRequestBuilder = QueryRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .consistentRead(true)
+ .keyConditionExpression("streamName = :streamName AND begins_with(shardId, :nodePrefix)")
+ .expressionAttributeValues(Map.of(
+ ":streamName", AttributeValue.builder().s(streamName).build(),
+ ":nodePrefix", AttributeValue.builder().s(CheckpointTableUtils.NODE_HEARTBEAT_PREFIX).build()));
+
+ Map exclusiveStartKey = null;
+ int activeNodes = 0;
+ do {
+ final QueryRequest queryRequest = exclusiveStartKey == null
+ ? queryRequestBuilder.build()
+ : queryRequestBuilder.exclusiveStartKey(exclusiveStartKey).build();
+ final QueryResponse queryResponse = dynamoDbClient.query(queryRequest);
+ for (final Map item : queryResponse.items()) {
+ final AttributeValue heartbeatValue = item.get("nodeHeartbeat");
+ if (heartbeatValue == null) {
+ continue;
+ }
+
+ final long heartbeatMillis = Long.parseLong(heartbeatValue.n());
+ if (now <= heartbeatMillis + nodeHeartbeatExpirationMillis) {
+ activeNodes++;
+ }
+ }
+
+ exclusiveStartKey = queryResponse.lastEvaluatedKey();
+ } while (exclusiveStartKey != null && !exclusiveStartKey.isEmpty());
+
+ return Math.max(1, activeNodes);
+ }
+
+ private void writeCheckpoint(final String shardId, final BigInteger checkpoint) {
+ final BigInteger written = highestWrittenCheckpoints.compute(shardId,
+ (key, existing) -> persistIfHigher(shardId, checkpoint, existing));
+
+ if (written != null && checkpoint.compareTo(written) < 0) {
+ logger.debug("Skipped checkpoint regression for shard {} (highest: {}, attempted: {})", shardId, written, checkpoint);
+ }
+ }
+
+ /**
+ * Writes the checkpoint to DynamoDB if it is higher than the existing value. Returns the
+ * new highest checkpoint on success, or {@code existing} if the checkpoint was lower or
+ * the write failed.
+ */
+ private BigInteger persistIfHigher(final String shardId, final BigInteger checkpoint, final BigInteger existing) {
+ if (existing != null && checkpoint.max(existing).equals(existing)) {
+ return existing;
+ }
+
+ try {
+ final long now = Instant.now().toEpochMilli();
+ final UpdateItemRequest checkpointRequest = UpdateItemRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .key(checkpointKey(shardId))
+ .updateExpression("SET sequenceNumber = :seq, lastUpdateTimestamp = :ts, leaseExpiry = :exp")
+ .conditionExpression("leaseOwner = :owner")
+ .expressionAttributeValues(Map.of(
+ ":seq", AttributeValue.builder().s(checkpoint.toString()).build(),
+ ":ts", AttributeValue.builder().n(String.valueOf(now)).build(),
+ ":exp", AttributeValue.builder().n(String.valueOf(now + leaseDurationMillis)).build(),
+ ":owner", AttributeValue.builder().s(nodeId).build()))
+ .build();
+ dynamoDbClient.updateItem(checkpointRequest);
+ logger.debug("Checkpointed shard {} at sequence {}", shardId, checkpoint);
+
+ return checkpoint;
+ } catch (final ConditionalCheckFailedException e) {
+ logger.warn("Lost lease on shard {} during checkpoint; another node may have taken it", shardId);
+ } catch (final Exception e) {
+ logger.error("Failed to write checkpoint for shard {}", shardId, e);
+ }
+
+ return existing;
+ }
+
+ private Map checkpointKey(final String shardId) {
+ return Map.of(
+ CheckpointTableUtils.ATTR_STREAM_NAME, AttributeValue.builder().s(streamName).build(),
+ CheckpointTableUtils.ATTR_SHARD_ID, AttributeValue.builder().s(shardId).build());
+ }
+
+ private void releaseLease(final String shardId) {
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(activeCheckpointTableName)
+ .key(checkpointKey(shardId))
+ .updateExpression("REMOVE leaseOwner, leaseExpiry")
+ .conditionExpression("leaseOwner = :owner")
+ .expressionAttributeValues(Map.of(":owner", AttributeValue.builder().s(nodeId).build()))
+ .build();
+ dynamoDbClient.updateItem(request);
+ }
+
+ private Map nodeHeartbeatKey() {
+ return checkpointKey(CheckpointTableUtils.NODE_HEARTBEAT_PREFIX + nodeId);
+ }
+
+ private record ShardCache(List shards, Instant refreshTime) { }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/LegacyCheckpointMigrator.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/LegacyCheckpointMigrator.java
new file mode 100644
index 000000000000..d994094d2c47
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/LegacyCheckpointMigrator.java
@@ -0,0 +1,432 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.exception.ProcessException;
+import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException;
+import software.amazon.awssdk.services.dynamodb.model.GetItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.GetItemResponse;
+import software.amazon.awssdk.services.dynamodb.model.PutItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.dynamodb.model.ScanRequest;
+import software.amazon.awssdk.services.dynamodb.model.ScanResponse;
+import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest;
+
+import java.time.Instant;
+import java.util.Map;
+
+/**
+ * Handles one-time migration of checkpoint data from legacy KCL-format DynamoDB tables
+ * to the new composite-key schema used by {@link KinesisShardManager}, including the
+ * table rename lifecycle. Uses distributed locks in the target table to coordinate
+ * migration and rename operations across clustered nodes.
+ */
+final class LegacyCheckpointMigrator {
+
+ private static final String MIGRATION_TABLE_SUFFIX = "_migration";
+ private static final String LEGACY_LEASE_KEY_ATTRIBUTE = "leaseKey";
+ private static final String LEGACY_CHECKPOINT_ATTRIBUTE = "checkpoint";
+ private static final String MIGRATION_STATUS_ATTRIBUTE = "migrationStatus";
+ private static final String MIGRATION_STATUS_IN_PROGRESS = "IN_PROGRESS";
+ private static final String MIGRATION_STATUS_COMPLETE = "COMPLETE";
+ private static final long MIGRATION_LOCK_STALE_MILLIS = 600_000;
+ private static final long MIGRATION_WAIT_MILLIS = 2_000;
+ private static final int MIGRATION_WAIT_MAX_ATTEMPTS = 180;
+ private static final long RENAME_LOCK_STALE_MILLIS = 120_000;
+ private static final long RENAME_POLL_MILLIS = 1_000;
+ private static final int RENAME_POLL_MAX_ATTEMPTS = 60;
+
+ private final DynamoDbClient dynamoDbClient;
+ private final String checkpointTableName;
+ private final String streamName;
+ private final String nodeId;
+ private final ComponentLog logger;
+
+ LegacyCheckpointMigrator(final DynamoDbClient dynamoDbClient, final String checkpointTableName,
+ final String streamName, final String nodeId, final ComponentLog logger) {
+ this.dynamoDbClient = dynamoDbClient;
+ this.checkpointTableName = checkpointTableName;
+ this.streamName = streamName;
+ this.nodeId = nodeId;
+ this.logger = logger;
+ }
+
+ String findMigrationTable() {
+ final String migrationTableName = checkpointTableName + MIGRATION_TABLE_SUFFIX;
+ if (CheckpointTableUtils.getTableSchema(dynamoDbClient, migrationTableName)
+ == CheckpointTableUtils.TableSchema.NEW) {
+ return migrationTableName;
+ }
+ return null;
+ }
+
+ void cleanupLingeringMigration() {
+ final String lingeringMigration = findMigrationTable();
+ if (lingeringMigration == null) {
+ return;
+ }
+ logger.info("Deleting orphaned migration table [{}]; legacy checkpoint table [{}] retains original data", lingeringMigration, checkpointTableName);
+ CheckpointTableUtils.deleteTable(dynamoDbClient, logger, lingeringMigration);
+ }
+
+ void migrateAndRename() {
+ final String existingMigration = findMigrationTable();
+ final String migrationTableName;
+
+ if (existingMigration == null) {
+ migrationTableName = checkpointTableName + MIGRATION_TABLE_SUFFIX;
+ logger.info("Legacy checkpoint table detected; migrating via [{}]", migrationTableName);
+ CheckpointTableUtils.createNewSchemaTable(dynamoDbClient, logger, migrationTableName);
+ CheckpointTableUtils.waitForTableActive(dynamoDbClient, logger, migrationTableName);
+ migrateCheckpoints(checkpointTableName, migrationTableName);
+ } else {
+ migrationTableName = existingMigration;
+ logger.info("Found existing migration table [{}]; completing rename to [{}]",
+ migrationTableName, checkpointTableName);
+ }
+
+ renameMigrationTable(migrationTableName);
+ }
+
+ void renameMigrationTable(final String migrationTableName) {
+ if (acquireRenameLock(migrationTableName)) {
+ CheckpointTableUtils.deleteTable(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.waitForTableDeleted(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.createNewSchemaTable(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.waitForTableActive(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.copyCheckpointItems(dynamoDbClient, logger, migrationTableName, checkpointTableName);
+ CheckpointTableUtils.deleteTable(dynamoDbClient, logger, migrationTableName);
+ } else {
+ waitForTableRenamed(migrationTableName);
+ }
+ }
+
+ private boolean acquireRenameLock(final String migrationTableName) {
+ try {
+ final long now = Instant.now().toEpochMilli();
+ final Map key = migrationMarkerKey();
+ final Map values = Map.of(
+ ":owner", AttributeValue.builder().s(nodeId).build(),
+ ":now", AttributeValue.builder().n(String.valueOf(now)).build());
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(migrationTableName)
+ .key(key)
+ .updateExpression("SET renameOwner = :owner, renameStartedAt = :now")
+ .conditionExpression("attribute_not_exists(renameOwner)")
+ .expressionAttributeValues(values)
+ .build();
+ dynamoDbClient.updateItem(request);
+ logger.info("Acquired rename lock for migration table [{}]", migrationTableName);
+ return true;
+ } catch (final ConditionalCheckFailedException e) {
+ logger.debug("Rename lock already held for migration table [{}]", migrationTableName);
+ return false;
+ } catch (final ResourceNotFoundException e) {
+ logger.debug("Migration table [{}] already deleted; rename must be complete", migrationTableName);
+ return false;
+ }
+ }
+
+ private boolean forceAcquireStaleRenameLock(final String migrationTableName) {
+ try {
+ final long now = Instant.now().toEpochMilli();
+ final long staleThreshold = now - RENAME_LOCK_STALE_MILLIS;
+ final Map key = migrationMarkerKey();
+ final Map values = Map.of(
+ ":owner", AttributeValue.builder().s(nodeId).build(),
+ ":now", AttributeValue.builder().n(String.valueOf(now)).build(),
+ ":staleThreshold", AttributeValue.builder().n(String.valueOf(staleThreshold)).build());
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(migrationTableName)
+ .key(key)
+ .updateExpression("SET renameOwner = :owner, renameStartedAt = :now")
+ .conditionExpression("attribute_exists(renameOwner) AND renameStartedAt < :staleThreshold")
+ .expressionAttributeValues(values)
+ .build();
+ dynamoDbClient.updateItem(request);
+ logger.info("Force-acquired stale rename lock for migration table [{}]", migrationTableName);
+ return true;
+ } catch (final ConditionalCheckFailedException | ResourceNotFoundException e) {
+ return false;
+ }
+ }
+
+ private Map migrationMarkerKey() {
+ return Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID).build());
+ }
+
+ private void waitForTableRenamed(final String migrationTableName) {
+ for (int i = 0; i < RENAME_POLL_MAX_ATTEMPTS; i++) {
+ if (CheckpointTableUtils.getTableSchema(dynamoDbClient, checkpointTableName)
+ == CheckpointTableUtils.TableSchema.NEW) {
+ logger.info("Migration table rename complete; table [{}] is now available", checkpointTableName);
+ return;
+ }
+
+ try {
+ Thread.sleep(RENAME_POLL_MILLIS);
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new ProcessException("Interrupted while waiting for migration table rename to complete", e);
+ }
+ }
+
+ logger.warn("Timed out waiting for migration table rename; attempting stale lock takeover");
+ if (forceAcquireStaleRenameLock(migrationTableName)) {
+ CheckpointTableUtils.deleteTable(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.waitForTableDeleted(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.createNewSchemaTable(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.waitForTableActive(dynamoDbClient, logger, checkpointTableName);
+ CheckpointTableUtils.copyCheckpointItems(dynamoDbClient, logger, migrationTableName, checkpointTableName);
+ CheckpointTableUtils.deleteTable(dynamoDbClient, logger, migrationTableName);
+ } else if (CheckpointTableUtils.getTableSchema(dynamoDbClient, checkpointTableName)
+ == CheckpointTableUtils.TableSchema.NEW) {
+ logger.info("Migration table rename completed during takeover attempt");
+ } else {
+ throw new ProcessException(
+ "Unable to complete migration table rename for [%s]".formatted(checkpointTableName));
+ }
+ }
+
+ private void migrateCheckpoints(final String sourceTableName, final String targetTableName) {
+ if (isMigrationComplete(targetTableName)) {
+ logger.debug("Legacy checkpoint migration already complete for stream [{}]", streamName);
+ return;
+ }
+
+ if (acquireMigrationLock(targetTableName)) {
+ performMigration(sourceTableName, targetTableName);
+ } else {
+ waitForMigrationComplete(targetTableName);
+ if (!isMigrationComplete(targetTableName)) {
+ logger.warn("Migration wait timed out with stale lock; attempting takeover for stream [{}]", streamName);
+ if (forceAcquireStaleMigrationLock(targetTableName)) {
+ performMigration(sourceTableName, targetTableName);
+ } else {
+ throw new ProcessException(
+ "Unable to acquire migration lock for stream [%s]".formatted(streamName));
+ }
+ }
+ }
+ }
+
+ private void performMigration(final String sourceTableName, final String targetTableName) {
+ try {
+ migrateLegacyCheckpoints(sourceTableName, targetTableName);
+ markMigrationComplete(targetTableName);
+ } catch (final Exception e) {
+ clearMigrationLock(targetTableName);
+ throw new ProcessException("Failed to migrate legacy checkpoints to [%s]".formatted(targetTableName), e);
+ }
+ }
+
+ private boolean acquireMigrationLock(final String targetTableName) {
+ try {
+ final long now = Instant.now().toEpochMilli();
+ final PutItemRequest request = PutItemRequest.builder()
+ .tableName(targetTableName)
+ .item(Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID).build(),
+ MIGRATION_STATUS_ATTRIBUTE, AttributeValue.builder().s(MIGRATION_STATUS_IN_PROGRESS).build(),
+ "migrationOwner", AttributeValue.builder().s(nodeId).build(),
+ "migrationStartedAt", AttributeValue.builder().n(String.valueOf(now)).build(),
+ "lastUpdateTimestamp", AttributeValue.builder().n(String.valueOf(now)).build()))
+ .conditionExpression("attribute_not_exists(streamName)")
+ .build();
+ dynamoDbClient.putItem(request);
+ logger.info("Acquired checkpoint migration lock for stream [{}]", streamName);
+ return true;
+ } catch (final ConditionalCheckFailedException e) {
+ return false;
+ }
+ }
+
+ private boolean forceAcquireStaleMigrationLock(final String targetTableName) {
+ try {
+ final long now = Instant.now().toEpochMilli();
+ final long staleThreshold = now - MIGRATION_LOCK_STALE_MILLIS;
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(targetTableName)
+ .key(Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID).build()))
+ .updateExpression("SET migrationOwner = :owner, migrationStartedAt = :now, lastUpdateTimestamp = :now")
+ .conditionExpression("migrationStatus = :inProgress AND migrationStartedAt < :staleThreshold")
+ .expressionAttributeValues(Map.of(
+ ":owner", AttributeValue.builder().s(nodeId).build(),
+ ":now", AttributeValue.builder().n(String.valueOf(now)).build(),
+ ":inProgress", AttributeValue.builder().s(MIGRATION_STATUS_IN_PROGRESS).build(),
+ ":staleThreshold", AttributeValue.builder().n(String.valueOf(staleThreshold)).build()))
+ .build();
+ dynamoDbClient.updateItem(request);
+ logger.info("Force-acquired stale migration lock for stream [{}]", streamName);
+ return true;
+ } catch (final ConditionalCheckFailedException e) {
+ return false;
+ }
+ }
+
+ private void migrateLegacyCheckpoints(final String sourceTableName, final String targetTableName) {
+ logger.info("Starting legacy checkpoint migration from [{}] to [{}] for stream [{}]",
+ sourceTableName, targetTableName, streamName);
+
+ Map exclusiveStartKey = null;
+ int scanned = 0;
+ int migrated = 0;
+ int skippedMissingAttr = 0;
+ int skippedNonNumeric = 0;
+ do {
+ final ScanRequest scanRequest = ScanRequest.builder()
+ .tableName(sourceTableName)
+ .exclusiveStartKey(exclusiveStartKey)
+ .build();
+ final ScanResponse scanResponse = dynamoDbClient.scan(scanRequest);
+
+ for (final Map item : scanResponse.items()) {
+ scanned++;
+ final AttributeValue leaseKeyAttr = item.get(LEGACY_LEASE_KEY_ATTRIBUTE);
+ final AttributeValue checkpointAttr = item.get(LEGACY_CHECKPOINT_ATTRIBUTE);
+ if (leaseKeyAttr == null || checkpointAttr == null) {
+ skippedMissingAttr++;
+ logger.warn("Skipping legacy item missing leaseKey or checkpoint: keys={}", item.keySet());
+ continue;
+ }
+
+ final String shardId = extractShardId(leaseKeyAttr.s());
+ if (shardId == null || shardId.isEmpty()) {
+ skippedMissingAttr++;
+ continue;
+ }
+
+ final String checkpoint = checkpointAttr.s();
+ if (!isValidSequenceNumber(checkpoint)) {
+ skippedNonNumeric++;
+ logger.warn("Skipping non-numeric legacy checkpoint [{}] for shard {}", checkpoint, shardId);
+ continue;
+ }
+
+ final long now = Instant.now().toEpochMilli();
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(targetTableName)
+ .key(Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(shardId).build()))
+ .updateExpression("SET sequenceNumber = :seq, lastUpdateTimestamp = :ts")
+ .expressionAttributeValues(Map.of(
+ ":seq", AttributeValue.builder().s(checkpoint).build(),
+ ":ts", AttributeValue.builder().n(String.valueOf(now)).build()))
+ .build();
+ dynamoDbClient.updateItem(request);
+ migrated++;
+ }
+
+ exclusiveStartKey = scanResponse.lastEvaluatedKey();
+ } while (exclusiveStartKey != null && !exclusiveStartKey.isEmpty());
+
+ logger.info("Legacy checkpoint migration complete for stream [{}]: scanned={}, migrated={}, skippedNonNumeric={}, skippedMissingAttr={}",
+ streamName, scanned, migrated, skippedNonNumeric, skippedMissingAttr);
+ }
+
+ private static String extractShardId(final String legacyLeaseKey) {
+ if (legacyLeaseKey == null || legacyLeaseKey.isEmpty()) {
+ return null;
+ }
+ final int separatorIndex = legacyLeaseKey.lastIndexOf(':');
+ if (separatorIndex >= 0 && separatorIndex + 1 < legacyLeaseKey.length()) {
+ return legacyLeaseKey.substring(separatorIndex + 1);
+ }
+ return legacyLeaseKey;
+ }
+
+ private static boolean isValidSequenceNumber(final String value) {
+ if (value == null || value.isEmpty()) {
+ return false;
+ }
+ for (int idx = 0; idx < value.length(); idx++) {
+ if (!Character.isDigit(value.charAt(idx))) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private void markMigrationComplete(final String targetTableName) {
+ final long now = Instant.now().toEpochMilli();
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(targetTableName)
+ .key(Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID).build()))
+ .updateExpression("SET migrationStatus = :status, migrationCompletedAt = :doneAt, lastUpdateTimestamp = :ts REMOVE migrationOwner")
+ .expressionAttributeValues(Map.of(
+ ":status", AttributeValue.builder().s(MIGRATION_STATUS_COMPLETE).build(),
+ ":doneAt", AttributeValue.builder().n(String.valueOf(now)).build(),
+ ":ts", AttributeValue.builder().n(String.valueOf(now)).build()))
+ .build();
+ dynamoDbClient.updateItem(request);
+ }
+
+ private void clearMigrationLock(final String targetTableName) {
+ final UpdateItemRequest request = UpdateItemRequest.builder()
+ .tableName(targetTableName)
+ .key(Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID).build()))
+ .updateExpression("REMOVE migrationStatus, migrationOwner, migrationStartedAt")
+ .build();
+ dynamoDbClient.updateItem(request);
+ }
+
+ private boolean isMigrationComplete(final String targetTableName) {
+ final GetItemResponse response = dynamoDbClient.getItem(GetItemRequest.builder()
+ .tableName(targetTableName)
+ .key(Map.of(
+ "streamName", AttributeValue.builder().s(streamName).build(),
+ "shardId", AttributeValue.builder().s(CheckpointTableUtils.MIGRATION_MARKER_SHARD_ID).build()))
+ .build());
+ if (!response.hasItem()) {
+ return false;
+ }
+ final AttributeValue status = response.item().get(MIGRATION_STATUS_ATTRIBUTE);
+ return status != null && MIGRATION_STATUS_COMPLETE.equals(status.s());
+ }
+
+ private void waitForMigrationComplete(final String targetTableName) {
+ for (int attempt = 0; attempt < MIGRATION_WAIT_MAX_ATTEMPTS; attempt++) {
+ if (isMigrationComplete(targetTableName)) {
+ logger.info("Observed completed checkpoint migration for stream [{}]", streamName);
+ return;
+ }
+
+ try {
+ Thread.sleep(MIGRATION_WAIT_MILLIS);
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new ProcessException("Interrupted while waiting for checkpoint migration to complete", e);
+ }
+ }
+
+ logger.warn("Timed out waiting for checkpoint migration to complete for stream [{}]; will check for stale lock",
+ streamName);
+ }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/MemoryBoundRecordBuffer.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/MemoryBoundRecordBuffer.java
deleted file mode 100644
index c4d1522a350d..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/MemoryBoundRecordBuffer.java
+++ /dev/null
@@ -1,725 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis;
-
-import jakarta.annotation.Nullable;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processors.aws.kinesis.RecordBuffer.ShardBufferId;
-import org.apache.nifi.processors.aws.kinesis.RecordBuffer.ShardBufferLease;
-import software.amazon.kinesis.exceptions.InvalidStateException;
-import software.amazon.kinesis.exceptions.KinesisClientLibDependencyException;
-import software.amazon.kinesis.exceptions.ShutdownException;
-import software.amazon.kinesis.exceptions.ThrottlingException;
-import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-import java.nio.ByteBuffer;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Optional;
-import java.util.Queue;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Collections.emptyList;
-
-/**
- * A record buffer which limits the maximum memory usage across all shard buffers.
- * If the memory limit is reached, adding new records will block until enough memory is freed.
- */
-final class MemoryBoundRecordBuffer implements RecordBuffer.ForKinesisClientLibrary, RecordBuffer.ForProcessor {
-
- private final ComponentLog logger;
-
- private final long checkpointIntervalMillis;
- private final BlockingMemoryTracker memoryTracker;
-
- private final AtomicLong bufferIdCounter = new AtomicLong(0);
-
- /**
- * All shard buffers stored by their ids.
- *
- * When a buffer is invalidated, it is removed from this map, but its id may still be present in the buffersToLease queue.
- * Since the buffer can be invalidated concurrently, it's possible for some buffer operations to be called
- * after the buffer was removed from this map. In that case the operations should take no effect.
- */
- private final ConcurrentMap shardBuffers = new ConcurrentHashMap<>();
-
- /**
- * A queue with ids shard buffers available for leasing.
- *
- * Note: when a buffer is invalidated its id is NOT removed from the queue immediately.
- */
- private final Queue buffersToLease = new ConcurrentLinkedQueue<>();
-
- MemoryBoundRecordBuffer(final ComponentLog logger, final long maxMemoryBytes, final Duration checkpointInterval) {
- this.logger = logger;
- this.memoryTracker = new BlockingMemoryTracker(logger, maxMemoryBytes);
- this.checkpointIntervalMillis = checkpointInterval.toMillis();
- }
-
- @Override
- public ShardBufferId createBuffer(final String shardId) {
- final ShardBufferId id = new ShardBufferId(shardId, bufferIdCounter.getAndIncrement());
-
- logger.debug("Creating new buffer for shard {} with id {}", shardId, id);
-
- shardBuffers.put(id, new ShardBuffer(id, logger, checkpointIntervalMillis));
- buffersToLease.add(id);
- return id;
- }
-
- @Override
- public void addRecords(final ShardBufferId bufferId, final List records, final RecordProcessorCheckpointer checkpointer) {
- if (records.isEmpty()) {
- return;
- }
-
- final ShardBuffer buffer = shardBuffers.get(bufferId);
- if (buffer == null) {
- logger.debug("Buffer with id {} not found. Cannot add records with sequence and subsequence numbers: {}.{} - {}.{}",
- bufferId,
- records.getFirst().sequenceNumber(),
- records.getFirst().subSequenceNumber(),
- records.getLast().sequenceNumber(),
- records.getLast().subSequenceNumber());
- return;
- }
-
- final RecordBatch recordBatch = new RecordBatch(records, checkpointer, calculateMemoryUsage(records));
- memoryTracker.reserveMemory(recordBatch, bufferId);
- final boolean addedRecords = buffer.offer(recordBatch);
-
- if (addedRecords) {
- logger.debug("Successfully added records with sequence and subsequence numbers: {}.{} - {}.{} to buffer with id {}",
- records.getFirst().sequenceNumber(),
- records.getFirst().subSequenceNumber(),
- records.getLast().sequenceNumber(),
- records.getLast().subSequenceNumber(),
- bufferId);
- } else {
- logger.debug("Buffer with id {} was invalidated. Cannot add records with sequence and subsequence numbers: {}.{} - {}.{}",
- bufferId,
- records.getFirst().sequenceNumber(),
- records.getFirst().subSequenceNumber(),
- records.getLast().sequenceNumber(),
- records.getLast().subSequenceNumber());
- // If the buffer was invalidated, we should free memory reserved for these records.
- memoryTracker.freeMemory(List.of(recordBatch), bufferId);
- }
- }
-
- @Override
- public void checkpointEndedShard(final ShardBufferId bufferId, final RecordProcessorCheckpointer checkpointer) {
- final ShardBuffer buffer = shardBuffers.get(bufferId);
- if (buffer == null) {
- logger.debug("Buffer with id {} not found. Cannot checkpoint the ended shard", bufferId);
- return;
- }
-
- logger.debug("Finishing consumption for buffer {}. Checkpointing the ended shard", bufferId);
- buffer.checkpointEndedShard(checkpointer);
-
- logger.debug("Removing buffer with id {} after successful ended shard checkpoint", bufferId);
- shardBuffers.remove(bufferId);
- }
-
- @Override
- public void shutdownShardConsumption(final ShardBufferId bufferId, final RecordProcessorCheckpointer checkpointer) {
- final ShardBuffer buffer = shardBuffers.remove(bufferId);
-
- if (buffer == null) {
- logger.debug("Buffer with id {} not found. Cannot shutdown shard consumption", bufferId);
- } else {
- logger.debug("Shutting down the buffer {}. Checkpointing last consumed record", bufferId);
- final Collection invalidatedBatches = buffer.shutdownBuffer(checkpointer);
- memoryTracker.freeMemory(invalidatedBatches, bufferId);
- }
- }
-
- @Override
- public void consumerLeaseLost(final ShardBufferId bufferId) {
- final ShardBuffer buffer = shardBuffers.remove(bufferId);
-
- if (buffer == null) {
- logger.debug("Buffer with id {} not found. Ignoring lease lost event", bufferId);
- } else {
- logger.debug("Lease lost for buffer {}: Invalidating", bufferId);
- final Collection invalidatedBatches = buffer.invalidate();
- memoryTracker.freeMemory(invalidatedBatches, bufferId);
- }
- }
-
- @Override
- public Optional acquireBufferLease() {
- final Set seenBuffers = new HashSet<>();
-
- while (true) {
- final ShardBufferId bufferId = buffersToLease.poll();
- if (bufferId == null) {
- // The queue is empty or all buffers were seen already. Nothing to consume.
- return Optional.empty();
- }
-
- if (seenBuffers.contains(bufferId)) {
- // If the same buffer is seen again, there is a high chance we iterated through most of the buffers and didn't find any that isn't empty.
- // To avoid burning CPU we return empty here, even if some buffer received records in the meantime. It will be picked up in the next iteration.
- buffersToLease.add(bufferId);
- return Optional.empty();
- }
-
- final ShardBuffer buffer = shardBuffers.get(bufferId);
-
- if (buffer == null) {
- // By the time the bufferId is polled, it might have been invalidated. No need to return it to the queue.
- logger.debug("Buffer with id {} was removed while polling for lease. Continuing to poll", bufferId);
- } else if (buffer.isEmpty()) {
- seenBuffers.add(bufferId);
- buffersToLease.add(bufferId);
- logger.debug("Buffer with id {} is empty. Continuing to poll", bufferId);
- } else {
- logger.debug("Acquired lease for buffer {}", bufferId);
- return Optional.of(new Lease(bufferId));
- }
- }
- }
-
- @Override
- public List consumeRecords(final Lease lease) {
- if (lease.isReturnedToPool()) {
- logger.warn("Attempting to consume records from a buffer that was already returned to the pool. Ignoring");
- return emptyList();
- }
-
- final ShardBufferId bufferId = lease.bufferId();
-
- final ShardBuffer buffer = shardBuffers.get(bufferId);
- if (buffer == null) {
- logger.debug("Buffer with id {} not found. Cannot consume records", bufferId);
- return emptyList();
- }
-
- return buffer.consumeRecords();
- }
-
- @Override
- public void commitConsumedRecords(final Lease lease) {
- if (lease.isReturnedToPool()) {
- logger.warn("Attempting to commit records from a buffer that was already returned to the pool. Ignoring");
- return;
- }
-
- final ShardBufferId bufferId = lease.bufferId();
-
- final ShardBuffer buffer = shardBuffers.get(bufferId);
- if (buffer == null) {
- logger.debug("Buffer with id {} not found. Cannot commit consumed records", bufferId);
- return;
- }
-
- logger.debug("Committing consumed records for buffer {}", bufferId);
- final List consumedBatches = buffer.commitConsumedRecords();
- memoryTracker.freeMemory(consumedBatches, bufferId);
- }
-
- @Override
- public void rollbackConsumedRecords(final Lease lease) {
- if (lease.isReturnedToPool()) {
- logger.warn("Attempting to rollback records from a buffer that was already returned to the pool. Ignoring");
- return;
- }
-
- final ShardBufferId bufferId = lease.bufferId();
- final ShardBuffer buffer = shardBuffers.get(bufferId);
-
- if (buffer != null) {
- logger.debug("Rolling back consumed records for buffer {}", bufferId);
- buffer.rollbackConsumedRecords();
- }
- }
-
- @Override
- public void returnBufferLease(final Lease lease) {
- if (lease.returnToPool()) {
- final ShardBufferId bufferId = lease.bufferId();
- buffersToLease.add(bufferId);
- logger.debug("The buffer {} is available for lease again", bufferId);
- } else {
- logger.warn("Attempting to return a buffer that was already returned to the pool. Ignoring");
- }
- }
-
- static final class Lease implements ShardBufferLease {
-
- private final ShardBufferId bufferId;
- private final AtomicBoolean returnedToPool = new AtomicBoolean(false);
-
- private Lease(final ShardBufferId bufferId) {
- this.bufferId = bufferId;
- }
-
- @Override
- public String shardId() {
- return bufferId.shardId();
- }
-
- private ShardBufferId bufferId() {
- return bufferId;
- }
-
- private boolean isReturnedToPool() {
- return returnedToPool.get();
- }
-
- /**
- * Marks the lease as returned to the pool.
- * @return true if the lease was not returned before, false otherwise.
- */
- private boolean returnToPool() {
- final boolean wasReturned = returnedToPool.getAndSet(true);
- return !wasReturned;
- }
- }
-
- /**
- * A memory tracker which blocks a thread when the memory usage exceeds the allowed maximum.
- *
- * In order to make progress, the memory consumption may exceed the limit, but any new records will not be accepted.
- * This is done to support the case when a single record batch is larger than the allowed memory limit.
- */
- private static class BlockingMemoryTracker {
-
- private static final long AWAIT_MILLIS = 100;
-
- private final ComponentLog logger;
-
- private final long maxMemoryBytes;
-
- private final AtomicLong consumedMemoryBytes = new AtomicLong(0);
- /**
- * Whenever memory is freed a latch opens. Then replaced with a new one.
- */
- private final AtomicReference memoryAvailableLatch = new AtomicReference<>(new CountDownLatch(1));
-
- BlockingMemoryTracker(final ComponentLog logger, final long maxMemoryBytes) {
- this.logger = logger;
- this.maxMemoryBytes = maxMemoryBytes;
- }
-
- void reserveMemory(final RecordBatch recordBatch, final ShardBufferId bufferId) {
- final long consumedBytes = recordBatch.batchSizeBytes();
-
- if (consumedBytes == 0) {
- logger.debug("The batch for buffer {} is empty. No need to reserve memory", bufferId);
- return;
- }
-
- while (true) {
- final long currentlyConsumedBytes = consumedMemoryBytes.get();
-
- if (currentlyConsumedBytes >= maxMemoryBytes) {
- // Not enough memory available, need to wait.
- try {
- memoryAvailableLatch.get().await(AWAIT_MILLIS, TimeUnit.MILLISECONDS);
- } catch (final InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new IllegalStateException("Thread interrupted while waiting for available memory in RecordBuffer", e);
- }
- } else {
- final long newConsumedBytes = currentlyConsumedBytes + consumedBytes;
- if (consumedMemoryBytes.compareAndSet(currentlyConsumedBytes, newConsumedBytes)) {
- logger.debug("Reserved {} bytes for {} records for buffer {}. Total consumed memory: {} bytes",
- consumedBytes, recordBatch.size(), bufferId, newConsumedBytes);
- break;
- }
- // If we're here, the compare and set operation failed, as another thread has modified the gauge in meantime.
- // Retrying the operation.
- }
- }
- }
-
- void freeMemory(final Collection consumedBatches, final ShardBufferId bufferId) {
- if (consumedBatches.isEmpty()) {
- logger.debug("No batches were consumed from buffer {}. No need to free memory", bufferId);
- return;
- }
-
- long freedBytes = 0;
- for (final RecordBatch batch : consumedBatches) {
- freedBytes += batch.batchSizeBytes();
- }
-
- while (true) {
- final long currentlyConsumedBytes = consumedMemoryBytes.get();
- if (currentlyConsumedBytes < freedBytes) {
- throw new IllegalStateException("Attempting to free more memory than currently used");
- }
-
- final long newTotal = currentlyConsumedBytes - freedBytes;
- if (consumedMemoryBytes.compareAndSet(currentlyConsumedBytes, newTotal)) {
- logger.debug("Freed {} bytes for {} batches from buffer {}. Total consumed memory: {} bytes",
- freedBytes, consumedBatches.size(), bufferId, newTotal);
-
- final CountDownLatch oldLatch = memoryAvailableLatch.getAndSet(new CountDownLatch(1));
- oldLatch.countDown(); // Release any waiting threads for free memory.
- break;
- }
- // If we're here, the compare and set operation failed, as another thread has modified the gauge in meantime.
- // Retrying the operation.
- }
- }
- }
-
- private record RecordBatch(List records,
- RecordProcessorCheckpointer checkpointer,
- long batchSizeBytes) {
- int size() {
- return records.size();
- }
- }
-
- private long calculateMemoryUsage(final Collection records) {
- long totalBytes = 0;
- for (final KinesisClientRecord record : records) {
- final ByteBuffer data = record.data();
- if (data != null) {
- totalBytes += data.capacity();
- }
- }
- return totalBytes;
- }
-
- /**
- * ShardBuffer stores all record batches for a single shard in two queues:
- * - IN_PROGRESS: record batches that have been consumed but not yet checkpointed.
- * - PENDING: record batches that have been added but not yet consumed.
- *
- * When consuming records all PENDING batches are moved to IN_PROGRESS.
- * After a successful checkpoint all IN_PROGRESS batches are cleared.
- * After a rollback all IN_PROGRESS batches are kept, allowing to retry consumption.
- *
- * Each batch preserves the original grouping of records as provided by Kinesis
- * along with their associated checkpointer, ensuring atomicity.
- */
- private static class ShardBuffer {
-
- private static final long AWAIT_MILLIS = 100;
-
- // Retry configuration.
- private static final int MAX_RETRY_ATTEMPTS = 5;
- private static final long BASE_RETRY_DELAY_MILLIS = 100;
- private static final long MAX_RETRY_DELAY_MILLIS = 10_000;
- private static final Random RANDOM = new Random();
-
- private final ShardBufferId bufferId;
- private final ComponentLog logger;
-
- private final long checkpointIntervalMillis;
- private volatile long nextCheckpointTimeMillis;
-
- /**
- * A last record checkpointer and sequence number that was ignored due to the checkpoint interval.
- * If null, the last checkpoint was successful or no checkpoint was attempted yet.
- */
- private volatile @Nullable LastIgnoredCheckpoint lastIgnoredCheckpoint;
-
- /**
- * Queues for managing record batches with their checkpointers in different states.
- */
- private final Queue inProgressBatches = new ConcurrentLinkedQueue<>();
- private final Queue pendingBatches = new ConcurrentLinkedQueue<>();
- /**
- * Counter for tracking the number of batches in the buffer. Can be larger than the number of batches in the queues.
- */
- private final AtomicInteger batchesCount = new AtomicInteger(0);
-
- /**
- * A countdown latch that is used to signal when the buffer becomes empty. Used when ShardBuffer should be closed.
- */
- private volatile @Nullable CountDownLatch emptyBufferLatch = null;
- private final AtomicBoolean invalidated = new AtomicBoolean(false);
-
- ShardBuffer(final ShardBufferId bufferId, final ComponentLog logger, final long checkpointIntervalMillis) {
- this.bufferId = bufferId;
- this.logger = logger;
- this.checkpointIntervalMillis = checkpointIntervalMillis;
- this.nextCheckpointTimeMillis = System.currentTimeMillis() + checkpointIntervalMillis;
- }
-
- /**
- * @param recordBatch record batch with records to add.
- * @return true if the records were added successfully, false if a buffer was invalidated.
- */
- boolean offer(final RecordBatch recordBatch) {
- if (invalidated.get()) {
- return false;
- }
-
- // Batches count must be always equal to or larger than the number of batches in the queues.
- // Thus, the ordering of the operations.
- batchesCount.incrementAndGet();
- pendingBatches.offer(recordBatch);
-
- return true;
- }
-
- List consumeRecords() {
- if (invalidated.get()) {
- return emptyList();
- }
-
- RecordBatch pendingBatch;
- while ((pendingBatch = pendingBatches.poll()) != null) {
- inProgressBatches.offer(pendingBatch);
- }
-
- final List recordsToConsume = new ArrayList<>();
- for (final RecordBatch batch : inProgressBatches) {
- recordsToConsume.addAll(batch.records());
- }
-
- return recordsToConsume;
- }
-
- List commitConsumedRecords() {
- if (invalidated.get()) {
- return emptyList();
- }
-
- final List checkpointedBatches = new ArrayList<>();
- RecordBatch batch;
- while ((batch = inProgressBatches.poll()) != null) {
- checkpointedBatches.add(batch);
- }
-
- if (checkpointedBatches.isEmpty()) {
- // The buffer could be invalidated in the meantime, or no records were consumed.
- return emptyList();
- }
-
- // Batches count must always be equal to or larger than the number of batches in the queues.
- // To achieve so, the count is decreased only after the queue has been emptied.
- batchesCount.addAndGet(-checkpointedBatches.size());
-
- final RecordProcessorCheckpointer lastBatchCheckpointer = checkpointedBatches.getLast().checkpointer();
- final KinesisClientRecord lastRecord = checkpointedBatches.getLast().records().getLast();
-
- if (System.currentTimeMillis() >= nextCheckpointTimeMillis) {
- checkpointSequenceNumber(lastBatchCheckpointer, lastRecord.sequenceNumber(), lastRecord.subSequenceNumber());
- nextCheckpointTimeMillis = System.currentTimeMillis() + checkpointIntervalMillis;
- lastIgnoredCheckpoint = null;
- } else {
- // Saving the checkpointer for later, in case shutdown happens before the next checkpoint.
- lastIgnoredCheckpoint = new LastIgnoredCheckpoint(lastBatchCheckpointer, lastRecord.sequenceNumber(), lastRecord.subSequenceNumber());
- }
-
- final CountDownLatch localEmptyBufferLatch = this.emptyBufferLatch;
- if (localEmptyBufferLatch != null && isEmpty()) {
- // If the latch is not null, it means we are waiting for the buffer to become empty.
- localEmptyBufferLatch.countDown();
- }
-
- return checkpointedBatches;
- }
-
- void rollbackConsumedRecords() {
- if (invalidated.get()) {
- return;
- }
-
- for (final RecordBatch recordBatch : inProgressBatches) {
- for (final KinesisClientRecord record : recordBatch.records()) {
- record.data().rewind();
- }
- }
- }
-
- void checkpointEndedShard(final RecordProcessorCheckpointer checkpointer) {
- while (true) {
- if (invalidated.get()) {
- return;
- }
-
- // Setting the latch first, so that if the buffer is being emptied concurrently in commitConsumedRecords
- // the latch is guaranteed to be visible to the commitConsumedRecords, and, therefore, opened.
- // This will eliminate unnecessary downtimes when waiting for a latch to be opened.
- final CountDownLatch localEmptyBufferLatch = new CountDownLatch(1);
- this.emptyBufferLatch = localEmptyBufferLatch;
-
- if (batchesCount.get() == 0) {
- // Buffer is empty, perform final checkpoint.
- checkpointLastReceivedRecord(checkpointer);
- return;
- }
-
- // Wait for the records to be consumed first.
- try {
- localEmptyBufferLatch.await(AWAIT_MILLIS, TimeUnit.MILLISECONDS);
- } catch (final InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new IllegalStateException("Thread interrupted while waiting for records to be consumed", e);
- }
- }
- }
-
- Collection shutdownBuffer(final RecordProcessorCheckpointer checkpointer) {
- if (invalidated.getAndSet(true)) {
- return emptyList();
- }
-
- if (batchesCount.get() == 0) {
- checkpointLastReceivedRecord(checkpointer);
- return emptyList();
- }
-
- // If there are still records in the buffer, checkpointing with the latest provided checkpointer is not safe.
- // But, if the records were committed without checkpointing in the past, we can checkpoint them now.
- final LastIgnoredCheckpoint ignoredCheckpoint = this.lastIgnoredCheckpoint;
- if (ignoredCheckpoint != null) {
- checkpointSequenceNumber(
- ignoredCheckpoint.checkpointer(),
- ignoredCheckpoint.sequenceNumber(),
- ignoredCheckpoint.subSequenceNumber()
- );
- }
-
- return drainInvalidatedBatches();
- }
-
- Collection invalidate() {
- if (invalidated.getAndSet(true)) {
- return emptyList();
- }
-
- return drainInvalidatedBatches();
- }
-
- private Collection drainInvalidatedBatches() {
- if (!invalidated.get()) {
- throw new IllegalStateException("Unable to drain invalidated batches for valid shard buffer: " + bufferId);
- }
-
- final List batches = new ArrayList<>();
- RecordBatch batch;
- // If both consumeRecords and drainInvalidatedBatches are called concurrently, invalidation must always consume all batches.
- // Since consumeRecords moves batches from pending to in_progress, during invalidation pending batches should be drained first.
- while ((batch = pendingBatches.poll()) != null) {
- batches.add(batch);
- }
- while ((batch = inProgressBatches.poll()) != null) {
- batches.add(batch);
- }
-
- // No need to adjust batchesCount after invalidation.
-
- return batches;
- }
-
- /**
- * Checks if the buffer has any records. Can produce false negatives.
- *
- * @return whether there are any records in the buffer.
- */
- boolean isEmpty() {
- return invalidated.get() || batchesCount.get() == 0;
- }
-
- private void checkpointLastReceivedRecord(final RecordProcessorCheckpointer checkpointer) {
- logger.debug("Performing checkpoint for buffer with id {}. Checkpointing the last received record", bufferId);
-
- checkpointSafely(checkpointer::checkpoint);
- }
-
- private void checkpointSequenceNumber(final RecordProcessorCheckpointer checkpointer, final String sequenceNumber, final long subSequenceNumber) {
- logger.debug("Performing checkpoint for buffer with id {}. Sequence number: [{}], sub sequence number: [{}]",
- bufferId, sequenceNumber, subSequenceNumber);
-
- checkpointSafely(() -> checkpointer.checkpoint(sequenceNumber, subSequenceNumber));
- }
-
- /**
- * Performs checkpointing using exponential backoff and jitter, if needed.
- *
- * @param checkpointAction the action which performs the checkpointing.
- */
- private void checkpointSafely(final CheckpointAction checkpointAction) {
- for (int attempt = 1; attempt <= MAX_RETRY_ATTEMPTS; attempt++) {
- try {
- checkpointAction.doCheckpoint();
- if (attempt > 1) {
- logger.debug("Checkpoint succeeded on attempt {}", attempt);
- }
- return;
- } catch (final ThrottlingException | InvalidStateException | KinesisClientLibDependencyException e) {
- if (attempt == MAX_RETRY_ATTEMPTS) {
- logger.error("Failed to checkpoint after {} attempts, giving up", MAX_RETRY_ATTEMPTS, e);
- return;
- }
-
- final long delayMillis = calculateRetryDelay(attempt);
-
- logger.debug("Checkpoint failed on attempt {} with {}, retrying in {} ms",
- attempt, e.getMessage(), delayMillis);
-
- try {
- Thread.sleep(delayMillis);
- } catch (final InterruptedException ie) {
- Thread.currentThread().interrupt();
- logger.warn("Thread interrupted while waiting to retry checkpoint. Exiting retry loop", ie);
- return;
- }
- } catch (final ShutdownException e) {
- logger.warn("Failed to checkpoint records due to shutdown. Ignoring checkpoint", e);
- return;
- } catch (final RuntimeException e) {
- logger.warn("Failed to checkpoint records due to an error. Ignoring checkpoint", e);
- return;
- }
- }
- }
-
- private long calculateRetryDelay(final int attempt) {
- final long desiredBaseDelayMillis = BASE_RETRY_DELAY_MILLIS * (1L << (attempt - 1));
- final long baseDelayMillis = Math.min(desiredBaseDelayMillis, MAX_RETRY_DELAY_MILLIS);
- final long jitterMillis = RANDOM.nextLong(baseDelayMillis / 4); // Up to 25% jitter.
- return baseDelayMillis + jitterMillis;
- }
-
- private interface CheckpointAction {
-
- /**
- * Throws the same set of exceptions as {@link RecordProcessorCheckpointer#checkpoint()} and {@link RecordProcessorCheckpointer#checkpoint(String, long)}.
- */
- void doCheckpoint() throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException;
- }
-
- private record LastIgnoredCheckpoint(RecordProcessorCheckpointer checkpointer, String sequenceNumber, long subSequenceNumber) {
- }
- }
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/PollingKinesisClient.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/PollingKinesisClient.java
new file mode 100644
index 000000000000..e21dd3c7ccb2
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/PollingKinesisClient.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import org.apache.nifi.logging.ComponentLog;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
+import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
+import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
+import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
+import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
+import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.ShardIteratorType;
+
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Shared-throughput Kinesis consumer that runs a continuous background fetch loop per shard.
+ * Each owned shard gets its own virtual thread that repeatedly calls GetRecords and enqueues
+ * results for the processor, mirroring the producer-consumer architecture of the KCL Scheduler.
+ * This keeps data flowing between onTrigger invocations rather than fetching on-demand.
+ *
+ * Concurrency is bounded by a semaphore with {@value #MAX_CONCURRENT_FETCHES} permits so
+ * that at most that many GetRecords HTTP calls are in flight at any moment, preventing
+ * connection-pool exhaustion. A second fair semaphore with {@value #MAX_QUEUED_RESULTS}
+ * permits ensures that fetch threads block when the result queue is full, with FIFO ordering
+ * guaranteeing that all shard threads get equal opportunity to enqueue results.
+ */
+final class PollingKinesisClient extends KinesisConsumerClient {
+
+ private static final long DEFAULT_EMPTY_SHARD_BACKOFF_NANOS = TimeUnit.MILLISECONDS.toNanos(500);
+ private static final long DEFAULT_ERROR_BACKOFF_NANOS = TimeUnit.SECONDS.toNanos(2);
+ static final int MAX_QUEUED_RESULTS = 200;
+ static final int MAX_CONCURRENT_FETCHES = 25;
+
+ private final ExecutorService fetchExecutor = Executors.newVirtualThreadPerTaskExecutor();
+ private final Map pollingShardStates = new ConcurrentHashMap<>();
+ private final Semaphore fetchPermits = new Semaphore(MAX_CONCURRENT_FETCHES, true);
+ private final Semaphore queuePermits = new Semaphore(MAX_QUEUED_RESULTS, true);
+ private final long emptyShardBackoffNanos;
+ private final long errorBackoffNanos;
+ PollingKinesisClient(final KinesisClient kinesisClient, final ComponentLog logger) {
+ this(kinesisClient, logger, DEFAULT_EMPTY_SHARD_BACKOFF_NANOS, DEFAULT_ERROR_BACKOFF_NANOS);
+ }
+
+ PollingKinesisClient(final KinesisClient kinesisClient, final ComponentLog logger,
+ final long emptyShardBackoffNanos, final long errorBackoffNanos) {
+ super(kinesisClient, logger);
+ this.emptyShardBackoffNanos = emptyShardBackoffNanos;
+ this.errorBackoffNanos = errorBackoffNanos;
+ }
+
+ @Override
+ void startFetches(final List shards, final String streamName, final int batchSize,
+ final String initialStreamPosition, final KinesisShardManager shardManager) {
+ if (fetchExecutor.isShutdown()) {
+ return;
+ }
+
+ for (final Shard shard : shards) {
+ final String shardId = shard.shardId();
+ final PollingShardState existing = pollingShardStates.get(shardId);
+ if (existing == null) {
+ final PollingShardState state = new PollingShardState();
+ if (pollingShardStates.putIfAbsent(shardId, state) == null && state.tryStartLoop()) {
+ launchFetchLoop(state, shardId, streamName, batchSize, initialStreamPosition, shardManager);
+ }
+ } else if (!existing.isExhausted() && !existing.isStopped() && !existing.isLoopRunning()
+ && existing.tryStartLoop()) {
+ logger.warn("Restarting dead fetch loop for stream [{}] shard [{}]", streamName, shardId);
+ launchFetchLoop(existing, shardId, streamName, batchSize, initialStreamPosition, shardManager);
+ }
+ }
+ }
+
+ @Override
+ boolean hasPendingFetches() {
+ if (hasQueuedResults()) {
+ return true;
+ }
+ for (final PollingShardState state : pollingShardStates.values()) {
+ if (!state.isExhausted() && !state.isStopped()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ void acknowledgeResults(final List results) {
+ }
+
+ @Override
+ void rollbackResults(final List results) {
+ for (final ShardFetchResult result : results) {
+ final PollingShardState state = pollingShardStates.get(result.shardId());
+ if (state != null) {
+ resetAndDrainShard(result.shardId(), state);
+ }
+ }
+ }
+
+ @Override
+ void removeUnownedShards(final Set ownedShards) {
+ pollingShardStates.entrySet().removeIf(entry -> {
+ if (!ownedShards.contains(entry.getKey())) {
+ entry.getValue().stop();
+ return true;
+ }
+ return false;
+ });
+ }
+
+ @Override
+ void logDiagnostics(final int ownedCount, final int cachedShardCount) {
+ if (!shouldLogDiagnostics()) {
+ return;
+ }
+
+ int active = 0;
+ int exhausted = 0;
+ int stopped = 0;
+ int dead = 0;
+ for (final PollingShardState state : pollingShardStates.values()) {
+ if (state.isExhausted()) {
+ exhausted++;
+ } else if (state.isStopped()) {
+ stopped++;
+ } else if (!state.isLoopRunning()) {
+ dead++;
+ } else {
+ active++;
+ }
+ }
+
+ logger.debug("Kinesis polling diagnostics: discoveredShards={}, ownedShards={}, queueDepth={}, "
+ + "fetchLoops={}, active={}, exhausted={}, stopped={}, dead={}, concurrentFetches={}",
+ cachedShardCount, ownedCount, totalQueuedResults(), pollingShardStates.size(),
+ active, exhausted, stopped, dead, MAX_CONCURRENT_FETCHES - fetchPermits.availablePermits());
+ }
+
+ @Override
+ protected void onResultPolled() {
+ queuePermits.release();
+ }
+
+ @Override
+ void close() {
+ for (final PollingShardState state : pollingShardStates.values()) {
+ state.stop();
+ }
+ pollingShardStates.clear();
+ fetchExecutor.shutdownNow();
+ super.close();
+ }
+
+ private void launchFetchLoop(final PollingShardState state, final String shardId,
+ final String streamName, final int batchSize, final String initialStreamPosition,
+ final KinesisShardManager shardManager) {
+ final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
+ try {
+ fetchExecutor.submit(() -> {
+ Thread.currentThread().setContextClassLoader(contextClassLoader);
+ try {
+ runFetchLoop(state, shardId, streamName, batchSize, initialStreamPosition, shardManager);
+ } catch (final Throwable t) {
+ if (!state.isStopped()) {
+ logger.error("Fetch loop for shard {} terminated unexpectedly", shardId, t);
+ }
+ } finally {
+ state.markLoopStopped();
+ }
+ });
+ } catch (final RejectedExecutionException e) {
+ state.markLoopStopped();
+ logger.debug("Executor shut down; cannot start fetch loop for stream [{}] shard [{}]", streamName, shardId);
+ }
+ }
+
+ private void runFetchLoop(final PollingShardState state, final String shardId,
+ final String streamName, final int batchSize, final String initialStreamPosition,
+ final KinesisShardManager shardManager) {
+
+ state.setIterator(getShardIterator(state, streamName, shardId, initialStreamPosition, shardManager));
+
+ while (!Thread.currentThread().isInterrupted() && !state.isStopped()) {
+ try {
+ if (state.isExhausted()) {
+ return;
+ }
+
+ if (state.isResetRequested()) {
+ state.clearReset();
+ final int drained = drainShardQueue(shardId);
+ if (drained > 0) {
+ queuePermits.release(drained);
+ }
+ state.setIterator(getShardIterator(state, streamName, shardId, initialStreamPosition, shardManager));
+ }
+
+ if (state.getIterator() == null) {
+ state.setIterator(getShardIterator(state, streamName, shardId, initialStreamPosition, shardManager));
+ if (state.getIterator() == null) {
+ sleepNanos(errorBackoffNanos);
+ continue;
+ }
+ }
+
+ try {
+ queuePermits.acquire();
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return;
+ }
+
+ boolean queuePermitConsumed = false;
+ try {
+ try {
+ fetchPermits.acquire();
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return;
+ }
+
+ final GetRecordsResponse response;
+ try {
+ response = fetchRecords(shardId, state, batchSize);
+ } finally {
+ fetchPermits.release();
+ }
+ if (response == null) {
+ continue;
+ }
+
+ final List records = response.records();
+ if (!records.isEmpty()) {
+ final long millisBehind = response.millisBehindLatest() != null ? response.millisBehindLatest() : -1;
+ queuePermitConsumed = enqueueIfActive(shardId, state, createFetchResult(shardId, records, millisBehind));
+ }
+
+ state.setIterator(response.nextShardIterator());
+ if (state.getIterator() == null) {
+ state.markExhausted();
+ return;
+ }
+
+ if (records.isEmpty()) {
+ sleepNanos(emptyShardBackoffNanos);
+ }
+ } finally {
+ if (!queuePermitConsumed) {
+ queuePermits.release();
+ }
+ }
+ } catch (final Exception e) {
+ if (!state.isStopped()) {
+ logger.warn("Unexpected error in fetch loop for shard [{}]; will retry", shardId, e);
+ state.setIterator(null);
+ sleepNanos(errorBackoffNanos);
+ }
+ }
+ }
+ }
+
+ private GetRecordsResponse fetchRecords(final String shardId, final PollingShardState state, final int batchSize) {
+ final GetRecordsRequest request = GetRecordsRequest.builder()
+ .shardIterator(state.getIterator())
+ .limit(batchSize)
+ .build();
+
+ try {
+ return kinesisClient.getRecords(request);
+ } catch (final ProvisionedThroughputExceededException | LimitExceededException e) {
+ logger.debug("GetRecords throttled for shard {}; will retry after backoff", shardId);
+ sleepNanos(errorBackoffNanos);
+ return null;
+ } catch (final ExpiredIteratorException e) {
+ logger.info("Shard iterator expired for shard {}; will re-acquire", shardId);
+ state.requestReset();
+ sleepNanos(errorBackoffNanos);
+ return null;
+ } catch (final SdkClientException e) {
+ if (!state.isStopped()) {
+ logger.warn("GetRecords failed for shard {}; will retry with existing iterator", shardId, e);
+ sleepNanos(errorBackoffNanos);
+ }
+ return null;
+ } catch (final Exception e) {
+ if (!state.isStopped()) {
+ logger.error("GetRecords failed for shard {}", shardId, e);
+ state.requestReset();
+ sleepNanos(errorBackoffNanos);
+ }
+ return null;
+ }
+ }
+
+ private boolean enqueueIfActive(final String shardId, final PollingShardState state, final ShardFetchResult result) {
+ synchronized (getShardLock(shardId)) {
+ if (state.isResetRequested()) {
+ return false;
+ }
+ enqueueResult(result);
+ return true;
+ }
+ }
+
+ private void resetAndDrainShard(final String shardId, final PollingShardState state) {
+ synchronized (getShardLock(shardId)) {
+ state.requestReset();
+ final int drained = drainShardQueue(shardId);
+ if (drained > 0) {
+ queuePermits.release(drained);
+ }
+ }
+ }
+
+ private static void sleepNanos(final long nanos) {
+ try {
+ TimeUnit.NANOSECONDS.sleep(nanos);
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ private String getShardIterator(final PollingShardState state, final String streamName,
+ final String shardId, final String initialStreamPosition, final KinesisShardManager shardManager) {
+ try {
+ fetchPermits.acquire();
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return null;
+ }
+
+ try {
+ final String lastSequenceNumber;
+ try {
+ lastSequenceNumber = shardManager.readCheckpoint(shardId);
+ } catch (final Exception e) {
+ if (!state.isStopped()) {
+ logger.warn("Failed to read checkpoint for shard {}; will retry", shardId, e);
+ }
+ return null;
+ }
+
+ final ShardIteratorType iteratorType;
+ final String startingSequenceNumber;
+ final Instant timestamp;
+ if (lastSequenceNumber != null) {
+ iteratorType = ShardIteratorType.AFTER_SEQUENCE_NUMBER;
+ startingSequenceNumber = lastSequenceNumber;
+ timestamp = null;
+ } else {
+ iteratorType = ShardIteratorType.fromValue(initialStreamPosition);
+ startingSequenceNumber = null;
+ timestamp = (iteratorType == ShardIteratorType.AT_TIMESTAMP) ? getTimestampForInitialPosition() : null;
+ }
+
+ logger.debug("Getting shard iterator for shard {} with type={}, startingSeq={}, timestamp={}",
+ shardId, iteratorType, startingSequenceNumber, timestamp);
+
+ final GetShardIteratorRequest.Builder iteratorRequestBuilder = GetShardIteratorRequest.builder()
+ .streamName(streamName)
+ .shardId(shardId)
+ .shardIteratorType(iteratorType);
+
+ if (startingSequenceNumber != null) {
+ iteratorRequestBuilder.startingSequenceNumber(startingSequenceNumber);
+ }
+ if (timestamp != null) {
+ iteratorRequestBuilder.timestamp(timestamp);
+ }
+
+ return kinesisClient.getShardIterator(iteratorRequestBuilder.build()).shardIterator();
+ } catch (final Exception e) {
+ if (!state.isStopped()) {
+ logger.error("Failed to get shard iterator for shard {} in stream {}", shardId, streamName, e);
+ }
+ return null;
+ } finally {
+ fetchPermits.release();
+ }
+ }
+
+ static final class PollingShardState {
+ private volatile String currentIterator;
+ private volatile boolean shardExhausted;
+ private volatile boolean stopped;
+ private volatile boolean resetRequested;
+ private final AtomicBoolean loopRunning = new AtomicBoolean();
+
+ String getIterator() {
+ return currentIterator;
+ }
+
+ void setIterator(final String iterator) {
+ currentIterator = iterator;
+ }
+
+ boolean isExhausted() {
+ return shardExhausted;
+ }
+
+ void markExhausted() {
+ shardExhausted = true;
+ }
+
+ boolean isStopped() {
+ return stopped;
+ }
+
+ void stop() {
+ stopped = true;
+ }
+
+ boolean isResetRequested() {
+ return resetRequested;
+ }
+
+ void requestReset() {
+ resetRequested = true;
+ }
+
+ void clearReset() {
+ resetRequested = false;
+ }
+
+ boolean tryStartLoop() {
+ return loopRunning.compareAndSet(false, true);
+ }
+
+ void markLoopStopped() {
+ loopRunning.set(false);
+ }
+
+ boolean isLoopRunning() {
+ return loopRunning.get();
+ }
+ }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ProducerLibraryDeaggregator.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ProducerLibraryDeaggregator.java
new file mode 100644
index 000000000000..db4b580a4798
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ProducerLibraryDeaggregator.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.WireFormat;
+import software.amazon.awssdk.services.kinesis.model.Record;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Deaggregates KPL (Kinesis Producer Library) aggregated records into individual user records.
+ *
+ * KPL aggregation packs multiple user records into a single Kinesis record using a protobuf
+ * envelope with a 4-byte magic header and a 16-byte MD5 trailer. Non-aggregated records
+ * pass through unchanged as a single {@link UserRecord} with {@code subSequenceNumber=0}.
+ *
+ *
If a record has the magic header but fails MD5 verification or protobuf parsing, it falls
+ * back to passthrough to avoid data loss.
+ *
+ *
We could make direct use of the KPL's protobuf definition and generated classes,
+ * but doing so requires bringing in 20+ transitive dependencies. Since the protobuf format is
+ * simple and well-documented, we implement a minimal custom parser using the protobuf wire format
+ * instead. Additionally, we have integration tests to verify compatibility.
+ *
+ * @see KPL Aggregation Format
+ */
+final class ProducerLibraryDeaggregator {
+
+ static final byte[] KPL_MAGIC = {(byte) 0xF3, (byte) 0x89, (byte) 0x9A, (byte) 0xC2};
+ private static final int MD5_DIGEST_LENGTH = 16;
+ private static final int MIN_AGGREGATED_LENGTH = KPL_MAGIC.length + MD5_DIGEST_LENGTH + 1;
+
+ private static final int FIELD_PARTITION_KEY_TABLE = 1;
+ private static final int FIELD_EXPLICIT_HASH_KEY_TABLE = 2;
+ private static final int FIELD_RECORDS = 3;
+
+ private static final int RECORD_FIELD_PARTITION_KEY_INDEX = 1;
+ private static final int RECORD_FIELD_EXPLICIT_HASH_KEY_INDEX = 2;
+ private static final int RECORD_FIELD_DATA = 3;
+
+ private ProducerLibraryDeaggregator() {
+ }
+
+ /**
+ * Deaggregates a list of Kinesis records, expanding any KPL-aggregated records into
+ * their constituent sub-records.
+ *
+ * @param shardId the shard these records were fetched from
+ * @param records raw Kinesis records from the API
+ * @return list of deaggregated records preserving original order
+ */
+ static List deaggregate(final String shardId, final List records) {
+ final List result = new ArrayList<>();
+ for (final Record record : records) {
+ deaggregateRecord(shardId, record, result);
+ }
+ return result;
+ }
+
+ private static void deaggregateRecord(final String shardId, final Record record, final List out) {
+ final byte[] data = record.data().asByteArrayUnsafe();
+
+ if (!isAggregated(data)) {
+ out.add(passthrough(shardId, record, data));
+ return;
+ }
+
+ final int protobufOffset = KPL_MAGIC.length;
+ final int protobufLength = data.length - KPL_MAGIC.length - MD5_DIGEST_LENGTH;
+
+ if (!verifyMd5(data, protobufOffset, protobufLength)) {
+ out.add(passthrough(shardId, record, data));
+ return;
+ }
+
+ try {
+ parseAggregatedRecord(shardId, record, data, protobufOffset, protobufLength, out);
+ } catch (final Exception e) {
+ out.add(passthrough(shardId, record, data));
+ }
+ }
+
+ static boolean isAggregated(final byte[] data) {
+ if (data.length < MIN_AGGREGATED_LENGTH) {
+ return false;
+ }
+ return data[0] == KPL_MAGIC[0]
+ && data[1] == KPL_MAGIC[1]
+ && data[2] == KPL_MAGIC[2]
+ && data[3] == KPL_MAGIC[3];
+ }
+
+ private static boolean verifyMd5(final byte[] data, final int protobufOffset, final int protobufLength) {
+ final MessageDigest md5 = getMd5Digest();
+ md5.update(data, protobufOffset, protobufLength);
+ final byte[] computed = md5.digest();
+ final int md5Offset = protobufOffset + protobufLength;
+ return Arrays.equals(computed, 0, MD5_DIGEST_LENGTH, data, md5Offset, md5Offset + MD5_DIGEST_LENGTH);
+ }
+
+ private static MessageDigest getMd5Digest() {
+ try {
+ return MessageDigest.getInstance("MD5");
+ } catch (final NoSuchAlgorithmException e) {
+ throw new IllegalStateException("MD5 algorithm not available", e);
+ }
+ }
+
+ private static void parseAggregatedRecord(final String shardId, final Record kinesisRecord,
+ final byte[] data, final int protobufOffset, final int protobufLength, final List out) throws Exception {
+
+ final List partitionKeyTable = new ArrayList<>();
+ final List subRecordData = new ArrayList<>();
+ final List subRecordPartitionKeyIndexes = new ArrayList<>();
+
+ final CodedInputStream input = CodedInputStream.newInstance(data, protobufOffset, protobufLength);
+ while (!input.isAtEnd()) {
+ final int tag = input.readTag();
+ final int fieldNumber = WireFormat.getTagFieldNumber(tag);
+ switch (fieldNumber) {
+ case FIELD_PARTITION_KEY_TABLE:
+ partitionKeyTable.add(input.readString());
+ break;
+ case FIELD_EXPLICIT_HASH_KEY_TABLE:
+ input.readString();
+ break;
+ case FIELD_RECORDS:
+ final int length = input.readRawVarint32();
+ final int oldLimit = input.pushLimit(length);
+ int partitionKeyIndex = 0;
+ byte[] subRecordPayload = new byte[0];
+ while (!input.isAtEnd()) {
+ final int innerTag = input.readTag();
+ final int innerField = WireFormat.getTagFieldNumber(innerTag);
+ switch (innerField) {
+ case RECORD_FIELD_PARTITION_KEY_INDEX:
+ partitionKeyIndex = (int) input.readUInt64();
+ break;
+ case RECORD_FIELD_EXPLICIT_HASH_KEY_INDEX:
+ input.readUInt64();
+ break;
+ case RECORD_FIELD_DATA:
+ subRecordPayload = input.readByteArray();
+ break;
+ default:
+ input.skipField(innerTag);
+ break;
+ }
+ }
+ input.popLimit(oldLimit);
+ subRecordData.add(subRecordPayload);
+ subRecordPartitionKeyIndexes.add(partitionKeyIndex);
+ break;
+ default:
+ input.skipField(tag);
+ break;
+ }
+ }
+
+ final String sequenceNumber = kinesisRecord.sequenceNumber();
+ final Instant arrival = kinesisRecord.approximateArrivalTimestamp();
+ final String fallbackPartitionKey = kinesisRecord.partitionKey();
+
+ for (int i = 0; i < subRecordData.size(); i++) {
+ final int partitionKeyTableIndex = subRecordPartitionKeyIndexes.get(i);
+ final String partitionKey = partitionKeyTableIndex < partitionKeyTable.size()
+ ? partitionKeyTable.get(partitionKeyTableIndex) : fallbackPartitionKey;
+
+ final UserRecord record = new UserRecord(shardId, sequenceNumber, i, partitionKey, subRecordData.get(i), arrival);
+ out.add(record);
+ }
+ }
+
+ private static UserRecord passthrough(final String shardId, final Record record, final byte[] data) {
+ return new UserRecord(
+ shardId,
+ record.sequenceNumber(),
+ 0,
+ record.partitionKey(),
+ data,
+ record.approximateArrivalTimestamp());
+ }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java
deleted file mode 100644
index fd2aa3be0fb4..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordConverter;
-import org.apache.nifi.schema.access.SchemaNotFoundException;
-import org.apache.nifi.serialization.MalformedRecordException;
-import org.apache.nifi.serialization.RecordReader;
-import org.apache.nifi.serialization.RecordReaderFactory;
-import org.apache.nifi.serialization.RecordSetWriter;
-import org.apache.nifi.serialization.RecordSetWriterFactory;
-import org.apache.nifi.serialization.WriteResult;
-import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordSchema;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static java.util.Collections.emptyMap;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.MIME_TYPE;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE;
-
-final class ReaderRecordProcessor {
-
- private final RecordReaderFactory recordReaderFactory;
- private final KinesisRecordConverter recordConverter;
- private final RecordSetWriterFactory recordWriterFactory;
- private final ComponentLog logger;
-
- ReaderRecordProcessor(
- final RecordReaderFactory recordReaderFactory,
- final KinesisRecordConverter recordConverter,
- final RecordSetWriterFactory recordWriterFactory,
- final ComponentLog logger) {
- this.recordReaderFactory = recordReaderFactory;
- this.recordConverter = recordConverter;
- this.recordWriterFactory = recordWriterFactory;
- this.logger = logger;
- }
-
- ProcessingResult processRecords(
- final ProcessSession session,
- final String streamName,
- final String shardId,
- final List records) {
- final List successFlowFiles = new ArrayList<>();
- final List failureFlowFiles = new ArrayList<>();
-
- ActiveFlowFile activeFlowFile = null;
-
- for (final KinesisClientRecord kinesisRecord : records) {
- final int dataSize = kinesisRecord.data().remaining();
- final byte[] data = new byte[dataSize];
- kinesisRecord.data().get(data);
-
- try (final InputStream in = new ByteArrayInputStream(data);
- final RecordReader reader = recordReaderFactory.createRecordReader(emptyMap(), in, data.length, logger)) {
-
- Record record;
- while ((record = reader.nextRecord()) != null) {
- final Record convertedRecord = recordConverter.convert(record, kinesisRecord, streamName, shardId);
- final RecordSchema writeSchema = recordWriterFactory.getSchema(emptyMap(), convertedRecord.getSchema());
-
- if (activeFlowFile == null) {
- activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId);
- } else if (!writeSchema.equals(activeFlowFile.schema())) {
- // If the write schema has changed, we need to complete the current FlowFile and start a new one.
- final FlowFile completedFlowFile = activeFlowFile.complete();
- successFlowFiles.add(completedFlowFile);
-
- activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId);
- }
-
- activeFlowFile.writeRecord(convertedRecord, kinesisRecord);
- }
- } catch (final IOException | MalformedRecordException | SchemaNotFoundException e) {
- logger.error("Reader or Writer failed to process Kinesis Record with Stream Name [{}] Shard Id [{}] Sequence Number [{}] SubSequence Number [{}]",
- streamName, shardId, kinesisRecord.sequenceNumber(), kinesisRecord.subSequenceNumber(), e);
- final FlowFile failureFlowFile = createParseFailureFlowFile(session, streamName, shardId, kinesisRecord, e);
- failureFlowFiles.add(failureFlowFile);
- }
- }
-
- if (activeFlowFile != null) {
- final FlowFile completedFlowFile = activeFlowFile.complete();
- successFlowFiles.add(completedFlowFile);
- }
-
- return new ProcessingResult(successFlowFiles, failureFlowFiles);
- }
-
- private static FlowFile createParseFailureFlowFile(
- final ProcessSession session,
- final String streamName,
- final String shardId,
- final KinesisClientRecord record,
- final Exception e) {
- FlowFile flowFile = session.create();
-
- record.data().rewind();
- flowFile = session.write(flowFile, out -> {
- try (final WritableByteChannel channel = Channels.newChannel(out)) {
- channel.write(record.data());
- }
- });
-
- final Map attributes = ConsumeKinesisAttributes.fromKinesisRecords(streamName, shardId, record, record);
-
- final Throwable cause = e.getCause() != null ? e.getCause() : e;
- attributes.put(RECORD_ERROR_MESSAGE, cause.toString());
-
- flowFile = session.putAllAttributes(flowFile, attributes);
- session.getProvenanceReporter().receive(flowFile, ProvenanceTransitUriFormat.toTransitUri(streamName, shardId));
-
- return flowFile;
- }
-
- record ProcessingResult(List successFlowFiles, List parseFailureFlowFiles) {
- }
-
- /**
- * A class that manages a single {@link FlowFile} with a static schema that is currently being written to.
- * On a schema change the current {@link ActiveFlowFile} should be completed a new instance of this class
- * with a new schema should be created.
- *
- * An {@link ActiveFlowFile} must have at least one record written to it before it can be completed.
- */
- private static final class ActiveFlowFile {
-
- private final ComponentLog logger;
-
- private final ProcessSession session;
- private final FlowFile flowFile;
- private final RecordSetWriter writer;
- private final RecordSchema schema;
-
- private final String streamName;
- private final String shardId;
-
- private KinesisClientRecord firstRecord;
- private KinesisClientRecord lastRecord;
-
- private ActiveFlowFile(
- final ComponentLog logger,
- final ProcessSession session,
- final FlowFile flowFile,
- final RecordSetWriter writer,
- final RecordSchema schema,
- final String streamName,
- final String shardId) {
- this.logger = logger;
- this.session = session;
- this.flowFile = flowFile;
- this.writer = writer;
- this.schema = schema;
- this.streamName = streamName;
- this.shardId = shardId;
- }
-
- static ActiveFlowFile startNewFile(
- final ComponentLog logger,
- final ProcessSession session,
- final RecordSetWriterFactory recordWriterFactory,
- final RecordSchema writeSchema,
- final String streamName,
- final String shardId) throws SchemaNotFoundException {
- final FlowFile flowFile = session.create();
- final OutputStream outputStream = session.write(flowFile);
-
- try {
- final RecordSetWriter writer = recordWriterFactory.createWriter(logger, writeSchema, outputStream, flowFile);
- writer.beginRecordSet();
-
- return new ActiveFlowFile(logger, session, flowFile, writer, writeSchema, streamName, shardId);
-
- } catch (final SchemaNotFoundException e) {
- logger.debug("Failed to find writeSchema for Kinesis stream record: {}", e.getMessage());
- try {
- outputStream.close();
- } catch (final IOException ioe) {
- e.addSuppressed(ioe);
- }
- throw e;
-
- } catch (final IOException e) {
- final ProcessException processException = new ProcessException("Failed to create a writer for a FlowFile", e);
-
- logger.debug("Stopping Kinesis records processing. Failed to create a writer for a FlowFile: {}", e.getMessage());
- try {
- outputStream.close();
- } catch (final IOException ioe) {
- processException.addSuppressed(ioe);
- }
- throw processException;
- }
- }
-
- RecordSchema schema() {
- return schema;
- }
-
- void writeRecord(final Record record, final KinesisClientRecord kinesisRecord) {
- try {
- writer.write(record);
- } catch (final IOException e) {
- logger.debug("Stopping Kinesis records processing. Failed to write to a FlowFile: {}", e.getMessage());
- throw new ProcessException("Failed to write a record into a FlowFile", e);
- }
-
- if (firstRecord == null) {
- firstRecord = kinesisRecord;
- }
- lastRecord = kinesisRecord;
- }
-
- FlowFile complete() {
- if (firstRecord == null || lastRecord == null) {
- throw new IllegalStateException("Cannot complete an ActiveFlowFile that has no records");
- }
-
- try {
- final WriteResult finalResult = writer.finishRecordSet();
- writer.close();
-
- final Map attributes = ConsumeKinesisAttributes.fromKinesisRecords(streamName, shardId, firstRecord, lastRecord);
- attributes.putAll(finalResult.getAttributes());
- attributes.put(RECORD_COUNT, String.valueOf(finalResult.getRecordCount()));
- attributes.put(MIME_TYPE, writer.getMimeType());
-
- final FlowFile completedFlowFile = session.putAllAttributes(flowFile, attributes);
- session.getProvenanceReporter().receive(completedFlowFile, ProvenanceTransitUriFormat.toTransitUri(streamName, shardId));
-
- return completedFlowFile;
-
- } catch (final IOException e) {
- final ProcessException processException = new ProcessException("Failed to complete a FlowFile", e);
-
- logger.debug("Stopping Kinesis records processing. Failed to complete a FlowFile: {}", e.getMessage());
- try {
- writer.close();
- } catch (final IOException ioe) {
- processException.addSuppressed(ioe);
- }
-
- throw processException;
- }
- }
- }
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/RecordBuffer.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/RecordBuffer.java
deleted file mode 100644
index 12fa52c6fe7a..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/RecordBuffer.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis;
-
-import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-import java.util.List;
-import java.util.Optional;
-
-/**
- * RecordBuffer keeps track of all created Shard buffers, including exclusive read access via leasing.
- * It acts as the main interface between KCL callbacks and the {@link ConsumeKinesis} processor,
- * routing events to appropriate Shard buffers and ensuring thread-safe operations.
- */
-interface RecordBuffer {
-
- /**
- * Interface for interactions from the Kinesis Client Library to the Record Buffer.
- * Reflects the methods called by {@link software.amazon.kinesis.processor.ShardRecordProcessor}.
- */
- interface ForKinesisClientLibrary {
-
- ShardBufferId createBuffer(String shardId);
-
- void addRecords(ShardBufferId bufferId, List records, RecordProcessorCheckpointer checkpointer);
-
- /**
- * Called when a shard ends - waits until the buffer is flushed then performs the final checkpoint.
- */
- void checkpointEndedShard(ShardBufferId bufferId, RecordProcessorCheckpointer checkpointer);
-
- /**
- * Called when a consumer is shut down. Performs the checkpoint and returns
- * without waiting for the buffer to be flushed.
- */
- void shutdownShardConsumption(ShardBufferId bufferId, RecordProcessorCheckpointer checkpointer);
-
- /**
- * Called when lease is lost - immediately invalidates the buffer to prevent further operations.
- */
- void consumerLeaseLost(ShardBufferId bufferId);
- }
-
- /**
- * Interface for interactions from {@link ConsumeKinesis} processor to the Record Buffer.
- */
- interface ForProcessor {
-
- /**
- * Acquires an exclusive lease for a buffer that has data available for consumption.
- * If no data is available in the buffers, returns an empty Optional.
- *
- * After acquiring a lease, the processor can consume records from the buffer.
- * After consuming the records the processor must always {@link #returnBufferLease(LEASE)}.
- */
- Optional acquireBufferLease();
-
- /**
- * Consumes records from the buffer associated with the given lease.
- * The records have to be committed or rolled back later.
- */
- List consumeRecords(LEASE lease);
-
- void commitConsumedRecords(LEASE lease);
-
- void rollbackConsumedRecords(LEASE lease);
-
- /**
- * Returns the lease for a buffer back to the pool making it available for consumption again.
- * The method can be called multiple times with the same lease, but only the first call will actually take an effect.
- */
- void returnBufferLease(LEASE lease);
- }
-
- record ShardBufferId(String shardId, long bufferId) {
- }
-
- interface ShardBufferLease {
- String shardId();
- }
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ProvenanceTransitUriFormat.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ShardFetchResult.java
similarity index 70%
rename from nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ProvenanceTransitUriFormat.java
rename to nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ShardFetchResult.java
index e7575d6fcf77..9b88d71a9a4a 100644
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ProvenanceTransitUriFormat.java
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ShardFetchResult.java
@@ -16,12 +16,16 @@
*/
package org.apache.nifi.processors.aws.kinesis;
-final class ProvenanceTransitUriFormat {
+import java.math.BigInteger;
+import java.util.List;
- static String toTransitUri(final String streamName, final String shardId) {
- return "kinesis:stream/" + streamName + "/" + shardId;
+record ShardFetchResult(String shardId, List records, long millisBehindLatest) {
+
+ BigInteger firstSequenceNumber() {
+ return new BigInteger(records.getFirst().sequenceNumber());
}
- private ProvenanceTransitUriFormat() {
+ BigInteger lastSequenceNumber() {
+ return new BigInteger(records.getLast().sequenceNumber());
}
}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/UserRecord.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/UserRecord.java
new file mode 100644
index 000000000000..e44de3c506f1
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/UserRecord.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import java.time.Instant;
+
+/**
+ * A single user record extracted from a Kinesis record. For non-aggregated records,
+ * the fields map directly from the Kinesis API {@code Record}. For KPL-aggregated
+ * records, each sub-record within the aggregate gets its own instance with a unique
+ * {@code subSequenceNumber}.
+ *
+ * @param shardId the shard from which this record was fetched
+ * @param sequenceNumber the Kinesis sequence number of the enclosing record
+ * @param subSequenceNumber zero for non-aggregated records; index within the aggregate for KPL records
+ * @param partitionKey the partition key (from the enclosing record or the KPL sub-record)
+ * @param data the user payload bytes
+ * @param approximateArrivalTimestamp approximate time the enclosing record arrived at Kinesis
+ */
+record UserRecord(
+ String shardId,
+ String sequenceNumber,
+ long subSequenceNumber,
+ String partitionKey,
+ byte[] data,
+ Instant approximateArrivalTimestamp) {
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/InjectMetadataRecordConverter.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/InjectMetadataRecordConverter.java
deleted file mode 100644
index d454577d420b..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/InjectMetadataRecordConverter.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis.converter;
-
-import org.apache.nifi.serialization.SimpleRecordSchema;
-import org.apache.nifi.serialization.record.MapRecord;
-import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordSchema;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordMetadata.FIELD_METADATA;
-import static org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordMetadata.METADATA;
-import static org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordMetadata.composeMetadataObject;
-
-public final class InjectMetadataRecordConverter implements KinesisRecordConverter {
-
- @Override
- public Record convert(final Record record, final KinesisClientRecord kinesisRecord, final String streamName, final String shardId) {
- final List schemaFields = new ArrayList<>(record.getSchema().getFields());
- schemaFields.add(FIELD_METADATA);
- final RecordSchema schema = new SimpleRecordSchema(schemaFields);
-
- final Record metadata = composeMetadataObject(kinesisRecord, streamName, shardId);
- final Map recordValues = new HashMap<>(record.toMap());
- recordValues.put(METADATA, metadata);
-
- return new MapRecord(schema, recordValues);
- }
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/KinesisRecordConverter.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/KinesisRecordConverter.java
deleted file mode 100644
index 83a65fd81f2f..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/KinesisRecordConverter.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis.converter;
-
-import org.apache.nifi.serialization.record.Record;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-public interface KinesisRecordConverter {
-
- Record convert(Record record, KinesisClientRecord kinesisRecord, String streamName, String shardId);
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/ValueRecordConverter.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/ValueRecordConverter.java
deleted file mode 100644
index b67ae22072cd..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/ValueRecordConverter.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis.converter;
-
-import org.apache.nifi.serialization.record.Record;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-public final class ValueRecordConverter implements KinesisRecordConverter {
-
- @Override
- public Record convert(final Record record, final KinesisClientRecord kinesisRecord, final String streamName, final String shardId) {
- return record;
- }
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/WrapperRecordConverter.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/WrapperRecordConverter.java
deleted file mode 100644
index 693077f4437e..000000000000
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/converter/WrapperRecordConverter.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.aws.kinesis.converter;
-
-import org.apache.nifi.serialization.SimpleRecordSchema;
-import org.apache.nifi.serialization.record.MapRecord;
-import org.apache.nifi.serialization.record.Record;
-import org.apache.nifi.serialization.record.RecordField;
-import org.apache.nifi.serialization.record.RecordFieldType;
-import org.apache.nifi.serialization.record.RecordSchema;
-import software.amazon.kinesis.retrieval.KinesisClientRecord;
-
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordMetadata.FIELD_METADATA;
-import static org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordMetadata.METADATA;
-import static org.apache.nifi.processors.aws.kinesis.converter.KinesisRecordMetadata.composeMetadataObject;
-
-public final class WrapperRecordConverter implements KinesisRecordConverter {
-
- private static final String VALUE = "value";
-
- @Override
- public Record convert(final Record record, final KinesisClientRecord kinesisRecord, final String streamName, final String shardId) {
- final Record metadata = composeMetadataObject(kinesisRecord, streamName, shardId);
-
- final RecordSchema convertedSchema = new SimpleRecordSchema(List.of(
- FIELD_METADATA,
- new RecordField(VALUE, RecordFieldType.RECORD.getRecordDataType(record.getSchema())))
- );
- final Map convertedRecord = Map.of(
- METADATA, metadata,
- VALUE, record
- );
-
- return new MapRecord(convertedSchema, convertedRecord);
- }
-}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/CheckpointTableUtilsTest.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/CheckpointTableUtilsTest.java
new file mode 100644
index 000000000000..22fc1505299b
--- /dev/null
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/CheckpointTableUtilsTest.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.aws.kinesis;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.PutItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.PutItemResponse;
+import software.amazon.awssdk.services.dynamodb.model.ScanRequest;
+import software.amazon.awssdk.services.dynamodb.model.ScanResponse;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class CheckpointTableUtilsTest {
+
+ private static final String STREAM_NAME = "my-stream";
+ private static final String SHARD_ID_1 = "shardId-0001";
+ private static final String SHARD_ID_2 = "shardId-0002";
+ private static final String SOURCE_TABLE = "source-table";
+ private static final String DEST_TABLE = "dest-table";
+
+ private static AttributeValue str(final String value) {
+ return AttributeValue.builder().s(value).build();
+ }
+
+ @Test
+ void testCopyCheckpointItemsCopiesShardItems() {
+ final DynamoDbClient dynamoDb = mock(DynamoDbClient.class);
+ final ComponentLog logger = mock(ComponentLog.class);
+
+ final Map item = Map.of(
+ "streamName", str(STREAM_NAME),
+ "shardId", str(SHARD_ID_1),
+ "sequenceNumber", str("12345"));
+ when(dynamoDb.scan(any(ScanRequest.class))).thenReturn(ScanResponse.builder().items(item).build());
+ when(dynamoDb.putItem(any(PutItemRequest.class))).thenReturn(PutItemResponse.builder().build());
+
+ CheckpointTableUtils.copyCheckpointItems(dynamoDb, logger, SOURCE_TABLE, DEST_TABLE);
+
+ final ArgumentCaptor putCaptor = ArgumentCaptor.forClass(PutItemRequest.class);
+ verify(dynamoDb, times(1)).putItem(putCaptor.capture());
+ assertEquals(item, putCaptor.getValue().item());
+ }
+
+ @Test
+ void testCopyCheckpointItemsSkipsNodeAndMigrationMarkers() {
+ final DynamoDbClient dynamoDb = mock(DynamoDbClient.class);
+ final ComponentLog logger = mock(ComponentLog.class);
+
+ final Map nodeItem = Map.of(
+ "streamName", str(STREAM_NAME),
+ "shardId", str("__node__#node-a"));
+ final Map migrationMarkerItem = Map.of(
+ "streamName", str(STREAM_NAME),
+ "shardId", str("__migration__"));
+ final Map shardItem = Map.of(
+ "streamName", str(STREAM_NAME),
+ "shardId", str(SHARD_ID_2),
+ "sequenceNumber", str("67890"));
+
+ when(dynamoDb.scan(any(ScanRequest.class))).thenReturn(
+ ScanResponse.builder().items(List.of(nodeItem, migrationMarkerItem, shardItem)).build());
+ when(dynamoDb.putItem(any(PutItemRequest.class))).thenReturn(PutItemResponse.builder().build());
+
+ CheckpointTableUtils.copyCheckpointItems(dynamoDb, logger, SOURCE_TABLE, DEST_TABLE);
+
+ final ArgumentCaptor putCaptor = ArgumentCaptor.forClass(PutItemRequest.class);
+ verify(dynamoDb, times(1)).putItem(putCaptor.capture());
+ assertEquals(shardItem, putCaptor.getValue().item());
+ }
+
+ @Test
+ void testCopyCheckpointItemsSkipsAllMarkers() {
+ final DynamoDbClient dynamoDb = mock(DynamoDbClient.class);
+ final ComponentLog logger = mock(ComponentLog.class);
+
+ final Map nodeItem = Map.of(
+ "streamName", str(STREAM_NAME),
+ "shardId", str("__node__#node-b"));
+
+ when(dynamoDb.scan(any(ScanRequest.class))).thenReturn(
+ ScanResponse.builder().items(List.of(nodeItem)).build());
+
+ CheckpointTableUtils.copyCheckpointItems(dynamoDb, logger, SOURCE_TABLE, DEST_TABLE);
+
+ verify(dynamoDb, never()).putItem(any(PutItemRequest.class));
+ }
+}
diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesisIT.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesisIT.java
index 6a5b866f25da..28e833222083 100644
--- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesisIT.java
+++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/test/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesisIT.java
@@ -16,764 +16,1048 @@
*/
package org.apache.nifi.processors.aws.kinesis;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import com.google.protobuf.ByteString;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.nifi.avro.AvroReader;
+import org.apache.nifi.avro.AvroRecordSetWriter;
+import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.json.JsonRecordSetWriter;
import org.apache.nifi.json.JsonTreeReader;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.FlowFileHandlingException;
+import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.processors.aws.region.RegionUtil;
import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.ProvenanceEventType;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.schema.access.SchemaAccessUtils;
-import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.MockProcessSession;
-import org.apache.nifi.util.SharedSessionState;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
-import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
-import org.junit.jupiter.api.parallel.Execution;
-import org.junit.jupiter.api.parallel.ExecutionMode;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.EnabledIfDockerAvailable;
+import org.testcontainers.junit.jupiter.Testcontainers;
import org.testcontainers.localstack.LocalStackContainer;
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.core.SdkBytes;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
import software.amazon.awssdk.services.kinesis.KinesisClient;
-import software.amazon.awssdk.services.kinesis.model.Consumer;
-import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse;
-import software.amazon.awssdk.services.kinesis.model.ListStreamConsumersResponse;
-import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
-import software.amazon.awssdk.services.kinesis.model.ScalingType;
-import software.amazon.awssdk.services.kinesis.model.StreamDescription;
+import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.kinesis.model.PutRecordRequest;
import software.amazon.awssdk.services.kinesis.model.StreamStatus;
-
-import java.net.URI;
-import java.time.Duration;
-import java.util.Collection;
+import software.amazon.kinesis.retrieval.kpl.Messages;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.IntStream;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.stream.Collectors.groupingBy;
-import static java.util.stream.Collectors.mapping;
-import static java.util.stream.Collectors.toList;
-import static java.util.stream.Collectors.toSet;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesis.REL_PARSE_FAILURE;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesis.REL_SUCCESS;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT;
-import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE;
-import static org.apache.nifi.processors.aws.kinesis.JsonRecordAssert.assertFlowFileRecordPayloads;
-import static org.junit.jupiter.api.Assertions.assertAll;
+import java.util.stream.Collectors;
+
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Timeout.ThreadMode.SEPARATE_THREAD;
-/**
- * Tests run in parallel to optimize execution time as Kinesis consumer coordination takes a lot.
- */
-@Execution(ExecutionMode.CONCURRENT)
-@Timeout(value = 5, unit = MINUTES, threadMode = SEPARATE_THREAD)
+@Testcontainers
+@EnabledIfDockerAvailable
class ConsumeKinesisIT {
- private static final Logger logger = LoggerFactory.getLogger(ConsumeKinesisIT.class);
- private static final DockerImageName LOCALSTACK_IMAGE = DockerImageName.parse("localstack/localstack:4.12.0");
+ @Container
+ private static final LocalStackContainer LOCALSTACK = new LocalStackContainer(
+ DockerImageName.parse("localstack/localstack:4"));
+
+ private static final String AVRO_SCHEMA_A = """
+ {
+ "type": "record",
+ "name": "A",
+ "fields": [
+ {"name": "id", "type": "int"},
+ {"name": "name", "type": "string"}
+ ]
+ }""";
+
+ private static final String AVRO_SCHEMA_B = """
+ {
+ "type": "record",
+ "name": "B",
+ "fields": [
+ {"name": "code", "type": "string"},
+ {"name": "value", "type": "double"}
+ ]
+ }""";
- private static final LocalStackContainer localstack = new LocalStackContainer(LOCALSTACK_IMAGE).withServices("kinesis", "dynamodb", "cloudwatch");
+ private TestRunner runner;
+ private KinesisClient kinesisClient;
+ private int credentialServiceCounter = 0;
- private static KinesisClient kinesisClient;
- private static DynamoDbClient dynamoDbClient;
+ @BeforeEach
+ void setUp() throws Exception {
+ kinesisClient = KinesisClient.builder()
+ .endpointOverride(LOCALSTACK.getEndpoint())
+ .credentialsProvider(StaticCredentialsProvider.create(
+ AwsBasicCredentials.create(LOCALSTACK.getAccessKey(), LOCALSTACK.getSecretKey())))
+ .region(Region.of(LOCALSTACK.getRegion()))
+ .build();
- private String streamName;
- private String applicationName;
- private TestRunner runner;
- private TestKinesisStreamClient streamClient;
+ runner = TestRunners.newTestRunner(FastTimingConsumeKinesis.class);
- @BeforeAll
- static void oneTimeSetup() {
- localstack.start();
+ final JsonTreeReader reader = new JsonTreeReader();
+ runner.addControllerService("json-reader", reader);
+ runner.enableControllerService(reader);
- final AwsCredentialsProvider credentialsProvider = StaticCredentialsProvider.create(
- AwsBasicCredentials.create(localstack.getAccessKey(), localstack.getSecretKey())
- );
+ final JsonRecordSetWriter normalWriter = new JsonRecordSetWriter();
+ runner.addControllerService("json-writer", normalWriter);
+ runner.enableControllerService(normalWriter);
- kinesisClient = KinesisClient.builder()
- .endpointOverride(localstack.getEndpoint())
- .credentialsProvider(credentialsProvider)
- .region(Region.of(localstack.getRegion()))
- .build();
+ final FailingRecordSetWriterFactory failingWriter = new FailingRecordSetWriterFactory();
+ runner.addControllerService("failing-writer", failingWriter);
+ runner.enableControllerService(failingWriter);
- dynamoDbClient = DynamoDbClient.builder()
- .endpointOverride(localstack.getEndpoint())
- .credentialsProvider(credentialsProvider)
- .region(Region.of(localstack.getRegion()))
- .build();
+ addCredentialService(runner, "creds");
+ runner.setProperty(ConsumeKinesis.APPLICATION_NAME, "test-app-" + System.currentTimeMillis());
+ runner.setProperty(ConsumeKinesis.AWS_CREDENTIALS_PROVIDER_SERVICE, "creds");
+ runner.setProperty(RegionUtil.REGION, LOCALSTACK.getRegion());
+ runner.setProperty(ConsumeKinesis.ENDPOINT_OVERRIDE, LOCALSTACK.getEndpoint().toString());
+ runner.setProperty(ConsumeKinesis.MAX_BATCH_DURATION, "200 ms");
}
- @AfterAll
- static void tearDown() {
+ @AfterEach
+ void tearDown() {
if (kinesisClient != null) {
kinesisClient.close();
}
- if (dynamoDbClient != null) {
- dynamoDbClient.close();
- }
- localstack.stop();
}
- @BeforeEach
- void setUp() throws InitializationException {
- final UUID testId = UUID.randomUUID();
- streamName = "%s-kinesis-stream-%s".formatted(getClass().getSimpleName(), testId);
- streamClient = new TestKinesisStreamClient(kinesisClient, streamName);
- applicationName = "%s-test-kinesis-app-%s".formatted(getClass().getSimpleName(), testId);
- runner = createTestRunner(streamName, applicationName);
- }
+ @Test
+ void testFlowFilePerRecordStrategy() throws Exception {
+ final String streamName = "per-record-test";
+ final int recordCount = 5;
- @AfterEach
- void tearDownEach() {
- runner.stop();
+ createStream(streamName);
+ publishRecords(streamName, recordCount);
- if (streamClient != null) {
- try {
- streamClient.deleteStream();
- } catch (final Exception e) {
- logger.warn("Failed to delete stream {}: {}", streamName, e.getMessage());
- }
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "FLOW_FILE");
+ runUntilOutput(runner);
+
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertEquals(recordCount, flowFiles.size(), "Expected one FlowFile per Kinesis record");
+
+ for (final MockFlowFile ff : flowFiles) {
+ assertEquals("1", ff.getAttribute("record.count"));
+ assertEquals(streamName, ff.getAttribute(ConsumeKinesis.ATTR_STREAM_NAME));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_SHARD_ID));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_FIRST_SEQUENCE));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_LAST_SEQUENCE));
+ assertEquals(ff.getAttribute(ConsumeKinesis.ATTR_FIRST_SEQUENCE), ff.getAttribute(ConsumeKinesis.ATTR_LAST_SEQUENCE));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_PARTITION_KEY));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_FIRST_SUBSEQUENCE));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_LAST_SUBSEQUENCE));
+
+ final String content = ff.getContent();
+ assertTrue(content.startsWith("{"), "Expected raw JSON content: " + content);
}
- // Removing tables generated by KCL.
- deleteTable(applicationName);
- deleteTable(applicationName + "-CoordinatorState");
- deleteTable(applicationName + "-WorkerMetricStats");
+ final Set emittedShardIds = flowFiles.stream()
+ .map(ff -> ff.getAttribute(ConsumeKinesis.ATTR_SHARD_ID))
+ .collect(Collectors.toSet());
+ final List receiveEvents = runner.getProvenanceEvents().stream()
+ .filter(event -> "RECEIVE".equals(event.getEventType().name()))
+ .toList();
+ assertEquals(recordCount, receiveEvents.size(), "Expected one RECEIVE event per emitted FlowFile");
+ for (final ProvenanceEventRecord receiveEvent : receiveEvents) {
+ final String transitUri = receiveEvent.getTransitUri();
+ assertNotNull(transitUri, "RECEIVE event should include a transit URI");
+ assertTrue(emittedShardIds.stream().anyMatch(shardId -> transitUri.endsWith("/" + shardId)),
+ "RECEIVE transit URI should include one of the emitted shard IDs: " + transitUri);
+ }
+ final Long counter = runner.getCounterValue("Records Consumed");
+ assertNotNull(counter, "Records Consumed counter should be set");
+ assertEquals(recordCount, counter.longValue());
}
- private void deleteTable(final String tableName) {
- try {
- dynamoDbClient.deleteTable(req -> req.tableName(tableName));
- } catch (final Exception e) {
- logger.warn("Failed to delete DynamoDB table {}: {}", tableName, e.getMessage());
+ @Test
+ void testRecordOrientedStrategy() throws Exception {
+ final String streamName = "record-oriented-test";
+ final int recordCount = 5;
+
+ createStream(streamName);
+ publishRecords(streamName, recordCount);
+
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "RECORD");
+ runner.setProperty(ConsumeKinesis.RECORD_READER, "json-reader");
+ runner.setProperty(ConsumeKinesis.RECORD_WRITER, "json-writer");
+ runUntilOutput(runner);
+
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertFalse(flowFiles.isEmpty(), "Expected at least one FlowFile");
+
+ long totalRecords = 0;
+ for (final MockFlowFile ff : flowFiles) {
+ totalRecords += Long.parseLong(ff.getAttribute("record.count"));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_STREAM_NAME));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_FIRST_SEQUENCE));
+ assertNotNull(ff.getAttribute(ConsumeKinesis.ATTR_LAST_SEQUENCE));
}
+ assertEquals(recordCount, totalRecords, "Total record count across all FlowFiles");
+
+ final Long counter = runner.getCounterValue("Records Consumed");
+ assertNotNull(counter, "Records Consumed counter should be set");
+ assertEquals(recordCount, counter.longValue());
}
@Test
- void testConsumeSingleMessageFromSingleShard() {
- streamClient.createStream(1);
-
- final String testMessage = "Hello, Kinesis!";
- streamClient.putRecord("test-partition-key", testMessage);
+ void testRecordOrientedStrategyWithInjectedMetadata() throws Exception {
+ final String streamName = "record-oriented-metadata-test";
+ final int recordCount = 5;
- runProcessorWithInitAndWaitForFiles(runner, 1);
+ createStream(streamName);
+ publishRecords(streamName, recordCount);
- runner.assertTransferCount(REL_SUCCESS, 1);
- final List flowFiles = runner.getFlowFilesForRelationship(REL_SUCCESS);
- final MockFlowFile flowFile = flowFiles.getFirst();
-
- flowFile.assertContentEquals(testMessage);
- flowFile.assertAttributeEquals("aws.kinesis.partition.key", "test-partition-key");
- assertNotNull(flowFile.getAttribute("aws.kinesis.first.sequence.number"));
- assertNotNull(flowFile.getAttribute("aws.kinesis.last.sequence.number"));
- assertNotNull(flowFile.getAttribute("aws.kinesis.shard.id"));
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "RECORD");
+ runner.setProperty(ConsumeKinesis.RECORD_READER, "json-reader");
+ runner.setProperty(ConsumeKinesis.RECORD_WRITER, "json-writer");
+ runner.setProperty(ConsumeKinesis.OUTPUT_STRATEGY, "INJECT_METADATA");
+ runUntilOutput(runner);
+
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertFalse(flowFiles.isEmpty(), "Expected at least one FlowFile");
+
+ long totalRecords = 0;
+ for (final MockFlowFile flowFile : flowFiles) {
+ totalRecords += Long.parseLong(flowFile.getAttribute("record.count"));
+
+ final String content = flowFile.getContent();
+ assertTrue(content.contains("\"kinesisMetadata\""), "Expected injected kinesisMetadata object");
+ assertTrue(content.contains("\"stream\":\"" + streamName + "\""), "Expected stream in injected metadata");
+ assertTrue(content.contains("\"shardId\":\""), "Expected shardId in injected metadata");
+ assertTrue(content.contains("\"sequenceNumber\":\""), "Expected sequenceNumber in injected metadata");
+ assertTrue(content.contains("\"subSequenceNumber\":0"), "Expected default subSequenceNumber in injected metadata");
+ assertTrue(content.contains("\"partitionKey\":\""), "Expected partitionKey in injected metadata");
+ }
- assertReceiveProvenanceEvents(runner.getProvenanceEvents(), flowFile);
+ assertEquals(recordCount, totalRecords, "Total record count across all FlowFiles");
- // Creates an enhanced fan-out consumer by default.
- assertEquals(
- List.of(applicationName),
- streamClient.getEnhancedFanOutConsumerNames(),
- "Expected a single enhanced fan-out consumer with an application name");
+ final Long counter = runner.getCounterValue("Records Consumed");
+ assertNotNull(counter, "Records Consumed counter should be set");
+ assertEquals(recordCount, counter.longValue());
}
@Test
- void testConsumeSingleMessageFromSingleShard_withoutEnhancedFanOut() {
- runner.setProperty(ConsumeKinesis.CONSUMER_TYPE, ConsumeKinesis.ConsumerType.SHARED_THROUGHPUT);
+ void testDemarcatorStrategy() throws Exception {
+ final String streamName = "demarcator-test";
+ final int recordCount = 3;
+
+ createStream(streamName);
+ publishRecords(streamName, recordCount);
+
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "LINE_DELIMITED");
+ runUntilOutput(runner);
+
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertFalse(flowFiles.isEmpty(), "Expected at least one FlowFile");
- streamClient.createStream(1);
+ final StringBuilder allContent = new StringBuilder();
+ long totalRecords = 0;
+ for (final MockFlowFile ff : flowFiles) {
+ allContent.append(ff.getContent());
+ totalRecords += Long.parseLong(ff.getAttribute("record.count"));
+ }
+ assertEquals(recordCount, totalRecords, "Total record count across all FlowFiles");
- final String testMessage = "Hello, Kinesis!";
- streamClient.putRecord("test-partition-key", testMessage);
+ final String[] lines = allContent.toString().split("\n");
+ assertEquals(recordCount, lines.length, "Expected one line per Kinesis record");
+ for (final String line : lines) {
+ assertTrue(line.startsWith("{"), "Expected JSON content: " + line);
+ }
- runProcessorWithInitAndWaitForFiles(runner, 1);
+ final Long counter = runner.getCounterValue("Records Consumed");
+ assertNotNull(counter, "Records Consumed counter should be set");
+ assertEquals(recordCount, counter.longValue());
+ }
- runner.assertTransferCount(REL_SUCCESS, 1);
- final List flowFiles = runner.getFlowFilesForRelationship(REL_SUCCESS);
- final MockFlowFile flowFile = flowFiles.getFirst();
+ @Test
+ void testDemarcatorStrategyWithCustomDelimiter() throws Exception {
+ final String streamName = "custom-delim-test";
+ final int recordCount = 3;
+ final String delimiter = "|||";
- flowFile.assertContentEquals(testMessage);
- flowFile.assertAttributeEquals("aws.kinesis.partition.key", "test-partition-key");
- assertNotNull(flowFile.getAttribute("aws.kinesis.first.sequence.number"));
- assertNotNull(flowFile.getAttribute("aws.kinesis.last.sequence.number"));
- assertNotNull(flowFile.getAttribute("aws.kinesis.shard.id"));
+ createStream(streamName);
+ publishRecords(streamName, recordCount);
- assertReceiveProvenanceEvents(runner.getProvenanceEvents(), flowFile);
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "DEMARCATOR");
+ runner.setProperty(ConsumeKinesis.MESSAGE_DEMARCATOR, delimiter);
+ runUntilOutput(runner);
+
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertFalse(flowFiles.isEmpty(), "Expected at least one FlowFile");
+
+ final StringBuilder allContent = new StringBuilder();
+ long totalRecords = 0;
+ for (final MockFlowFile ff : flowFiles) {
+ allContent.append(ff.getContent());
+ totalRecords += Long.parseLong(ff.getAttribute("record.count"));
+ }
+ assertEquals(recordCount, totalRecords, "Total record count across all FlowFiles");
- assertTrue(
- streamClient.getEnhancedFanOutConsumerNames().isEmpty(),
- "No enhanced fan-out consumers should be created for Shared Throughput consumer type");
+ final String[] parts = allContent.toString().split("\\|\\|\\|");
+ assertEquals(recordCount, parts.length, "Expected records separated by custom delimiter");
+ for (final String part : parts) {
+ assertTrue(part.startsWith("{"), "Expected JSON content: " + part);
+ }
+
+ final Long counter = runner.getCounterValue("Records Consumed");
+ assertNotNull(counter, "Records Consumed counter should be set");
+ assertEquals(recordCount, counter.longValue());
}
@Test
- void testConsumeManyMessagesFromSingleShardWithOrdering() {
- final int messageCount = 10;
+ void testFailedWriteRollsBackAndRecordsAreReConsumed() throws Exception {
+ final String streamName = "rollback-test";
+ final int recordCount = 5;
+
+ createStream(streamName);
+ publishRecords(streamName, recordCount);
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "RECORD");
+ runner.setProperty(ConsumeKinesis.RECORD_READER, "json-reader");
- streamClient.createStream(1);
+ runner.setProperty(ConsumeKinesis.RECORD_WRITER, "failing-writer");
+ runUntilOutput(runner);
- final List messages = IntStream.range(0, messageCount).mapToObj(i -> "Message-" + i).toList();
- streamClient.putRecords("partition-key", messages);
+ runner.assertTransferCount(ConsumeKinesis.REL_SUCCESS, 0);
+ runner.assertTransferCount(ConsumeKinesis.REL_PARSE_FAILURE, 0);
- runProcessorWithInitAndWaitForFiles(runner, messageCount);
+ runner.setProperty(ConsumeKinesis.RECORD_WRITER, "json-writer");
+ runUntilOutput(runner);
- runner.assertTransferCount(REL_SUCCESS, messageCount);
- final List flowFiles = runner.getFlowFilesForRelationship(REL_SUCCESS);
- final List flowFilesContent = flowFiles.stream().map(MockFlowFile::getContent).toList();
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertFalse(flowFiles.isEmpty(), "Expected at least one FlowFile transferred to success");
- assertEquals(messages, flowFilesContent);
+ long totalRecords = 0;
+ for (final MockFlowFile ff : flowFiles) {
+ totalRecords += Long.parseLong(ff.getAttribute("record.count"));
+ }
+ assertEquals(recordCount, totalRecords, "All records should be re-consumed after rollback");
- assertReceiveProvenanceEvents(runner.getProvenanceEvents(), flowFiles);
+ final Long counter = runner.getCounterValue("Records Consumed");
+ assertNotNull(counter, "Records Consumed counter should be set");
+ assertEquals(recordCount, counter.longValue());
}
@Test
- void testConsumeMessagesFromMultipleShardsStream() {
- final int shardCount = 5;
- final int messagesPerPartitionKey = 5;
+ void testClusterSimulationDistributesShards() throws Exception {
+ final String streamName = "cluster-sim-test";
+ final int shardCount = 4;
+ final String appName = "cluster-app-" + System.currentTimeMillis();
+
+ createStream(streamName, shardCount);
- streamClient.createStream(shardCount);
+ final TestRunner runner1 = createConfiguredRunner(streamName, appName);
+ final TestRunner runner2 = createConfiguredRunner(streamName, appName);
- // Every shard has message with the same payload.
- final List shardMessages = IntStream.range(0, messagesPerPartitionKey).mapToObj(String::valueOf).toList();
+ runner1.run(1, false, true);
+ runner2.run(1, false, true);
- IntStream.range(0, shardCount).forEach(shard -> streamClient.putRecords(String.valueOf(shard), shardMessages));
+ int recordId = 0;
+ final long deadline = System.currentTimeMillis() + 30_000;
+ while (System.currentTimeMillis() < deadline) {
+ publishRecord(streamName, recordId++);
+ Thread.sleep(10);
- // Run processor and wait for all records
- final int totalMessages = shardCount * messagesPerPartitionKey;
- runProcessorWithInitAndWaitForFiles(runner, totalMessages);
+ runner1.run(1, false, false);
+ runner2.run(1, false, false);
- // Verify results
- runner.assertTransferCount(REL_SUCCESS, totalMessages);
- final List flowFiles = runner.getFlowFilesForRelationship(REL_SUCCESS);
+ if (hasFlowFiles(runner1) && hasFlowFiles(runner2)) {
+ break;
+ }
+ }
- final Map> partitionKey2Messages = flowFiles.stream()
- .collect(groupingBy(
- f -> f.getAttribute("aws.kinesis.partition.key"),
- mapping(MockFlowFile::getContent, toList())
- ));
+ runner1.run(1, true, false);
+ runner2.run(1, true, false);
- assertEquals(shardCount, partitionKey2Messages.size());
+ final List flowFiles1 = runner1.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ final List flowFiles2 = runner2.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
- assertAll(
- partitionKey2Messages.values().stream()
- .map(actual -> () -> assertEquals(shardMessages, actual))
- );
+ assertFalse(flowFiles1.isEmpty(), "Runner 1 should have received data");
+ assertFalse(flowFiles2.isEmpty(), "Runner 2 should have received data");
- assertReceiveProvenanceEvents(runner.getProvenanceEvents(), flowFiles);
+ final Set uniqueRecords = new HashSet<>();
+ for (final MockFlowFile ff : flowFiles1) {
+ uniqueRecords.add(ff.getContent());
+ }
+ for (final MockFlowFile ff : flowFiles2) {
+ uniqueRecords.add(ff.getContent());
+ }
+ assertEquals(flowFiles1.size() + flowFiles2.size(), uniqueRecords.size(),
+ "No duplicate records should be consumed");
}
@Test
- @Disabled("Does not work with LocalStack: https://github.com/localstack/localstack/issues/12833. Enable only when using real Kinesis.")
- void testResharding_inParallelWithConsumption() {
- // Using partition keys with uniformally distributed hashes to ensure the data is distributed across split shards.
- final List partitionKeys = List.of(
- "pk-0-14", // 035517ff4ca68849589f43842c07362f
- "pk-1-14", // 5f045ae51eea9bd124d76041a6a27073
- "pk-2-2", // 85fb9a2b01b009904eb8a6fa13a21d6c
- "pk-3-2" // dbf24a6e26910143c60188e2fcb53b4f
- );
-
- // Data to be produced at each stage
- final int partitionRecordsPerStage = 3;
- final int totalStages = 5; // initial + 4 resharding stages with data
- final int totalRecordsPerPartition = partitionRecordsPerStage * totalStages;
- final int totalRecords = partitionKeys.size() * totalRecordsPerPartition;
-
- // Start resharding and data production operations in background thread.
- final Thread reshardingThread = new Thread(() -> {
- int messageSeq = 0; // For each partition key the message content are sequential numbers.
-
- streamClient.createStream(1);
- putRecords(partitionKeys, partitionRecordsPerStage, messageSeq);
- messageSeq += partitionRecordsPerStage;
-
- streamClient.reshardStream(2);
- putRecords(partitionKeys, partitionRecordsPerStage, messageSeq);
- messageSeq += partitionRecordsPerStage;
-
- streamClient.reshardStream(4);
- putRecords(partitionKeys, partitionRecordsPerStage, messageSeq);
- messageSeq += partitionRecordsPerStage;
-
- streamClient.reshardStream(3);
- putRecords(partitionKeys, partitionRecordsPerStage, messageSeq);
- messageSeq += partitionRecordsPerStage;
-
- streamClient.reshardStream(2);
- putRecords(partitionKeys, partitionRecordsPerStage, messageSeq);
- });
-
- reshardingThread.start();
-
- runProcessorWithInitAndWaitForFiles(runner, totalRecords);
-
- runner.assertTransferCount(REL_SUCCESS, totalRecords);
- final List flowFiles = runner.getFlowFilesForRelationship(REL_SUCCESS);
-
- final Map> partitionKeyToMessages = flowFiles.stream()
- .collect(groupingBy(
- f -> f.getAttribute("aws.kinesis.partition.key"),
- mapping(MockFlowFile::getContent, toList())
- ));
-
- final List expectedPartitionMessages = IntStream.range(0, totalRecordsPerPartition).mapToObj(Integer::toString).toList();
- assertAll(
- partitionKeyToMessages.entrySet().stream()
- .map(actual -> () -> assertEquals(
- expectedPartitionMessages,
- actual.getValue(),
- "Partition messages do not match expected for partition key: " + actual.getKey()))
- );
- }
-
- private void putRecords(final Collection partitionKeys, final int count, final int startIndex) {
- IntStream.range(startIndex, startIndex + count).forEach(i -> {
- final String message = Integer.toString(i);
- partitionKeys.forEach(partitionKey -> streamClient.putRecord(partitionKey, message));
- });
+ void testClusterScaleDownAndScaleUpRebalancesShards() throws Exception {
+ final String streamName = "cluster-rebalance-test";
+ final int shardCount = 6;
+ final String appName = "cluster-rebalance-app-" + System.currentTimeMillis();
+
+ createStream(streamName, shardCount);
+
+ final TestRunner runner1 = createConfiguredRunner(streamName, appName);
+ final TestRunner runner2 = createConfiguredRunner(streamName, appName);
+
+ runner1.run(1, false, true);
+ runner2.run(1, false, true);
+
+ int recordId = 0;
+ long deadline = System.currentTimeMillis() + 30_000;
+ while (System.currentTimeMillis() < deadline) {
+ publishRecord(streamName, recordId++);
+ Thread.sleep(10);
+ runner1.run(1, false, false);
+ runner2.run(1, false, false);
+ if (hasFlowFiles(runner1) && hasFlowFiles(runner2)) {
+ break;
+ }
+ }
+ assertFalse(runner1.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS).isEmpty(),
+ "Runner 1 should receive data in stage one");
+ assertFalse(runner2.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS).isEmpty(),
+ "Runner 2 should receive data in stage one");
+
+ final int runner1Checkpoint = runner1.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS).size();
+ runner2.run(1, true, false);
+
+ deadline = System.currentTimeMillis() + 30_000;
+ while (System.currentTimeMillis() < deadline) {
+ publishRecord(streamName, recordId++);
+ Thread.sleep(10);
+ runner1.run(1, false, false);
+
+ final Set shards = new HashSet<>();
+ for (final MockFlowFile ff : getNewFlowFiles(runner1, runner1Checkpoint)) {
+ shards.add(ff.getAttribute(ConsumeKinesis.ATTR_SHARD_ID));
+ }
+ if (shards.size() == shardCount) {
+ break;
+ }
+ }
+
+ final Set stageTwoShards = new HashSet<>();
+ for (final MockFlowFile ff : getNewFlowFiles(runner1, runner1Checkpoint)) {
+ stageTwoShards.add(ff.getAttribute(ConsumeKinesis.ATTR_SHARD_ID));
+ }
+ assertEquals(shardCount, stageTwoShards.size(), "Single active runner should consume from all shards");
+
+ final TestRunner runner3 = createConfiguredRunner(streamName, appName);
+ final TestRunner runner4 = createConfiguredRunner(streamName, appName);
+ final int runner1StageThreeCheckpoint = runner1.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS).size();
+
+ runner3.run(1, false, true);
+ runner4.run(1, false, true);
+
+ deadline = System.currentTimeMillis() + 30_000;
+ while (System.currentTimeMillis() < deadline) {
+ publishRecord(streamName, recordId++);
+ Thread.sleep(10);
+ runner1.run(1, false, false);
+ runner3.run(1, false, false);
+ runner4.run(1, false, false);
+
+ if (!getNewFlowFiles(runner1, runner1StageThreeCheckpoint).isEmpty()
+ && hasFlowFiles(runner3) && hasFlowFiles(runner4)) {
+ break;
+ }
+ }
+
+ runner1.run(1, true, false);
+ runner3.run(1, true, false);
+ runner4.run(1, true, false);
+
+ assertFalse(getNewFlowFiles(runner1, runner1StageThreeCheckpoint).isEmpty(),
+ "Runner 1 should receive data in stage three");
+ assertFalse(runner3.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS).isEmpty(),
+ "Runner 3 should receive data in stage three");
+ assertFalse(runner4.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS).isEmpty(),
+ "Runner 4 should receive data in stage three");
}
@Test
- void testSessionRollback() throws InterruptedException {
- streamClient.createStream(1);
+ void testKplAggregatedRecordsFlowFilePerRecord() throws Exception {
+ final String streamName = "kpl-per-record-test";
+ createStream(streamName);
- // Initialize the processor.
- runner.run(1, false, true);
+ publishAggregatedRecord(streamName, "agg-pk-1",
+ List.of("pk-a", "pk-b"),
+ List.of("{\"id\":1,\"name\":\"Alice\"}", "{\"id\":2,\"name\":\"Bob\"}", "{\"id\":3,\"name\":\"Charlie\"}"),
+ List.of(0, 1, 0));
- final ConsumeKinesis processor = (ConsumeKinesis) runner.getProcessor();
+ runner.setProperty(ConsumeKinesis.STREAM_NAME, streamName);
+ runner.setProperty(ConsumeKinesis.PROCESSING_STRATEGY, "FLOW_FILE");
+ runUntilOutput(runner);
- final String firstMessage = "Initial-Rollback-Message";
- streamClient.putRecord("key", firstMessage);
+ final List flowFiles = runner.getFlowFilesForRelationship(ConsumeKinesis.REL_SUCCESS);
+ assertEquals(3, flowFiles.size(), "Each sub-record in the aggregate should produce its own FlowFile");
- // First attempt with a failing session - should rollback.
- while (true) {
- final MockProcessSession failingSession = createFailingSession(processor);
- try {
- processor.onTrigger(runner.getProcessContext(), failingSession);
- } catch (final FlowFileHandlingException ignored) {
- failingSession.assertAllFlowFilesTransferred(REL_SUCCESS, 0);
- break; // Expected rollback occurred
- }
- Thread.sleep(1000);
+ final Set