exceptionfactory commented on code in PR #6506:
URL: https://github.com/apache/nifi/pull/6506#discussion_r1018429196


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java:
##########
@@ -268,48 +336,67 @@ public void onTrigger(final ProcessContext context, final 
ProcessSession session
         final ComponentLog logger = getLogger();
         for (FlowFile flowFile : flowFiles) {
             // call this to capture potential error
-            final long accrualAmount = getFlowFileAccrual(flowFile);
-            if (accrualAmount < 0) {
-                logger.error("Routing {} to 'failure' due to missing or 
invalid attribute", new Object[]{flowFile});
+            if (!isAccrualPossible(flowFile)) {
+                logger.error("Routing {} to 'failure' due to missing or 
invalid attribute", flowFile);
                 session.transfer(flowFile, REL_FAILURE);
             } else {
-                logger.info("transferring {} to 'success'", new 
Object[]{flowFile});
+                logger.info("transferring {} to 'success'", flowFile);
                 session.transfer(flowFile, REL_SUCCESS);
             }
         }
     }
 
+    /*
+     * Determine if the accrual amount is valid for the type of throttle being 
applied. For example, if throttling based on
+     * flowfile attribute, the specified attribute must be present and must be 
a long integer.
+     */
+    private boolean isAccrualPossible(FlowFile flowFile) {
+        if (rateControlCriteria.equals(ATTRIBUTE_RATE)) {
+            final String attributeValue = 
flowFile.getAttribute(rateControlAttribute);
+            return attributeValue != null && 
POSITIVE_LONG_PATTERN.matcher(attributeValue).matches();
+        }
+        return true;
+    }
+
     /*
      * Determine the amount this FlowFile will incur against the maximum 
allowed rate.
-     * If the value returned is negative then the flowfile given is missing 
the required attribute
-     * or the attribute has an invalid value for accrual.
+     * This is applicable to data size accrual only
      */
-    private long getFlowFileAccrual(FlowFile flowFile) {
-        long rateValue;
-        switch (rateControlCriteria) {
-            case DATA_RATE:
-                rateValue = flowFile.getSize();
-                break;
-            case FLOWFILE_RATE:
-                rateValue = 1;
-                break;
-            case ATTRIBUTE_RATE:
-                final String attributeValue = 
flowFile.getAttribute(rateControlAttribute);
-                if (attributeValue == null) {
-                    return -1L;
-                }
+    private long getDataSizeAccrual(FlowFile flowFile) {
+        return flowFile.getSize();
+    }
 
-                if (!POSITIVE_LONG_PATTERN.matcher(attributeValue).matches()) {
-                    return -1L;
-                }
-                rateValue = Long.parseLong(attributeValue);
-                break;
-            default:
-                throw new AssertionError("<Rate Control Criteria> property set 
to illegal value of " + rateControlCriteria);
+    /*
+     * Determine the amount this FlowFile will incur against the maximum 
allowed rate.
+     * This is applicable to counting accruals, flowfiles or attributes
+     */
+    private long getCountAccrual(FlowFile flowFile) {
+        long rateValue = -1L;

Review Comment:
   It would be helpful to define a `private static final` value for the default 
value of `-1` and reuse that in multiple places.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java:
##########
@@ -408,34 +498,59 @@ public FlowFileFilterResult filter(FlowFile flowFile) {
                 groupName = DEFAULT_GROUP_ATTRIBUTE;
             }
 
-            Throttle throttle = throttleMap.get(groupName);
-            if (throttle == null) {
-                throttle = new Throttle(timePeriodSeconds, TimeUnit.SECONDS, 
getLogger());
+            Throttle dataThrottle = dataThrottleMap.get(groupName);
+            Throttle countThrottle = countThrottleMap.get(groupName);
 
-                final long newRate;
-                if 
(DataUnit.DATA_SIZE_PATTERN.matcher(maximumRateStr).matches()) {
-                    newRate = DataUnit.parseDataSize(maximumRateStr, 
DataUnit.B).longValue();
-                } else {
-                    newRate = Long.parseLong(maximumRateStr);
+            boolean dataThrottlingActive = false;
+            if (dataThrottleRequired()) {
+                if (dataThrottle == null) {
+                    dataThrottle = new Throttle(timePeriodSeconds, 
TimeUnit.SECONDS, getLogger());
+                    
dataThrottle.setMaxRate(DataUnit.parseDataSize(maximumRateStr, 
DataUnit.B).longValue());
+                    dataThrottleMap.put(groupName, dataThrottle);
                 }
-                throttle.setMaxRate(newRate);
 
-                throttleMap.put(groupName, throttle);
+                dataThrottle.lock();
+                try {
+                    if (dataThrottle.tryAdd(getDataSizeAccrual(flowFile))) {
+                        flowFilesInBatch += 1;
+                        if (flowFilesInBatch>= flowFilesPerBatch) {
+                            flowFilesInBatch = 0;
+                            return FlowFileFilterResult.ACCEPT_AND_TERMINATE;
+                        } else {
+                            // only accept flowfile if additional count 
throttle does not need to run
+                            if (!countThrottleRequired()) {
+                                return 
FlowFileFilterResult.ACCEPT_AND_CONTINUE;
+                            }
+                        }
+                    } else {
+                        dataThrottlingActive = true;
+                    }
+                } finally {
+                    dataThrottle.unlock();
+                }
             }
 
-            throttle.lock();
-            try {
-                if (throttle.tryAdd(accrual)) {
-                    flowFilesInBatch += 1;
-                    if (flowFilesInBatch>= flowFilesPerBatch) {
-                        flowFilesInBatch = 0;
-                        return FlowFileFilterResult.ACCEPT_AND_TERMINATE;
-                    } else {
-                        return FlowFileFilterResult.ACCEPT_AND_CONTINUE;
+            // continue processing count throttle only if required and if data 
throttle is not already limiting flowfiles
+            if (countThrottleRequired() && !dataThrottlingActive) {
+                if (countThrottle == null) {
+                    countThrottle = new Throttle(timePeriodSeconds, 
TimeUnit.SECONDS, getLogger());
+                    
countThrottle.setMaxRate(Long.parseLong(maximumCountRateStr));
+                    countThrottleMap.put(groupName, countThrottle);
+                }
+                countThrottle.lock();
+                try {
+                    if (countThrottle.tryAdd(getCountAccrual(flowFile))) {
+                        flowFilesInBatch += 1;

Review Comment:
   ```suggestion
                           flowFilesInBatch++;
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java:
##########
@@ -408,34 +498,59 @@ public FlowFileFilterResult filter(FlowFile flowFile) {
                 groupName = DEFAULT_GROUP_ATTRIBUTE;
             }
 
-            Throttle throttle = throttleMap.get(groupName);
-            if (throttle == null) {
-                throttle = new Throttle(timePeriodSeconds, TimeUnit.SECONDS, 
getLogger());
+            Throttle dataThrottle = dataThrottleMap.get(groupName);
+            Throttle countThrottle = countThrottleMap.get(groupName);
 
-                final long newRate;
-                if 
(DataUnit.DATA_SIZE_PATTERN.matcher(maximumRateStr).matches()) {
-                    newRate = DataUnit.parseDataSize(maximumRateStr, 
DataUnit.B).longValue();
-                } else {
-                    newRate = Long.parseLong(maximumRateStr);
+            boolean dataThrottlingActive = false;
+            if (dataThrottleRequired()) {
+                if (dataThrottle == null) {
+                    dataThrottle = new Throttle(timePeriodSeconds, 
TimeUnit.SECONDS, getLogger());
+                    
dataThrottle.setMaxRate(DataUnit.parseDataSize(maximumRateStr, 
DataUnit.B).longValue());
+                    dataThrottleMap.put(groupName, dataThrottle);
                 }
-                throttle.setMaxRate(newRate);
 
-                throttleMap.put(groupName, throttle);
+                dataThrottle.lock();
+                try {
+                    if (dataThrottle.tryAdd(getDataSizeAccrual(flowFile))) {
+                        flowFilesInBatch += 1;

Review Comment:
   ```suggestion
                           flowFilesInBatch++;
   ```



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