markap14 commented on a change in pull request #5267:
URL: https://github.com/apache/nifi/pull/5267#discussion_r737567192



##########
File path: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
##########
@@ -491,4 +508,35 @@ public boolean requiresAllDestinationsAvailable() {
             return false;
         }
     }
+
+    private class OverflowStrategy implements DistributionStrategy {
+
+        @Override
+        public Relationship mapToRelationship(final ProcessContext context, 
final FlowFile flowFile) {
+            final List<Relationship> relationshipList = 
DistributeLoad.this.weightedRelationshipListRef.get();
+            final int numRelationships = relationshipList.size();
+
+            boolean foundFreeRelationship = false;
+            Relationship relationship = null;
+            // Getting set of available relationships only once. This may miss 
a relationship that recently became available, but
+            // overall is more efficient than re-calling for every 
relationship evaluation
+            Set<Relationship> availableRelationships = 
context.getAvailableRelationships();
+
+            int weightedIndex = 0;
+            while (!foundFreeRelationship) {

Review comment:
       @markobean I don't believe your assertion there is accurate. "when this 
is the case, the processor is not scheduled." It can still be scheduled if 
there is more than one concurrent task, for example. That said, this case is 
handled below by checking if weightedIndex has reached a value of 
numRelationships

##########
File path: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
##########
@@ -68,19 +69,31 @@
         + "strategy, the default is to assign each destination a weighting of 
1 (evenly distributed). However, optional properties"
         + "can be added to the change this; adding a property with the name 
'5' and value '10' means that the relationship with name "
         + "'5' will be receive 10 FlowFiles in each iteration instead of 1.")
-@DynamicProperty(name = "The relationship name(positive number)", value = "The 
relationship Weight(positive number)", description = "adding a "
+@DynamicProperty(name = "The relationship name (positive number)", value = 
"The relationship Weight (positive number)", description = "Adding a "
         + "property with the name '5' and value '10' means that the 
relationship with name "
-        + "'5' will be receive 10 FlowFiles in each iteration instead of 1.")
+        + "'5' will receive 10 FlowFiles in each iteration instead of 1.")
 @DynamicRelationship(name = "A number 1..<Number Of Relationships>", 
description = "FlowFiles are sent to this relationship per the "
         + "<Distribution Strategy>")
 @WritesAttributes(
-        @WritesAttribute(attribute = "distribute.load.relationship", 
description = "The name of the specific relationship the flow file has been 
routed through")
+        @WritesAttribute(attribute = "distribute.load.relationship", 
description = "The name of the specific relationship the FlowFile has been 
routed through")
 )
 public class DistributeLoad extends AbstractProcessor {
 
-    public static final String STRATEGY_ROUND_ROBIN = "round robin";
-    public static final String STRATEGY_NEXT_AVAILABLE = "next available";
-    public static final String STRATEGY_LOAD_DISTRIBUTION_SERVICE = "load 
distribution service";
+    public static final String ROUND_ROBIN = "round robin";
+    public static final String NEXT_AVAILABLE = "next available";
+    public static final String LOAD_DISTRIBUTION_SERVICE = "load distribution 
service";
+    public static final String OVERFLOW = "overflow";
+
+    public static final AllowableValue STRATEGY_ROUND_ROBIN = new 
AllowableValue(ROUND_ROBIN, ROUND_ROBIN,
+            "Relationship selection is evenly distributed in a round robin 
fashion; all relationships must be available.");
+    public static final AllowableValue STRATEGY_NEXT_AVAILABLE = new 
AllowableValue(NEXT_AVAILABLE, NEXT_AVAILABLE,
+            "Relationship selection is distributed across all available 
relationships in order of their weight; at least one relationship must be 
available.");
+    public static final AllowableValue STRATEGY_LOAD_DISTRIBUTION_SERVICE = 
new AllowableValue(LOAD_DISTRIBUTION_SERVICE, LOAD_DISTRIBUTION_SERVICE,
+            "Relationship selection is distributed by supplied 
LoadDistributionService Controller Service; at least one relationship must be 
available.");
+    public static final AllowableValue STRATEGY_OVERFLOW = new 
AllowableValue(OVERFLOW, OVERFLOW,
+            "Relationship selection is the highest weighted relationship 
available without further distribution; at least one relationship must be 
available.");

Review comment:
       This is not an accurate description of the processor's behavior. 
FlowFiles are always routed to the first Relationship that is available - NOT 
the relationship with the highest weighting. For example, if I have 4 
relationships and add a property named "3" with a value of 5, then the "3" 
relationship is the highest weighting. But this is ignored, and the FlowFiles 
will go to relationship "1" as long as it is available. This seems like a 
reasonable behavior to me, but we need to make sure that the documentation 
matches the behavior of the processor.




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