awelless commented on code in PR #10077:
URL: https://github.com/apache/nifi/pull/10077#discussion_r2358168582


##########
nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/MemoryBoundRecordBuffer.java:
##########
@@ -0,0 +1,673 @@
+/*
+ * 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);
+    private final ConcurrentMap<ShardBufferId, ShardBuffer> shardBuffers = new 
ConcurrentHashMap<>();
+
+    /**
+     * A queue with ids shard buffers available for leasing.
+     * <p>
+     * Note: when a buffer is invalidated its id is NOT removed from the queue 
immediately.
+     */
+    private final Queue<ShardBufferId> 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<KinesisClientRecord> 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);
+        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));
+        }
+    }
+
+    @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.get(bufferId);
+        if (buffer == null) {
+            logger.debug("Buffer with id {} not found. Cannot shutdown shard 
consumption", bufferId);
+            return;
+        }
+
+        logger.debug("Shutting down the buffer {}. Checkpointing last consumed 
record", bufferId);
+        buffer.shutdownBuffer(checkpointer);
+
+        logger.debug("Removing buffer with id {} after successful last 
consumed record checkpoint", bufferId);
+        shardBuffers.remove(bufferId);
+    }
+
+    @Override
+    public void consumerLeaseLost(final ShardBufferId bufferId) {
+        final ShardBuffer buffer = shardBuffers.remove(bufferId);
+
+        logger.debug("Lease lost for buffer {}: Invalidating", bufferId);
+
+        if (buffer != null) {
+            final Collection<RecordBatch> invalidatedBatches = 
buffer.invalidate();
+            memoryTracker.freeMemory(invalidatedBatches);
+        }
+    }
+
+    @Override
+    public Optional<ShardBufferLease> acquireBufferLease() {
+        final Set<ShardBufferId> 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 StandardShardBufferLease(bufferId));
+            }
+        }
+    }
+
+    @Override
+    public List<KinesisClientRecord> consumeRecords(final ShardBufferLease 
lease) {
+        if (!(lease instanceof StandardShardBufferLease standardLease)) {
+            throw new IllegalArgumentException("Unexpected lease type: " + 
lease.getClass().getName());
+        }
+
+        if (standardLease.returnedToPool.get()) {
+            logger.warn("Attempting to consume records from a buffer that was 
already returned to the pool. Ignoring.");
+            return emptyList();
+        }
+
+        final ShardBufferId bufferId = standardLease.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 ShardBufferLease lease) {
+        if (!(lease instanceof StandardShardBufferLease standardLease)) {

Review Comment:
   The instance of the buffer accepts only it's own lease instances. I don't 
want to expose lease internal api (like `bufferId` and `returnedToPool`) in the 
interface.
   
   To avoid casting, we can make the lease class as a generic, but then in the 
client code a specific lease class must be specified. I'm not happy with 
requiring a particular lease class in `ConsumeKinesis`, but this approach is 
more type safe for sure.
   [Example](https://gist.github.com/awelless/dfe3e7a844ef7601392f2b5dd1d634ed)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to