diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
index 4634f041e1..34b9a8144b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java
@@ -16,21 +16,6 @@
*/
package org.apache.nifi.processors.standard;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.regex.Pattern;
-
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SideEffectFree;
@@ -42,7 +27,6 @@ import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
@@ -59,6 +43,21 @@ import org.apache.nifi.util.timebuffer.LongEntityAccess;
import org.apache.nifi.util.timebuffer.TimedBuffer;
import org.apache.nifi.util.timebuffer.TimestampedLong;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
@SideEffectFree
@TriggerSerially
@InputRequirement(Requirement.INPUT_REQUIRED)
@@ -71,30 +70,58 @@ public class ControlRate extends AbstractProcessor {
public static final String DATA_RATE = "data rate";
public static final String FLOWFILE_RATE = "flowfile count";
public static final String ATTRIBUTE_RATE = "attribute value";
+ public static final String DATA_OR_FLOWFILE_RATE = "data rate or flowfile count";
+
public static final AllowableValue DATA_RATE_VALUE = new AllowableValue(DATA_RATE, DATA_RATE,
"Rate is controlled by counting bytes transferred per time duration.");
public static final AllowableValue FLOWFILE_RATE_VALUE = new AllowableValue(FLOWFILE_RATE, FLOWFILE_RATE,
- "Rate is controlled by counting flowfiles transferred per time duration");
+ "Rate is controlled by counting FlowFiles transferred per time duration");
public static final AllowableValue ATTRIBUTE_RATE_VALUE = new AllowableValue(ATTRIBUTE_RATE, ATTRIBUTE_RATE,
"Rate is controlled by accumulating the value of a specified attribute that is transferred per time duration");
+ public static final AllowableValue DATA_OR_FLOWFILE_RATE_VALUE = new AllowableValue(DATA_OR_FLOWFILE_RATE, DATA_OR_FLOWFILE_RATE,
+ "Rate is controlled by counting bytes and FlowFiles transferred per time duration; if either threshold is met, throttling is enforced");
// based on testing to balance commits and 10,000 FF swap limit
public static final int MAX_FLOW_FILES_PER_BATCH = 1000;
+ private static final long DEFAULT_ACCRUAL_COUNT = -1L;
public static final PropertyDescriptor RATE_CONTROL_CRITERIA = new PropertyDescriptor.Builder()
.name("Rate Control Criteria")
+ .displayName("Rate Control Criteria")
.description("Indicates the criteria that is used to control the throughput rate. Changing this value resets the rate counters.")
.required(true)
- .allowableValues(DATA_RATE_VALUE, FLOWFILE_RATE_VALUE, ATTRIBUTE_RATE_VALUE)
+ .allowableValues(DATA_RATE_VALUE, FLOWFILE_RATE_VALUE, ATTRIBUTE_RATE_VALUE, DATA_OR_FLOWFILE_RATE_VALUE)
.defaultValue(DATA_RATE)
.build();
public static final PropertyDescriptor MAX_RATE = new PropertyDescriptor.Builder()
.name("Maximum Rate")
+ .displayName("Maximum Rate")
.description("The maximum rate at which data should pass through this processor. The format of this property is expected to be a "
+ "positive integer, or a Data Size (such as '1 MB') if Rate Control Criteria is set to 'data rate'.")
- .required(true)
+ .required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) // validated in customValidate b/c dependent on Rate Control Criteria
+ .dependsOn(RATE_CONTROL_CRITERIA, DATA_RATE_VALUE, FLOWFILE_RATE_VALUE, ATTRIBUTE_RATE_VALUE)
.build();
+ public static final PropertyDescriptor MAX_DATA_RATE = new PropertyDescriptor.Builder()
+ .name("Maximum Data Rate")
+ .displayName("Maximum Data Rate")
+ .description("The maximum rate at which data should pass through this processor. The format of this property is expected to be a "
+ + "Data Size (such as '1 MB') representing bytes per Time Duration.")
+ .required(false)
+ .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+ .dependsOn(RATE_CONTROL_CRITERIA, DATA_OR_FLOWFILE_RATE)
+ .build();
+
+ public static final PropertyDescriptor MAX_COUNT_RATE = new PropertyDescriptor.Builder()
+ .name("Maximum FlowFile Rate")
+ .displayName("Maximum FlowFile Rate")
+ .description("The maximum rate at which FlowFiles should pass through this processor. The format of this property is expected to be a "
+ + "positive integer representing FlowFiles count per Time Duration")
+ .required(false)
+ .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+ .dependsOn(RATE_CONTROL_CRITERIA, DATA_OR_FLOWFILE_RATE)
+ .build();
+
public static final PropertyDescriptor RATE_CONTROL_ATTRIBUTE_NAME = new PropertyDescriptor.Builder()
.name("Rate Controlled Attribute")
.description("The name of an attribute whose values build toward the rate limit if Rate Control Criteria is set to 'attribute value'. "
@@ -103,6 +130,7 @@ public class ControlRate extends AbstractProcessor {
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
+ .dependsOn(RATE_CONTROL_CRITERIA, ATTRIBUTE_RATE)
.build();
public static final PropertyDescriptor TIME_PERIOD = new PropertyDescriptor.Builder()
.name("Time Duration")
@@ -135,11 +163,13 @@ public class ControlRate extends AbstractProcessor {
private List This processor throttles throughput of FlowFiles based on a configured rate. The rate can be specified as either a direct data rate (bytes per time period), or by
+ counting FlowFiles or a specific attribute value. In all cases, the time period for measurement is specified in the Time Duration property.
+ The processor operates in one of four available modes. The mode is determined by the Rate Control Criteria property.
+
+
+
+
+
+ Mode
+ Description
+
+
+ Data Rate
+ The FlowFile content size is accumulated for all FlowFiles passing through this processor. FlowFiles are throttled to ensure a maximum overall data rate (bytes per time period)
+ is not exceeded. The Maximum Rate property specifies the maximum bytes allowed per Time Duration.
+
+
+ FlowFile Count
+ FlowFiles are counted regardless of content size. No more than the specified number of FlowFiles pass through this processor in the given Time Duration. The Maximum Rate property
+ specifies the maximum number of FlowFiles allowed per Time Duration.
+
+
+ Attribute Value
+ The value of an attribute is accumulated to determine overall rate. The Rate Controlled Attribute property specifies the attribute whose value will be accumulated. The value of
+ the specified attribute is expected to be an integer. This mode is independent of overall FlowFile size and count.
+
+
+ Data Rate or FlowFile Count
+ This mode provides a combination of Data Rate and FlowFile Count. Both rates are accumulated and FlowFiles are throttled if either rate is exceeded. Both Maximum Data Rate and
+ Maximum FlowFile Rate properties must be specified to determine content size and FlowFile count per Time Duration.
+
If the Grouping Attribute property is specified, all rates are accumulated separately for unique values of the specified attribute. For example, assume Grouping Attribute property is + specified and the its value is "city". All FlowFiles containing a "city" attribute with value "Albuquerque" will have an accumulated rate calculated. A separate rate will be calculated + for all FlowFiles containing a "city" attribute with a value "Boston". In other words, separate rate calculations will be accumulated for all unique values of the Grouping Attribute. +
+ + + + diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestControlRate.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestControlRate.java index fbecd767c4..0b68022622 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestControlRate.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestControlRate.java @@ -31,6 +31,8 @@ import org.junit.jupiter.api.Test; public class TestControlRate { + private static final long ONE_SEC_PLUS = 1010L; + @Test public void testLimitExceededThenOtherLimitNotExceeded() { // If we have flowfiles queued that have different values for the "Rate Controlled Attribute" @@ -84,7 +86,7 @@ public class TestControlRate { runner.assertQueueNotEmpty(); // we have sent 3 files and after 1 second, we should be able to send the 4th - Thread.sleep(1100L); + Thread.sleep(ONE_SEC_PLUS); runner.run(); runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 1); runner.assertQueueEmpty(); @@ -116,7 +118,7 @@ public class TestControlRate { runner.assertQueueNotEmpty(); // we have sent 2 files per group and after 1 second, we should be able to send the remaining 1 file per group - Thread.sleep(1100L); + Thread.sleep(ONE_SEC_PLUS); runner.run(2); runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 2); runner.assertQueueEmpty(); @@ -145,7 +147,7 @@ public class TestControlRate { runner.assertQueueNotEmpty(); // we have sent 20 bytes and after 1 second, we should be able to send 20 more - Thread.sleep(1100L); + Thread.sleep(ONE_SEC_PLUS); runner.run(2, false); runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 2); runner.assertQueueEmpty(); @@ -192,6 +194,28 @@ public class TestControlRate { runner.assertQueueEmpty(); } + @Test + public void testAttributeDoesNotExist() throws InterruptedException { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.ATTRIBUTE_RATE); + runner.setProperty(ControlRate.RATE_CONTROL_ATTRIBUTE_NAME, "no.such.attribute"); + runner.setProperty(ControlRate.MAX_RATE, "20000"); + runner.setProperty(ControlRate.TIME_PERIOD, "1 sec"); + + createFlowFile(runner, 1000); + createFlowFile(runner, 3000); + createFlowFile(runner, 5000); + createFlowFile(runner, 20000); + createFlowFile(runner, 1000); + + runner.run(5, false); + + // all flowfiles transfer to failure since throttling attribute is not present + runner.assertAllFlowFilesTransferred(ControlRate.REL_FAILURE, 5); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 0); + runner.assertQueueEmpty(); + } + @Test public void testBadAttributeRate() { final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); @@ -211,7 +235,7 @@ public class TestControlRate { } @Test - public void testBatchLimit() throws InterruptedException { + public void testBatchLimit() { final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.FLOWFILE_RATE); runner.setProperty(ControlRate.MAX_RATE, "5555"); @@ -240,7 +264,7 @@ public class TestControlRate { } @Test - public void testNonExistingGroupAttribute() throws InterruptedException { + public void testNonExistingGroupAttribute() { final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.FLOWFILE_RATE); runner.setProperty(ControlRate.MAX_RATE, "2"); @@ -258,10 +282,245 @@ public class TestControlRate { runner.assertQueueEmpty(); } + @Test + public void testIncreaseDataRate() throws InterruptedException { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.DATA_RATE); + runner.setProperty(ControlRate.MAX_RATE, "11 B"); + runner.setProperty(ControlRate.TIME_PERIOD, "1 sec"); + + runner.enqueue("test data 1"); + runner.enqueue("test data 2"); + runner.enqueue("test data 3"); + runner.enqueue("test data 4"); + runner.enqueue("test data 5"); + runner.enqueue("test data 6"); + + runner.run(7, true); + + runner.assertTransferCount(ControlRate.REL_SUCCESS, 1); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + + // Increase rate after stopping processor. Previous count should remain since we are still inside time period + runner.setProperty(ControlRate.MAX_RATE, "33 B"); + runner.run(7, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 3); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + + // after 1 second, we should be able to send the up to 3 more flowfiles + Thread.sleep(ONE_SEC_PLUS); + runner.run(7, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 6); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueEmpty(); + } + + @Test + public void testIncreaseFlowFileRate() throws InterruptedException { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.FLOWFILE_RATE); + runner.setProperty(ControlRate.MAX_RATE, "1"); + runner.setProperty(ControlRate.TIME_PERIOD, "1 sec"); + + runner.enqueue("test data 1"); + runner.enqueue("test data 2"); + runner.enqueue("test data 3"); + runner.enqueue("test data 4"); + runner.enqueue("test data 5"); + runner.enqueue("test data 6"); + + runner.run(7, true); + + runner.assertTransferCount(ControlRate.REL_SUCCESS, 1); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + + // Increase rate after stopping processor. Previous count should remain since we are still inside time period + runner.setProperty(ControlRate.MAX_RATE, "3"); + runner.run(7, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 3); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + + // after 1 second, we should be able to send the up to 3 more flowfiles + Thread.sleep(ONE_SEC_PLUS); + runner.run(7, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 6); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueEmpty(); + } + + @Test + public void testDataOrFlowFileCountLimitedByBytes() throws InterruptedException { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.DATA_OR_FLOWFILE_RATE); + runner.setProperty(ControlRate.TIME_PERIOD, "1 sec"); + // Data rate will throttle before FlowFile count + runner.setProperty(ControlRate.MAX_DATA_RATE, "22 B"); + runner.setProperty(ControlRate.MAX_COUNT_RATE, "3"); + + runner.enqueue("test data 1"); + runner.enqueue("test data 2"); + runner.enqueue("test data 3"); + + runner.run(4, false); + + runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 2); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + runner.clearTransferState(); + + runner.run(4, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 0); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + // we have sent 22 bytes and after 1 second, we should be able to send 22 more + Thread.sleep(ONE_SEC_PLUS); + runner.run(4, false); + runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 1); + runner.assertQueueEmpty(); + } + + @Test + public void testDataOrFlowFileCountLimitedByCount() throws InterruptedException { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.DATA_OR_FLOWFILE_RATE); + runner.setProperty(ControlRate.TIME_PERIOD, "1 sec"); + // FlowFile count rate will throttle before data rate + runner.setProperty(ControlRate.MAX_DATA_RATE, "44 B"); // greater than all flowfiles to be queued + runner.setProperty(ControlRate.MAX_COUNT_RATE, "1"); // limit to 1 flowfile per second + + runner.enqueue("test data 1"); + runner.enqueue("test data 2"); + runner.enqueue("test data 3"); + + runner.run(4, false); + + runner.assertTransferCount(ControlRate.REL_SUCCESS, 1); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + + // we have sent 1 flowfile and after 1 second, we should be able to send 1 more + Thread.sleep(ONE_SEC_PLUS); + runner.run(4, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 2); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + + // we have sent 2 flowfile and after 1 second, we should be able to send 1 more + Thread.sleep(ONE_SEC_PLUS); + runner.run(4, false); + runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 3); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueEmpty(); + } + + @Test + public void testDataOrFlowFileCountLimitedByBytesThenCount() throws InterruptedException { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.DATA_OR_FLOWFILE_RATE); + runner.setProperty(ControlRate.TIME_PERIOD, "1 sec"); + // Data rate will throttle before FlowFile count + runner.setProperty(ControlRate.MAX_DATA_RATE, "22 B"); + runner.setProperty(ControlRate.MAX_COUNT_RATE, "5"); + + runner.enqueue("test data 1"); + runner.enqueue("test data 2"); + runner.enqueue("test data 3"); + runner.enqueue("4"); + runner.enqueue("5"); + runner.enqueue("6"); + runner.enqueue("7"); + runner.enqueue("8"); + + runner.run(10, false); + + runner.assertTransferCount(ControlRate.REL_SUCCESS, 2); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + runner.clearTransferState(); + + // we have sent 2 flowfile and after 1 second, we should be able to send more, now limited by flowfile count + Thread.sleep(ONE_SEC_PLUS); + runner.run(10, false); + runner.assertTransferCount(ControlRate.REL_SUCCESS, 5); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueNotEmpty(); + runner.clearTransferState(); + + // after 1 second, we should be able to send the remaining flowfile + Thread.sleep(ONE_SEC_PLUS); + runner.run(10, false); + runner.assertAllFlowFilesTransferred(ControlRate.REL_SUCCESS, 1); + runner.assertTransferCount(ControlRate.REL_FAILURE, 0); + runner.assertQueueEmpty(); + } + + @Test + public void testValidate() { + final TestRunner runner = TestRunners.newTestRunner(new ControlRate()); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.DATA_RATE); + runner.assertNotValid(); // MAX_RATE is not set + runner.setProperty(ControlRate.MAX_RATE, "1"); + runner.assertNotValid(); // MAX_RATE is not a byte size + runner.setProperty(ControlRate.MAX_RATE, "1 MB"); + runner.assertValid(); + runner.setProperty(ControlRate.MAX_DATA_RATE, "1 MB"); + runner.assertValid(); // MAX_DATA_RATE is ignored + runner.removeProperty(ControlRate.MAX_RATE); + runner.assertNotValid(); // MAX_RATE is a required property for this rate control criteria + + runner.clearProperties(); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.FLOWFILE_RATE); + runner.assertNotValid(); // MAX_RATE is not set + runner.setProperty(ControlRate.MAX_RATE, "1 MB"); + runner.assertNotValid(); // MAX_RATE is not an integer + runner.setProperty(ControlRate.MAX_RATE, "1"); + runner.assertValid(); + runner.setProperty(ControlRate.MAX_COUNT_RATE, "1"); + runner.assertValid(); // MAX_COUNT_RATE is ignored + runner.removeProperty(ControlRate.MAX_RATE); + runner.assertNotValid(); // MAX_RATE is a required property for this rate control criteria + + runner.clearProperties(); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.ATTRIBUTE_RATE); + runner.setProperty(ControlRate.RATE_CONTROL_ATTRIBUTE_NAME, "count"); + runner.assertNotValid(); // MAX_RATE is not set + runner.setProperty(ControlRate.MAX_RATE, "1 MB"); + runner.assertNotValid(); // MAX_RATE is not an integer + runner.setProperty(ControlRate.MAX_RATE, "1"); + runner.assertValid(); + runner.setProperty(ControlRate.MAX_COUNT_RATE, "1"); + runner.assertValid(); // MAX_COUNT_RATE is ignored + runner.removeProperty(ControlRate.MAX_RATE); + runner.assertNotValid();// MAX_RATE is a required property for this rate control criteria + runner.setProperty(ControlRate.MAX_RATE, "1"); + runner.removeProperty(ControlRate.RATE_CONTROL_ATTRIBUTE_NAME); + runner.assertNotValid();// RATE_CONTROL_ATTRIBUTE_NAME is a required property for this rate control criteria + + runner.clearProperties(); + runner.setProperty(ControlRate.RATE_CONTROL_CRITERIA, ControlRate.DATA_OR_FLOWFILE_RATE); + runner.setProperty(ControlRate.MAX_DATA_RATE, "1 MB"); + runner.setProperty(ControlRate.MAX_COUNT_RATE, "1"); + runner.setProperty(ControlRate.MAX_COUNT_RATE, "2"); + runner.assertValid(); // both MAX_DATA_RATE and MAX_COUNT_RATE are set + runner.removeProperty(ControlRate.MAX_COUNT_RATE); + runner.assertNotValid(); // MAX_COUNT_RATE is not set + runner.setProperty(ControlRate.MAX_COUNT_RATE, "1"); + runner.removeProperty(ControlRate.MAX_DATA_RATE); + runner.assertNotValid();// MAX_DATA_RATE is not set + runner.setProperty(ControlRate.MAX_DATA_RATE, "1 MB"); + runner.setProperty(ControlRate.MAX_RATE, "1 MB"); + runner.assertValid(); // MAX_RATE is ignored + } + private void createFlowFile(final TestRunner runner, final int value) { final Map