You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/11/09 21:28:44 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6506: NIFI-10243: allow ControlRate to throttle on combination of data rate or flowfile rate

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: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org