turcsanyip commented on code in PR #10538:
URL: https://github.com/apache/nifi/pull/10538#discussion_r2535604387


##########
nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -548,10 +652,34 @@ private KafkaConsumerService getConsumerService(final 
ProcessContext context) {
         }
 
         getLogger().info("No Kafka Consumer Service available; creating a new 
one. Active count: {}", activeCount);
-        final KafkaConnectionService connectionService = 
context.getProperty(CONNECTION_SERVICE).asControllerService(KafkaConnectionService.class);
         return connectionService.getConsumerService(pollingContext);
     }
 
+    public int getPartitionCount(final KafkaConnectionService 
connectionService) {

Review Comment:
   Minor:
   ```suggestion
       private int getPartitionCount(final KafkaConnectionService 
connectionService) {
   ```



##########
nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -540,6 +603,47 @@ private KafkaConsumerService getConsumerService(final 
ProcessContext context) {
             return consumerService;
         }
 
+        final KafkaConnectionService connectionService = 
context.getProperty(CONNECTION_SERVICE).asControllerService(KafkaConnectionService.class);
+
+        consumerServiceToPartitionedPollingContext.clear();
+
+        final boolean isExplicitPartitionMapping = 
ConsumerPartitionsUtil.isPartitionAssignmentExplicit(context.getAllProperties());
+
+        if (isExplicitPartitionMapping) {

Review Comment:
   The assignment of the partitions must happen exactly once after the 
processor is started.
   This code is called from `onTrigger()` which can cause concurrency issues 
when Concurrent Tasks > 1:
   
   1. Multiple threads are trying to initialize the consumers in parallel
   2. If there are configured more threads than partitions, then a thread can 
reinitalize the assignment if there is no available consumer in the pool
   
   Ideally, the assigment should happen in `onScheduled()`. Isn't it possible?
   Also, `consumerServiceToPartitionedPollingContext` should be cleared in 
`onStopped()`.
   



-- 
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