Details

    • Type: New Feature
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.2.0
    • Component/s: None
    • Labels:
      None

      Description

      For some flows, it is imperative that the flow files are processed in a certain order. The PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order, but depending on error-handling, branching, and other flow designs, it is possible for flow files to get out-of-order.

      I propose an EnforceOrder processor, which would be single-threaded and have (at a minimum) the following properties:

      1) Order Attribute: This would be the name of a flow file attribute from which the current value will be retrieved.
      2) Initial Value: This property specifies an initial value for the order. The processor is stateful, however, so this property is only used when there is no entry in the state map for current value.

      The processor would store the Initial Value into the state map (if no state map entry exists), then for each incoming flow file, it checks the value in the Order Attribute against the current value. If the attribute value matches the current value, the flow file is transferred to the "success" relationship, and the current value is incremented in the state map. If the attribute value does not match the current value, the session will be rolled back.

      Using this processor, along with a PriorityAttributePrioritizer on the incoming connection, will allow for out-of-order flow files to have a sort of "barrier", thereby guaranteeing that flow files transferred to the "success" relationship are in the specified order.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on the issue:

          https://github.com/apache/nifi/pull/1496

          +1 LGTM, ran with the latest, full build and a number of use cases. All looks great, thanks very much! Merging to master

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on the issue: https://github.com/apache/nifi/pull/1496 +1 LGTM, ran with the latest, full build and a number of use cases. All looks great, thanks very much! Merging to master
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/nifi/pull/1496

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/nifi/pull/1496
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura
          [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ]

          NIFI-3414: Added EnforceOrder processor

          Use it with FirstInFirstOutPrioritizer, it can enforce original ordering
          of 'out-of-order' FlowFiles.

          nifi-mock is modified to support FlowFile assertion using Prioritizer.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporated review comments, added displayNames.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporate review comments:

          • Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml.
          • Changed default initial order from 1 to 0.
          • Fixed typos.
          • Use session.get(batchCount).

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          This closes #1496

          Show
          jira-bot ASF subversion and git services added a comment - Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ] NIFI-3414 : Added EnforceOrder processor Use it with FirstInFirstOutPrioritizer, it can enforce original ordering of 'out-of-order' FlowFiles. nifi-mock is modified to support FlowFile assertion using Prioritizer. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporated review comments, added displayNames. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporate review comments: Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml. Changed default initial order from 1 to 0. Fixed typos. Use session.get(batchCount). Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure. Signed-off-by: Matt Burgess <mattyb149@apache.org> This closes #1496
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura
          [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ]

          NIFI-3414: Added EnforceOrder processor

          Use it with FirstInFirstOutPrioritizer, it can enforce original ordering
          of 'out-of-order' FlowFiles.

          nifi-mock is modified to support FlowFile assertion using Prioritizer.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporated review comments, added displayNames.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporate review comments:

          • Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml.
          • Changed default initial order from 1 to 0.
          • Fixed typos.
          • Use session.get(batchCount).

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          This closes #1496

          Show
          jira-bot ASF subversion and git services added a comment - Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ] NIFI-3414 : Added EnforceOrder processor Use it with FirstInFirstOutPrioritizer, it can enforce original ordering of 'out-of-order' FlowFiles. nifi-mock is modified to support FlowFile assertion using Prioritizer. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporated review comments, added displayNames. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporate review comments: Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml. Changed default initial order from 1 to 0. Fixed typos. Use session.get(batchCount). Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure. Signed-off-by: Matt Burgess <mattyb149@apache.org> This closes #1496
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura
          [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ]

          NIFI-3414: Added EnforceOrder processor

          Use it with FirstInFirstOutPrioritizer, it can enforce original ordering
          of 'out-of-order' FlowFiles.

          nifi-mock is modified to support FlowFile assertion using Prioritizer.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporated review comments, added displayNames.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporate review comments:

          • Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml.
          • Changed default initial order from 1 to 0.
          • Fixed typos.
          • Use session.get(batchCount).

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          This closes #1496

          Show
          jira-bot ASF subversion and git services added a comment - Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ] NIFI-3414 : Added EnforceOrder processor Use it with FirstInFirstOutPrioritizer, it can enforce original ordering of 'out-of-order' FlowFiles. nifi-mock is modified to support FlowFile assertion using Prioritizer. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporated review comments, added displayNames. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporate review comments: Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml. Changed default initial order from 1 to 0. Fixed typos. Use session.get(batchCount). Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure. Signed-off-by: Matt Burgess <mattyb149@apache.org> This closes #1496
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura
          [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ]

          NIFI-3414: Added EnforceOrder processor

          Use it with FirstInFirstOutPrioritizer, it can enforce original ordering
          of 'out-of-order' FlowFiles.

          nifi-mock is modified to support FlowFile assertion using Prioritizer.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporated review comments, added displayNames.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          Incorporate review comments:

          • Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml.
          • Changed default initial order from 1 to 0.
          • Fixed typos.
          • Use session.get(batchCount).

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          NIFI-3414: Added EnforceOrder processor

          When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure.

          Signed-off-by: Matt Burgess <mattyb149@apache.org>

          This closes #1496

          Show
          jira-bot ASF subversion and git services added a comment - Commit 9583ca99c1e4b2d3c43511939a2a148d20f8e1ac in nifi's branch refs/heads/master from Koji Kawamura [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9583ca9 ] NIFI-3414 : Added EnforceOrder processor Use it with FirstInFirstOutPrioritizer, it can enforce original ordering of 'out-of-order' FlowFiles. nifi-mock is modified to support FlowFile assertion using Prioritizer. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporated review comments, added displayNames. Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor Incorporate review comments: Moved nifi-standard-prioritizers dependency to top level nifi/pom.xml. Changed default initial order from 1 to 0. Fixed typos. Use session.get(batchCount). Signed-off-by: Matt Burgess <mattyb149@apache.org> NIFI-3414 : Added EnforceOrder processor When a FlowFile is transferred to success, remove attributes previously set when it was transferred to wait or failure. Signed-off-by: Matt Burgess <mattyb149@apache.org> This closes #1496
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110668232

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,543 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("0")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that have larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + List<FlowFile> flowFiles = session.get(batchCount);
          + if (flowFiles == null || flowFiles.isEmpty())

          { + return; + }

          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (FlowFile flowFile : flowFiles) {
          + oc.setFlowFile(flowFile);
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValid();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + // Parse it to check if it is a valid integer. + Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValid() {
          + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>());
          +
          + final FlowFile validFlowFile;
          + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT)))

          { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + }

          else

          { + validFlowFile = flowFile; + }

          +
          + groupedFlowFiles.add(validFlowFile);
          + }
          +
          + private void transferFlowFiles() {
          + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry ->

          { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + }

          ).forEach(entry -> {
          + // Check current state.
          + final String groupId = entry.getKey();
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder));
          + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder);
          + final List<FlowFile> groupedFlowFiles = entry.getValue();
          + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId));
          +
          + groupedFlowFiles.forEach(f -> {
          + final Integer order = getOrder.apply(f);
          + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr));
          +
          + if (order == targetOrder.get()) {
          + transferResult(f, REL_SUCCESS, null, null);
          + if (!isMaxOrder)

          { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + }

          +
          + } else if (order > targetOrder.get()) {
          +
          + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis)

          { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + }

          else

          { + transferResult(f, REL_WAIT, null, targetOrder.get()); + }

          +
          + } else {
          + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get());
          + logger.warn(msg + ". {}", new Object[]

          {f}

          );
          + transferResult(f, REL_SKIPPED, msg, targetOrder.get());
          + }
          +
          + });
          +
          + if (previousTargetOrder != targetOrder.get())

          { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          + });
          + }
          +
          + private void transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) {
          + final Map<String, String> attributes = new HashMap<>();
          + attributes.put(ATTR_RESULT, result.getName());
          + if (expectedOrder != null) {
          — End diff –

          Pushed one more commit to address this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110668232 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,543 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("0") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that have larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + List<FlowFile> flowFiles = session.get(batchCount); + if (flowFiles == null || flowFiles.isEmpty()) { + return; + } + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (FlowFile flowFile : flowFiles) { + oc.setFlowFile(flowFile); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValid(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + // Parse it to check if it is a valid integer. + Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValid() { + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>()); + + final FlowFile validFlowFile; + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT))) { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + } else { + validFlowFile = flowFile; + } + + groupedFlowFiles.add(validFlowFile); + } + + private void transferFlowFiles() { + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry -> { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + } ).forEach(entry -> { + // Check current state. + final String groupId = entry.getKey(); + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder)); + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder); + final List<FlowFile> groupedFlowFiles = entry.getValue(); + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId)); + + groupedFlowFiles.forEach(f -> { + final Integer order = getOrder.apply(f); + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr)); + + if (order == targetOrder.get()) { + transferResult(f, REL_SUCCESS, null, null); + if (!isMaxOrder) { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + } + + } else if (order > targetOrder.get()) { + + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis) { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + } else { + transferResult(f, REL_WAIT, null, targetOrder.get()); + } + + } else { + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get()); + logger.warn(msg + ". {}", new Object[] {f} ); + transferResult(f, REL_SKIPPED, msg, targetOrder.get()); + } + + }); + + if (previousTargetOrder != targetOrder.get()) { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + }); + } + + private void transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) { + final Map<String, String> attributes = new HashMap<>(); + attributes.put(ATTR_RESULT, result.getName()); + if (expectedOrder != null) { — End diff – Pushed one more commit to address this.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110667002

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,543 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("0")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that have larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + List<FlowFile> flowFiles = session.get(batchCount);
          + if (flowFiles == null || flowFiles.isEmpty())

          { + return; + }

          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (FlowFile flowFile : flowFiles) {
          + oc.setFlowFile(flowFile);
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValid();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + // Parse it to check if it is a valid integer. + Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValid() {
          + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>());
          +
          + final FlowFile validFlowFile;
          + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT)))

          { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + }

          else

          { + validFlowFile = flowFile; + }

          +
          + groupedFlowFiles.add(validFlowFile);
          + }
          +
          + private void transferFlowFiles() {
          + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry ->

          { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + }

          ).forEach(entry -> {
          + // Check current state.
          + final String groupId = entry.getKey();
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder));
          + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder);
          + final List<FlowFile> groupedFlowFiles = entry.getValue();
          + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId));
          +
          + groupedFlowFiles.forEach(f -> {
          + final Integer order = getOrder.apply(f);
          + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr));
          +
          + if (order == targetOrder.get()) {
          + transferResult(f, REL_SUCCESS, null, null);
          + if (!isMaxOrder)

          { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + }

          +
          + } else if (order > targetOrder.get()) {
          +
          + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis)

          { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + }

          else

          { + transferResult(f, REL_WAIT, null, targetOrder.get()); + }

          +
          + } else {
          + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get());
          + logger.warn(msg + ". {}", new Object[]

          {f}

          );
          + transferResult(f, REL_SKIPPED, msg, targetOrder.get());
          + }
          +
          + });
          +
          + if (previousTargetOrder != targetOrder.get())

          { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          + });
          + }
          +
          + private void transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) {
          + final Map<String, String> attributes = new HashMap<>();
          + attributes.put(ATTR_RESULT, result.getName());
          + if (expectedOrder != null) {
          — End diff –

          Thanks for catching this. I'll clear those attribute when it transferred to 'success'.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110667002 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,543 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("0") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that have larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + List<FlowFile> flowFiles = session.get(batchCount); + if (flowFiles == null || flowFiles.isEmpty()) { + return; + } + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (FlowFile flowFile : flowFiles) { + oc.setFlowFile(flowFile); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValid(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + // Parse it to check if it is a valid integer. + Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValid() { + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>()); + + final FlowFile validFlowFile; + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT))) { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + } else { + validFlowFile = flowFile; + } + + groupedFlowFiles.add(validFlowFile); + } + + private void transferFlowFiles() { + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry -> { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + } ).forEach(entry -> { + // Check current state. + final String groupId = entry.getKey(); + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder)); + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder); + final List<FlowFile> groupedFlowFiles = entry.getValue(); + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId)); + + groupedFlowFiles.forEach(f -> { + final Integer order = getOrder.apply(f); + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr)); + + if (order == targetOrder.get()) { + transferResult(f, REL_SUCCESS, null, null); + if (!isMaxOrder) { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + } + + } else if (order > targetOrder.get()) { + + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis) { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + } else { + transferResult(f, REL_WAIT, null, targetOrder.get()); + } + + } else { + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get()); + logger.warn(msg + ". {}", new Object[] {f} ); + transferResult(f, REL_SKIPPED, msg, targetOrder.get()); + } + + }); + + if (previousTargetOrder != targetOrder.get()) { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + }); + } + + private void transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) { + final Map<String, String> attributes = new HashMap<>(); + attributes.put(ATTR_RESULT, result.getName()); + if (expectedOrder != null) { — End diff – Thanks for catching this. I'll clear those attribute when it transferred to 'success'.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110662972

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,543 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("0")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that have larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + List<FlowFile> flowFiles = session.get(batchCount);
          + if (flowFiles == null || flowFiles.isEmpty())

          { + return; + }

          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (FlowFile flowFile : flowFiles) {
          + oc.setFlowFile(flowFile);
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValid();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + // Parse it to check if it is a valid integer. + Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValid() {
          + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>());
          +
          + final FlowFile validFlowFile;
          + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT)))

          { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + }

          else

          { + validFlowFile = flowFile; + }

          +
          + groupedFlowFiles.add(validFlowFile);
          + }
          +
          + private void transferFlowFiles() {
          + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry ->

          { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + }

          ).forEach(entry -> {
          + // Check current state.
          + final String groupId = entry.getKey();
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder));
          + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder);
          + final List<FlowFile> groupedFlowFiles = entry.getValue();
          + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId));
          +
          + groupedFlowFiles.forEach(f -> {
          + final Integer order = getOrder.apply(f);
          + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr));
          +
          + if (order == targetOrder.get()) {
          + transferResult(f, REL_SUCCESS, null, null);
          + if (!isMaxOrder)

          { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + }

          +
          + } else if (order > targetOrder.get()) {
          +
          + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis)

          { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + }

          else

          { + transferResult(f, REL_WAIT, null, targetOrder.get()); + }

          +
          + } else {
          + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get());
          + logger.warn(msg + ". {}", new Object[]

          {f}

          );
          + transferResult(f, REL_SKIPPED, msg, targetOrder.get());
          + }
          +
          + });
          +
          + if (previousTargetOrder != targetOrder.get())

          { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          + });
          + }
          +
          + private void transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) {
          + final Map<String, String> attributes = new HashMap<>();
          + attributes.put(ATTR_RESULT, result.getName());
          + if (expectedOrder != null) {
          — End diff –

          Seems like when the flow file is to be transferred to SUCCESS then the attribute will not be set, but it may have already been set for the flow file if it had been transferred to WAIT. We should set the attribute to the actual order, or even better delete it to save memory since it won't be a useful attribute value anyway.

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110662972 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,543 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("0") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that have larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + List<FlowFile> flowFiles = session.get(batchCount); + if (flowFiles == null || flowFiles.isEmpty()) { + return; + } + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (FlowFile flowFile : flowFiles) { + oc.setFlowFile(flowFile); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValid(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + // Parse it to check if it is a valid integer. + Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValid() { + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>()); + + final FlowFile validFlowFile; + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT))) { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + } else { + validFlowFile = flowFile; + } + + groupedFlowFiles.add(validFlowFile); + } + + private void transferFlowFiles() { + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry -> { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + } ).forEach(entry -> { + // Check current state. + final String groupId = entry.getKey(); + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder)); + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder); + final List<FlowFile> groupedFlowFiles = entry.getValue(); + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId)); + + groupedFlowFiles.forEach(f -> { + final Integer order = getOrder.apply(f); + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr)); + + if (order == targetOrder.get()) { + transferResult(f, REL_SUCCESS, null, null); + if (!isMaxOrder) { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + } + + } else if (order > targetOrder.get()) { + + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis) { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + } else { + transferResult(f, REL_WAIT, null, targetOrder.get()); + } + + } else { + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get()); + logger.warn(msg + ". {}", new Object[] {f} ); + transferResult(f, REL_SKIPPED, msg, targetOrder.get()); + } + + }); + + if (previousTargetOrder != targetOrder.get()) { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + }); + } + + private void transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) { + final Map<String, String> attributes = new HashMap<>(); + attributes.put(ATTR_RESULT, result.getName()); + if (expectedOrder != null) { — End diff – Seems like when the flow file is to be transferred to SUCCESS then the attribute will not be set, but it may have already been set for the flow file if it had been transferred to WAIT. We should set the attribute to the actual order, or even better delete it to save memory since it won't be a useful attribute value anyway.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on the issue:

          https://github.com/apache/nifi/pull/1496

          @mattyb149 I've incorporated your comments. Thanks for reviewing!

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on the issue: https://github.com/apache/nifi/pull/1496 @mattyb149 I've incorporated your comments. Thanks for reviewing!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110565159

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          — End diff –

          I was using SplitText for testing, but I agree with you, 0 is preferable. Changed default value from 1 to 0.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110565159 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") — End diff – I was using SplitText for testing, but I agree with you, 0 is preferable. Changed default value from 1 to 0.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110565041

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          +
          + oc.setFlowFile(session.get());
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValied();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValied() {
          + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>());
          +
          + final FlowFile validFlowFile;
          + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT)))

          { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + }

          else

          { + validFlowFile = flowFile; + }

          +
          + groupedFlowFiles.add(validFlowFile);
          + }
          +
          + private void transferFlowFiles() {
          + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry ->

          { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + }

          ).forEach(entry -> {
          + // Check current state.
          + final String groupId = entry.getKey();
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder));
          + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder);
          + final List<FlowFile> groupedFlowFiles = entry.getValue();
          + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId));
          +
          + groupedFlowFiles.forEach(f -> {
          + final Integer order = getOrder.apply(f);
          + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr));
          +
          + if (order == targetOrder.get()) {
          + transferResult(f, REL_SUCCESS, null, null);
          + if (!isMaxOrder)

          { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + }

          +
          + } else if (order > targetOrder.get()) {
          +
          + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis)

          { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + }

          else

          { + transferResult(f, REL_WAIT, null, targetOrder.get()); + }

          +
          + } else {
          + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get());
          + logger.warn(msg + ". {}", new Object[]

          {f}

          );
          + transferResult(f, REL_SKIPPED, msg, targetOrder.get());
          + }
          +
          + });
          +
          + if (previousTargetOrder != targetOrder.get())

          { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          + });
          + }
          +
          + private FlowFile transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) {
          — End diff –

          Agreed, changed its return to void. Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110565041 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { + + oc.setFlowFile(session.get()); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValied(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValied() { + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>()); + + final FlowFile validFlowFile; + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT))) { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + } else { + validFlowFile = flowFile; + } + + groupedFlowFiles.add(validFlowFile); + } + + private void transferFlowFiles() { + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry -> { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + } ).forEach(entry -> { + // Check current state. + final String groupId = entry.getKey(); + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder)); + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder); + final List<FlowFile> groupedFlowFiles = entry.getValue(); + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId)); + + groupedFlowFiles.forEach(f -> { + final Integer order = getOrder.apply(f); + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr)); + + if (order == targetOrder.get()) { + transferResult(f, REL_SUCCESS, null, null); + if (!isMaxOrder) { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + } + + } else if (order > targetOrder.get()) { + + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis) { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + } else { + transferResult(f, REL_WAIT, null, targetOrder.get()); + } + + } else { + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get()); + logger.warn(msg + ". {}", new Object[] {f} ); + transferResult(f, REL_SKIPPED, msg, targetOrder.get()); + } + + }); + + if (previousTargetOrder != targetOrder.get()) { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + }); + } + + private FlowFile transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) { — End diff – Agreed, changed its return to void. Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110564978

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          +
          + oc.setFlowFile(session.get());
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValied();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValied() {
          — End diff –

          Thanks, fixed it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110564978 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { + + oc.setFlowFile(session.get()); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValied(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValied() { — End diff – Thanks, fixed it.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110564955

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          +
          + oc.setFlowFile(session.get());
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValied();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k -> {
          + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue();
          + final int initOrder = Integer.parseInt(initOrderStr);
          — End diff –

          You are correct, I used parseInt here just for checking number format. I removed initOrder variable assignment and added a comment to clarify. Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110564955 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { + + oc.setFlowFile(session.get()); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValied(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); — End diff – You are correct, I used parseInt here just for checking number format. I removed initOrder variable assignment and added a comment to clarify. Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110564894

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          — End diff –

          Replaced it with session.get(batchCount). Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110564894 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { — End diff – Replaced it with session.get(batchCount). Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110563127

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml —
          @@ -214,6 +214,11 @@ language governing permissions and limitations under the License. -->
          <scope>test</scope>
          </dependency>
          <dependency>
          + <groupId>org.apache.nifi</groupId>
          + <artifactId>nifi-standard-prioritizers</artifactId>
          — End diff –

          Moved standard-prioritizer dependency from nifi-framework-bundle to nifi. Confirmed that I can build it from scratch.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110563127 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml — @@ -214,6 +214,11 @@ language governing permissions and limitations under the License. --> <scope>test</scope> </dependency> <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-standard-prioritizers</artifactId> — End diff – Moved standard-prioritizer dependency from nifi-framework-bundle to nifi. Confirmed that I can build it from scratch.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on the issue:

          https://github.com/apache/nifi/pull/1496

          @mattyb149 Glad to hear that EnforceOrderProcessor works for your use-cases. I'll fix the typos and apply suggested changes. Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on the issue: https://github.com/apache/nifi/pull/1496 @mattyb149 Glad to hear that EnforceOrderProcessor works for your use-cases. I'll fix the typos and apply suggested changes. Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110395349

          — Diff: nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java —
          @@ -753,23 +763,8 @@ public void transfer(final Collection<FlowFile> flowFiles, final Relationship re
          transfer(flowFiles);
          return;
          }

          • if(!processor.getRelationships().contains(relationship)){
              • End diff –

          I replaced those by calling this [transfer(flowfile, relationship)](https://github.com/ijokarumawak/nifi/blob/6c15af2d89289d53fd51cdfe8e675b55bfe128f8/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java#L743) that does the same validations.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110395349 — Diff: nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java — @@ -753,23 +763,8 @@ public void transfer(final Collection<FlowFile> flowFiles, final Relationship re transfer(flowFiles); return; } if(!processor.getRelationships().contains(relationship)){ End diff – I replaced those by calling this [transfer(flowfile, relationship)] ( https://github.com/ijokarumawak/nifi/blob/6c15af2d89289d53fd51cdfe8e675b55bfe128f8/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java#L743 ) that does the same validations.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110388716

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          — End diff –

          minor typo, "any FlowFiles that have"

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110388716 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + — End diff – minor typo, "any FlowFiles that have"
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on the issue:

          https://github.com/apache/nifi/pull/1496

          I added a [gist](https://gist.github.com/mattyb149/bc601d0d00409922e788c0d4e5f8b0e2) with a couple of use cases as well (branching w random delay, and multiple Split processors).

          I left a few minor comments, overall it looks and works great, very well designed and concise clear code. Great work!

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on the issue: https://github.com/apache/nifi/pull/1496 I added a [gist] ( https://gist.github.com/mattyb149/bc601d0d00409922e788c0d4e5f8b0e2 ) with a couple of use cases as well (branching w random delay, and multiple Split processors). I left a few minor comments, overall it looks and works great, very well designed and concise clear code. Great work!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110383364

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          — End diff –

          Argh, found this in SplitText:
          `int fragmentIndex = 1; // set to 1 to preserve the existing behavior only. Perhaps should be deprecated to follow the 0,1,2... scheme`
          Still, I think most of them are 0-based

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110383364 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") — End diff – Argh, found this in SplitText: `int fragmentIndex = 1; // set to 1 to preserve the existing behavior only . Perhaps should be deprecated to follow the 0,1,2... scheme` Still, I think most of them are 0-based
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110380550

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          — End diff –

          A number of processors (the Split processors, CaptureChange, etc.) use zero as the first index / sequence ID by default, I'm thinking we should default this property to zero as well.

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110380550 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") — End diff – A number of processors (the Split processors, CaptureChange, etc.) use zero as the first index / sequence ID by default, I'm thinking we should default this property to zero as well.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110270160

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          +
          + oc.setFlowFile(session.get());
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValied();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValied() {
          + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>());
          +
          + final FlowFile validFlowFile;
          + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT)))

          { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + }

          else

          { + validFlowFile = flowFile; + }

          +
          + groupedFlowFiles.add(validFlowFile);
          + }
          +
          + private void transferFlowFiles() {
          + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry ->

          { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + }

          ).forEach(entry -> {
          + // Check current state.
          + final String groupId = entry.getKey();
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder));
          + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder);
          + final List<FlowFile> groupedFlowFiles = entry.getValue();
          + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId));
          +
          + groupedFlowFiles.forEach(f -> {
          + final Integer order = getOrder.apply(f);
          + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr));
          +
          + if (order == targetOrder.get()) {
          + transferResult(f, REL_SUCCESS, null, null);
          + if (!isMaxOrder)

          { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + }

          +
          + } else if (order > targetOrder.get()) {
          +
          + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis)

          { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + }

          else

          { + transferResult(f, REL_WAIT, null, targetOrder.get()); + }

          +
          + } else {
          + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get());
          + logger.warn(msg + ". {}", new Object[]

          {f}

          );
          + transferResult(f, REL_SKIPPED, msg, targetOrder.get());
          + }
          +
          + });
          +
          + if (previousTargetOrder != targetOrder.get())

          { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          + });
          + }
          +
          + private FlowFile transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) {
          — End diff –

          None of the calls to transferResult appear to use the returned FlowFile and it's not a public method, so probably should be void

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110270160 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { + + oc.setFlowFile(session.get()); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValied(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValied() { + final List<FlowFile> groupedFlowFiles = flowFileGroups.computeIfAbsent(groupId, k -> new ArrayList<>()); + + final FlowFile validFlowFile; + if (isBlank(flowFile.getAttribute(ATTR_STARTED_AT))) { + validFlowFile = processSession.putAttribute(flowFile, ATTR_STARTED_AT, String.valueOf(now)); + } else { + validFlowFile = flowFile; + } + + groupedFlowFiles.add(validFlowFile); + } + + private void transferFlowFiles() { + flowFileGroups.entrySet().stream().filter(entry -> !entry.getValue().isEmpty()).map(entry -> { + // Sort flow files within each group. + final List<FlowFile> groupedFlowFiles = entry.getValue(); + groupedFlowFiles.sort(Comparator.comparing(getOrder)); + return entry; + } ).forEach(entry -> { + // Check current state. + final String groupId = entry.getKey(); + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + final int previousTargetOrder = Integer.parseInt(groupStates.get(stateKeyOrder)); + final AtomicInteger targetOrder = new AtomicInteger(previousTargetOrder); + final List<FlowFile> groupedFlowFiles = entry.getValue(); + final String maxOrderStr = groupStates.get(STATE_MAX_ORDER.apply(groupId)); + + groupedFlowFiles.forEach(f -> { + final Integer order = getOrder.apply(f); + final boolean isMaxOrder = !isBlank(maxOrderStr) && order.equals(Integer.parseInt(maxOrderStr)); + + if (order == targetOrder.get()) { + transferResult(f, REL_SUCCESS, null, null); + if (!isMaxOrder) { + // If max order is specified and this FlowFile has the max order, don't increment target anymore. + targetOrder.incrementAndGet(); + } + + } else if (order > targetOrder.get()) { + + if (now - Long.parseLong(f.getAttribute(ATTR_STARTED_AT)) > waitTimeoutMillis) { + transferResult(f, REL_OVERTOOK, null, targetOrder.get()); + targetOrder.set(isMaxOrder ? order : order + 1); + } else { + transferResult(f, REL_WAIT, null, targetOrder.get()); + } + + } else { + final String msg = String.format("Skipped, FlowFile order was %d but current target is %d", order, targetOrder.get()); + logger.warn(msg + ". {}", new Object[] {f} ); + transferResult(f, REL_SKIPPED, msg, targetOrder.get()); + } + + }); + + if (previousTargetOrder != targetOrder.get()) { + groupStates.put(stateKeyOrder, String.valueOf(targetOrder.get())); + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + }); + } + + private FlowFile transferResult(final FlowFile flowFile, final Relationship result, final String detail, final Integer expectedOrder) { — End diff – None of the calls to transferResult appear to use the returned FlowFile and it's not a public method, so probably should be void
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110269816

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          +
          + oc.setFlowFile(session.get());
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValied();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k ->

          { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + }

          );
          + // If these map modification is in the computeIfAbsent function, it causes this issue.
          + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find.
          + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667
          + if (!isBlank(computedInitOrder.get()))

          { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + }

          +
          + } catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          + return true;
          + }
          +
          + private void markFlowFileValied() {
          — End diff –

          minor typo in the method name, should be Valid

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110269816 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { + + oc.setFlowFile(session.get()); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValied(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); + computedInitOrder.set(initOrderStr); + return initOrderStr; + } ); + // If these map modification is in the computeIfAbsent function, it causes this issue. + // JDK-8071667 : HashMap.computeIfAbsent() adds entry that HashMap.get() does not find. + // http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8071667 + if (!isBlank(computedInitOrder.get())) { + groupStates.put(STATE_UPDATED_AT.apply(groupId), String.valueOf(now)); + } + + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Initial Order for Group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + return true; + } + + private void markFlowFileValied() { — End diff – minor typo in the method name, should be Valid
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110269623

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          +
          + oc.setFlowFile(session.get());
          + if (oc.flowFile == null)

          { + break; + }

          +
          + if (!oc.computeGroupId()
          + || !oc.computeOrder()
          + || !oc.computeInitialOrder()
          + || !oc.computeMaxOrder())

          { + continue; + }

          +
          + // At this point, the flow file is confirmed to be valid.
          + oc.markFlowFileValied();
          + }
          +
          + oc.transferFlowFiles();
          +
          + oc.cleanupInactiveStates();
          +
          + try

          { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + }

          catch (final IOException e)

          { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + }

          +
          + }
          +
          + private class OrderingContext {
          +
          + private final ComponentLog logger = getLogger();
          + private final ProcessSession processSession;
          + private final ProcessContext processContext;
          +
          + // Following properties are static global setting for all groups.
          + private final String orderAttribute;
          + private final Long waitTimeoutMillis;
          + private final Function<FlowFile, Integer> getOrder;
          +
          + private final Map<String, String> groupStates = new HashMap<>();
          + private final long now = System.currentTimeMillis();
          +
          + // Following properties are computed per flow file.
          + private final PropertyValue groupIdentifierProperty ;
          +
          + // Followings are per group objects.
          + private final PropertyValue initOrderProperty;
          + private final PropertyValue maxOrderProperty;
          + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>();
          +
          + // Current variables within incoming FlowFiles loop.
          + private FlowFile flowFile;
          + private String groupId;
          + private Integer order;
          +
          + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession)

          { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + }

          +
          + private void setFlowFile(final FlowFile flowFile)

          { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + }

          +
          + private boolean computeGroupId() {
          + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue();
          + if (isBlank(groupId))

          { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + }

          + return true;
          + }
          +
          + private boolean computeOrder() {
          + try

          { + order = getOrder.apply(flowFile); + }

          catch (final NumberFormatException e)

          { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + }

          + return true;
          + }
          +
          + private boolean computeMaxOrder() {
          + if (maxOrderProperty.isSet()) {
          + // Compute maxOrder for this group if it's not there yet.
          + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId),
          + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue());
          + if (isBlank(maxOrderStr))

          { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + }

          +
          + final Integer maxOrder;
          + try

          { + maxOrder = Integer.parseInt(maxOrderStr); + }

          catch (final NumberFormatException e)

          { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + }

          +
          + // Check max order.
          + if (order > maxOrder)

          { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + }

          + }
          + return true;
          + }
          +
          + private boolean computeInitialOrder() {
          + // Compute initial order. Use asInteger() to check if it's a valid integer.
          + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId);
          + try {
          + final AtomicReference<String> computedInitOrder = new AtomicReference<>();
          + groupStates.computeIfAbsent(stateKeyOrder, k -> {
          + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue();
          + final int initOrder = Integer.parseInt(initOrderStr);
          — End diff –

          initOrder is unused, looks like its here in case the string can't be parsed (NumberFormatException)? If you remove initOrder there's still a warning so you don't have to remove it, I'm just making sure you didn't mean to use it for something other than trying to parse the string

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110269623 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { + + oc.setFlowFile(session.get()); + if (oc.flowFile == null) { + break; + } + + if (!oc.computeGroupId() + || !oc.computeOrder() + || !oc.computeInitialOrder() + || !oc.computeMaxOrder()) { + continue; + } + + // At this point, the flow file is confirmed to be valid. + oc.markFlowFileValied(); + } + + oc.transferFlowFiles(); + + oc.cleanupInactiveStates(); + + try { + context.getStateManager().setState(oc.groupStates, Scope.LOCAL); + } catch (final IOException e) { + throw new RuntimeException("Failed to update state due to " + e + + ". Session will be rollback and processor will be yielded for a while.", e); + } + + } + + private class OrderingContext { + + private final ComponentLog logger = getLogger(); + private final ProcessSession processSession; + private final ProcessContext processContext; + + // Following properties are static global setting for all groups. + private final String orderAttribute; + private final Long waitTimeoutMillis; + private final Function<FlowFile, Integer> getOrder; + + private final Map<String, String> groupStates = new HashMap<>(); + private final long now = System.currentTimeMillis(); + + // Following properties are computed per flow file. + private final PropertyValue groupIdentifierProperty ; + + // Followings are per group objects. + private final PropertyValue initOrderProperty; + private final PropertyValue maxOrderProperty; + private final Map<String, List<FlowFile>> flowFileGroups = new TreeMap<>(); + + // Current variables within incoming FlowFiles loop. + private FlowFile flowFile; + private String groupId; + private Integer order; + + private OrderingContext(final ProcessContext processContext, final ProcessSession processSession) { + this.processContext = processContext; + this.processSession = processSession; + + orderAttribute = processContext.getProperty(ORDER_ATTRIBUTE).getValue(); + waitTimeoutMillis = processContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + getOrder = flowFile -> Integer.parseInt(flowFile.getAttribute(orderAttribute)); + + + groupIdentifierProperty = processContext.getProperty(GROUP_IDENTIFIER); + + initOrderProperty = processContext.getProperty(INITIAL_ORDER); + maxOrderProperty = processContext.getProperty(MAX_ORDER); + } + + private void setFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + this.groupId = null; + this.order = null; + } + + private boolean computeGroupId() { + groupId = groupIdentifierProperty.evaluateAttributeExpressions(flowFile).getValue(); + if (isBlank(groupId)) { + transferToFailure(flowFile, "Failed to get Group Identifier."); + return false; + } + return true; + } + + private boolean computeOrder() { + try { + order = getOrder.apply(flowFile); + } catch (final NumberFormatException e) { + transferToFailure(flowFile, "Failed to parse order attribute due to " + e, e); + return false; + } + return true; + } + + private boolean computeMaxOrder() { + if (maxOrderProperty.isSet()) { + // Compute maxOrder for this group if it's not there yet. + final String maxOrderStr = groupStates.computeIfAbsent(STATE_MAX_ORDER.apply(groupId), + k -> maxOrderProperty.evaluateAttributeExpressions(flowFile).getValue()); + if (isBlank(maxOrderStr)) { + transferToFailure(flowFile, String.format("%s was specified but result was empty.", MAX_ORDER.getDisplayName())); + return false; + } + + final Integer maxOrder; + try { + maxOrder = Integer.parseInt(maxOrderStr); + } catch (final NumberFormatException e) { + final String msg = String.format("Failed to get Maximum Order for group [%s] due to %s", groupId, e); + transferToFailure(flowFile, msg, e); + return false; + } + + // Check max order. + if (order > maxOrder) { + final String msg = String.format("Order (%d) is greater than the Maximum Order (%d) for Group [%s]", order, maxOrder, groupId); + transferToFailure(flowFile, msg); + return false; + } + } + return true; + } + + private boolean computeInitialOrder() { + // Compute initial order. Use asInteger() to check if it's a valid integer. + final String stateKeyOrder = STATE_TARGET_ORDER.apply(groupId); + try { + final AtomicReference<String> computedInitOrder = new AtomicReference<>(); + groupStates.computeIfAbsent(stateKeyOrder, k -> { + final String initOrderStr = initOrderProperty.evaluateAttributeExpressions(flowFile).getValue(); + final int initOrder = Integer.parseInt(initOrderStr); — End diff – initOrder is unused, looks like its here in case the string can't be parsed (NumberFormatException)? If you remove initOrder there's still a warning so you don't have to remove it, I'm just making sure you didn't mean to use it for something other than trying to parse the string
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110268161

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java —
          @@ -0,0 +1,538 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements. See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +package org.apache.nifi.processors.standard;
          +
          +import org.apache.nifi.annotation.behavior.EventDriven;
          +import org.apache.nifi.annotation.behavior.InputRequirement;
          +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
          +import org.apache.nifi.annotation.behavior.Stateful;
          +import org.apache.nifi.annotation.behavior.TriggerSerially;
          +import org.apache.nifi.annotation.behavior.WritesAttribute;
          +import org.apache.nifi.annotation.behavior.WritesAttributes;
          +import org.apache.nifi.annotation.documentation.CapabilityDescription;
          +import org.apache.nifi.annotation.documentation.Tags;
          +import org.apache.nifi.components.PropertyDescriptor;
          +import org.apache.nifi.components.PropertyValue;
          +import org.apache.nifi.components.ValidationContext;
          +import org.apache.nifi.components.ValidationResult;
          +import org.apache.nifi.components.state.Scope;
          +import org.apache.nifi.components.state.StateMap;
          +import org.apache.nifi.flowfile.FlowFile;
          +import org.apache.nifi.logging.ComponentLog;
          +import org.apache.nifi.processor.AbstractProcessor;
          +import org.apache.nifi.processor.ProcessContext;
          +import org.apache.nifi.processor.ProcessSession;
          +import org.apache.nifi.processor.Relationship;
          +import org.apache.nifi.processor.exception.ProcessException;
          +import org.apache.nifi.processor.util.StandardValidators;
          +
          +import java.io.IOException;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Collections;
          +import java.util.Comparator;
          +import java.util.HashMap;
          +import java.util.HashSet;
          +import java.util.List;
          +import java.util.Map;
          +import java.util.Set;
          +import java.util.TreeMap;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicInteger;
          +import java.util.concurrent.atomic.AtomicReference;
          +import java.util.function.Function;
          +import java.util.stream.Collectors;
          +
          +import static org.apache.commons.lang3.StringUtils.isBlank;
          +
          +@EventDriven
          +@Tags(

          {"sort", "order"}

          )
          +@InputRequirement(Requirement.INPUT_REQUIRED)
          +@TriggerSerially
          +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " +
          + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," +
          + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." +
          + " EnforceOrder can be used to enforce original ordering for those FlowFiles." +
          + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" +
          + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.")
          +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" +
          + " '<groupId>.target' is a order number which is being waited to arrive next." +
          + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," +
          + " target order will be updated to (FlowFile.order + 1)." +
          + " '<groupId>.max is the maximum order number for a group." +
          + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." +
          + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.")
          +@WritesAttributes(

          { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +}

          )
          +public class EnforceOrder extends AbstractProcessor {
          +
          + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt";
          + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder";
          + public static final String ATTR_RESULT = "EnforceOrder.result";
          + public static final String ATTR_DETAIL = "EnforceOrder.detail";
          + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target";
          + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt";
          + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT;
          + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max";
          +
          + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder()
          + .name("group-id")
          + .displayName("Group Identifier")
          + .description("EnforceOrder is capable of multiple ordering groups." +
          + " 'Group Identifier' is used to determine which group a FlowFile belongs to." +
          + " This property will be evaluated with each incoming FlowFile." +
          + " If evaluated result is empty, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("$

          {filename}

          ")
          + .build();
          +
          + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder()
          + .name("order-attribute")
          + .displayName("Order Attribute")
          + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." +
          + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.")
          + .required(true)
          + .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder()
          + .name("initial-order")
          + .displayName("Initial Order")
          + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." +
          + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.")
          + .required(true)
          + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .defaultValue("1")
          + .build();
          +
          + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder()
          + .name("maximum-order")
          + .displayName("Maximum Order")
          + .description("If specified, any FlowFiles that has larger order will be routed to failure." +
          + " This property is computed only once for a given group." +
          + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." +
          + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," +
          + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.")
          + .required(false)
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(true)
          + .build();
          +
          + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("wait-timeout")
          + .displayName("Wait Timeout")
          + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.")
          + .required(true)
          + .defaultValue("10 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder()
          + .name("inactive-timeout")
          + .displayName("Inactive Timeout")
          + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." +
          + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." +
          + " Inactive Timeout must be longer than Wait Timeout." +
          + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," +
          + " but will never match the order since expected preceding FlowFiles are already gone." +
          + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." +
          + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.")
          + .required(true)
          + .defaultValue("30 min")
          + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder()
          + .name("batch-count")
          + .displayName("Batch Count")
          + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.")
          + .required(true)
          + .defaultValue("1000")
          + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
          + .expressionLanguageSupported(false)
          + .build();
          +
          + public static final Relationship REL_SUCCESS = new Relationship.Builder()
          + .name("success")
          + .description("A FlowFile with a matching order number will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_FAILURE = new Relationship.Builder()
          + .name("failure")
          + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_WAIT = new Relationship.Builder()
          + .name("wait")
          + .description("A FlowFile with non matching order will be routed to this relationship")
          + .build();
          +
          + public static final Relationship REL_OVERTOOK = new Relationship.Builder()
          + .name("overtook")
          + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.")
          + .build();
          +
          + public static final Relationship REL_SKIPPED = new Relationship.Builder()
          + .name("skipped")
          + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.")
          + .build();
          +
          + private final Set<Relationship> relationships;
          +
          + public EnforceOrder()

          { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + }

          +
          + @Override
          + protected List<PropertyDescriptor> getSupportedPropertyDescriptors()

          { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + }

          +
          + @Override
          + public Set<Relationship> getRelationships()

          { + return relationships; + }

          +
          +
          + @Override
          + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
          + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
          +
          + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS);
          +
          + if (waitTimeoutMillis >= inactiveTimeoutMillis)

          { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + }

          +
          + return results;
          + }
          +
          + @Override
          + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
          +
          + final ComponentLog logger = getLogger();
          + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger();
          +
          + final StateMap stateMap;
          + try

          { + stateMap = context.getStateManager().getState(Scope.LOCAL); + }

          catch (final IOException e) {
          + logger.error("Failed to retrieve state from StateManager due to {}" + e, e);
          + context.yield();
          + return;
          + }
          +
          + final OrderingContext oc = new OrderingContext(context, session);
          +
          + oc.groupStates.putAll(stateMap.toMap());
          +
          + for (int i = 0; i < batchCount; i++) {
          — End diff –

          It might be better to use session.get(batchCount) and then iterate over those, rather than individual calls to session.get()?

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110268161 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EnforceOrder.java — @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.standard; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.StringUtils.isBlank; + +@EventDriven +@Tags( {"sort", "order"} ) +@InputRequirement(Requirement.INPUT_REQUIRED) +@TriggerSerially +@CapabilityDescription("Enforces expected ordering of FlowFiles those belong to the same data group. " + + " Although PriorityAttributePrioritizer can be used on a connection to ensure that flow files going through that connection are in priority order," + + " depending on error-handling, branching, and other flow designs, it is possible for FlowFiles to get out-of-order." + + " EnforceOrder can be used to enforce original ordering for those FlowFiles." + + " [IMPORTANT] In order to take effect of EnforceOrder, FirstInFirstOutPrioritizer should be used at EVERY downstream relationship" + + " UNTIL the order of FlowFiles physically get FIXED by operation such as MergeContent or being stored to the final destination.") +@Stateful(scopes = Scope.LOCAL, description = "EnforceOrder uses following states per ordering group:" + + " '<groupId>.target' is a order number which is being waited to arrive next." + + " When a FlowFile with a matching order arrives, or a FlowFile overtakes the FlowFile being waited for because of wait timeout," + + " target order will be updated to (FlowFile.order + 1)." + + " '<groupId>.max is the maximum order number for a group." + + " '<groupId>.updatedAt' is a timestamp when the order of a group was updated last time." + + " These managed states will be removed automatically once a group is determined as inactive, see 'Inactive Timeout' for detail.") +@WritesAttributes( { + @WritesAttribute(attribute = EnforceOrder.ATTR_STARTED_AT, + description = "All FlowFiles going through this processor will have this attribute. This value is used to determine wait timeout."), + @WritesAttribute(attribute = EnforceOrder.ATTR_RESULT, + description = "All FlowFiles going through this processor will have this attribute denoting which relationship it was routed to."), + @WritesAttribute(attribute = EnforceOrder.ATTR_DETAIL, + description = "FlowFiles routed to 'failure' or 'skipped' relationship will have this attribute describing details."), + @WritesAttribute(attribute = EnforceOrder.ATTR_EXPECTED_ORDER, + description = "FlowFiles routed to 'wait' or 'skipped' relationship will have this attribute denoting expected order when the FlowFile was processed.") +} ) +public class EnforceOrder extends AbstractProcessor { + + public static final String ATTR_STARTED_AT = "EnforceOrder.startedAt"; + public static final String ATTR_EXPECTED_ORDER = "EnforceOrder.expectedOrder"; + public static final String ATTR_RESULT = "EnforceOrder.result"; + public static final String ATTR_DETAIL = "EnforceOrder.detail"; + private static final Function<String, String> STATE_TARGET_ORDER = groupId -> groupId + ".target"; + private static final String STATE_SUFFIX_UPDATED_AT = ".updatedAt"; + private static final Function<String, String> STATE_UPDATED_AT = groupId -> groupId + STATE_SUFFIX_UPDATED_AT; + private static final Function<String, String> STATE_MAX_ORDER = groupId -> groupId + ".max"; + + public static final PropertyDescriptor GROUP_IDENTIFIER = new PropertyDescriptor.Builder() + .name("group-id") + .displayName("Group Identifier") + .description("EnforceOrder is capable of multiple ordering groups." + + " 'Group Identifier' is used to determine which group a FlowFile belongs to." + + " This property will be evaluated with each incoming FlowFile." + + " If evaluated result is empty, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("$ {filename} ") + .build(); + + public static final PropertyDescriptor ORDER_ATTRIBUTE = new PropertyDescriptor.Builder() + .name("order-attribute") + .displayName("Order Attribute") + .description("A name of FlowFile attribute whose value will be used to enforce order of FlowFiles within a group." + + " If a FlowFile does not have this attribute, or its value is not an integer, the FlowFile will be routed to failure.") + .required(true) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INITIAL_ORDER = new PropertyDescriptor.Builder() + .name("initial-order") + .displayName("Initial Order") + .description("When the first FlowFile of a group arrives, initial target order will be computed and stored in the managed state." + + " After that, target order will start being tracked by EnforceOrder and stored in the state management store." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and initial order will be left unknown until consecutive FlowFiles provide a valid initial order.") + .required(true) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("1") + .build(); + + public static final PropertyDescriptor MAX_ORDER = new PropertyDescriptor.Builder() + .name("maximum-order") + .displayName("Maximum Order") + .description("If specified, any FlowFiles that has larger order will be routed to failure." + + " This property is computed only once for a given group." + + " After a maximum order is computed, it will be persisted in the state management store and used for other FlowFiles belonging to the same group." + + " If Expression Language is used but evaluated result was not an integer, then the FlowFile will be routed to failure," + + " and maximum order will be left unknown until consecutive FlowFiles provide a valid maximum order.") + .required(false) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor WAIT_TIMEOUT = new PropertyDescriptor.Builder() + .name("wait-timeout") + .displayName("Wait Timeout") + .description("Indicates the duration after which waiting FlowFiles will be routed to the 'overtook' relationship.") + .required(true) + .defaultValue("10 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor INACTIVE_TIMEOUT = new PropertyDescriptor.Builder() + .name("inactive-timeout") + .displayName("Inactive Timeout") + .description("Indicates the duration after which state for an inactive group will be cleared from managed state." + + " Group is determined as inactive if any new incoming FlowFile has not seen for a group for specified duration." + + " Inactive Timeout must be longer than Wait Timeout." + + " If a FlowFile arrives late after its group is already cleared, it will be treated as a brand new group," + + " but will never match the order since expected preceding FlowFiles are already gone." + + " The FlowFile will eventually timeout for waiting and routed to 'overtook'." + + " To avoid this, group states should be kept long enough, however, shorter duration would be helpful for reusing the same group identifier again.") + .required(true) + .defaultValue("30 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor BATCH_COUNT = new PropertyDescriptor.Builder() + .name("batch-count") + .displayName("Batch Count") + .description("The maximum number of FlowFiles that EnforceOrder can process at an execution.") + .required(true) + .defaultValue("1000") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(false) + .build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("A FlowFile with a matching order number will be routed to this relationship.") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("A FlowFiles which does not have required attributes, or fails to compute those will be routed to this relationship") + .build(); + + public static final Relationship REL_WAIT = new Relationship.Builder() + .name("wait") + .description("A FlowFile with non matching order will be routed to this relationship") + .build(); + + public static final Relationship REL_OVERTOOK = new Relationship.Builder() + .name("overtook") + .description("A FlowFile that waited for preceding FlowFiles longer than Wait Timeout and overtook those FlowFiles, will be routed to this relationship.") + .build(); + + public static final Relationship REL_SKIPPED = new Relationship.Builder() + .name("skipped") + .description("A FlowFile that has an order younger than current, which means arrived too late and skipped, will be routed to this relationship.") + .build(); + + private final Set<Relationship> relationships; + + public EnforceOrder() { + final Set<Relationship> rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_WAIT); + rels.add(REL_OVERTOOK); + rels.add(REL_FAILURE); + rels.add(REL_SKIPPED); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + final List<PropertyDescriptor> descriptors = new ArrayList<>(); + descriptors.add(GROUP_IDENTIFIER); + descriptors.add(ORDER_ATTRIBUTE); + descriptors.add(INITIAL_ORDER); + descriptors.add(MAX_ORDER); + descriptors.add(BATCH_COUNT); + descriptors.add(WAIT_TIMEOUT); + descriptors.add(INACTIVE_TIMEOUT); + return descriptors; + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + + @Override + protected Collection<ValidationResult> customValidate(ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext)); + + final Long waitTimeoutMillis = validationContext.getProperty(WAIT_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + final Long inactiveTimeoutMillis = validationContext.getProperty(INACTIVE_TIMEOUT).asTimePeriod(TimeUnit.MICROSECONDS); + + if (waitTimeoutMillis >= inactiveTimeoutMillis) { + results.add(new ValidationResult.Builder().input(validationContext.getProperty(INACTIVE_TIMEOUT).getValue()) + .subject(INACTIVE_TIMEOUT.getDisplayName()) + .explanation(String.format("%s should be longer than %s", + INACTIVE_TIMEOUT.getDisplayName(), WAIT_TIMEOUT.getDisplayName())) + .valid(false) + .build()); + } + + return results; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + final ComponentLog logger = getLogger(); + final Integer batchCount = context.getProperty(BATCH_COUNT).asInteger(); + + final StateMap stateMap; + try { + stateMap = context.getStateManager().getState(Scope.LOCAL); + } catch (final IOException e) { + logger.error("Failed to retrieve state from StateManager due to {}" + e, e); + context.yield(); + return; + } + + final OrderingContext oc = new OrderingContext(context, session); + + oc.groupStates.putAll(stateMap.toMap()); + + for (int i = 0; i < batchCount; i++) { — End diff – It might be better to use session.get(batchCount) and then iterate over those, rather than individual calls to session.get()?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110264405

          — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml —
          @@ -214,6 +214,11 @@ language governing permissions and limitations under the License. -->
          <scope>test</scope>
          </dependency>
          <dependency>
          + <groupId>org.apache.nifi</groupId>
          + <artifactId>nifi-standard-prioritizers</artifactId>
          — End diff –

          The version for this artifact is specified in the nifi-framework-bundle, it should probably be moved up to the top-level POM's dependencyManagement section since now another NAR depends on it. I couldn't build this PR from scratch without specifying the version here.

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110264405 — Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml — @@ -214,6 +214,11 @@ language governing permissions and limitations under the License. --> <scope>test</scope> </dependency> <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-standard-prioritizers</artifactId> — End diff – The version for this artifact is specified in the nifi-framework-bundle, it should probably be moved up to the top-level POM's dependencyManagement section since now another NAR depends on it. I couldn't build this PR from scratch without specifying the version here.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user mattyb149 commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1496#discussion_r110260826

          — Diff: nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java —
          @@ -753,23 +763,8 @@ public void transfer(final Collection<FlowFile> flowFiles, final Relationship re
          transfer(flowFiles);
          return;
          }

          • if(!processor.getRelationships().contains(relationship)){
              • End diff –

          I couldn't see why/where this section of code needs to be remove, can you elaborate?

          Show
          githubbot ASF GitHub Bot added a comment - Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1496#discussion_r110260826 — Diff: nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java — @@ -753,23 +763,8 @@ public void transfer(final Collection<FlowFile> flowFiles, final Relationship re transfer(flowFiles); return; } if(!processor.getRelationships().contains(relationship)){ End diff – I couldn't see why/where this section of code needs to be remove, can you elaborate?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on the issue:

          https://github.com/apache/nifi/pull/1496

          Hey @pvillard31 thanks for trying out this PR. I've update it to use both `.name()` and `.displayName`. Please let me know if you find anything suspicious during your test.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on the issue: https://github.com/apache/nifi/pull/1496 Hey @pvillard31 thanks for trying out this PR. I've update it to use both `.name()` and `.displayName`. Please let me know if you find anything suspicious during your test.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user pvillard31 commented on the issue:

          https://github.com/apache/nifi/pull/1496

          Hey @ijokarumawak, I did few tests with your processor following your Gist explanations and using your template. It worked as expected. I'll try to test other scenarios. In the meantime, could you use both ``.name()`` and ``.displayName()`` in the processor properties?

          Show
          githubbot ASF GitHub Bot added a comment - Github user pvillard31 commented on the issue: https://github.com/apache/nifi/pull/1496 Hey @ijokarumawak, I did few tests with your processor following your Gist explanations and using your template. It worked as expected. I'll try to test other scenarios. In the meantime, could you use both ``.name()`` and ``.displayName()`` in the processor properties?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ijokarumawak commented on the issue:

          https://github.com/apache/nifi/pull/1496

          I've written a Gist with a flow template to describe how this is intended to work, hope this helps for reviewing PR. Thanks!
          https://gist.github.com/ijokarumawak/88fc30a2300845b3c27a79113fc72d41

          Show
          githubbot ASF GitHub Bot added a comment - Github user ijokarumawak commented on the issue: https://github.com/apache/nifi/pull/1496 I've written a Gist with a flow template to describe how this is intended to work, hope this helps for reviewing PR. Thanks! https://gist.github.com/ijokarumawak/88fc30a2300845b3c27a79113fc72d41
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user ijokarumawak opened a pull request:

          https://github.com/apache/nifi/pull/1496

          NIFI-3414: Added EnforceOrder processor

          Use it with FirstInFirstOutPrioritizer, it can enforce original ordering
          of 'out-of-order' FlowFiles.

          nifi-mock is modified to support FlowFile assertion using Prioritizer.

          Thank you for submitting a contribution to Apache NiFi.

          In order to streamline the review of the contribution we ask you
          to ensure the following steps have been taken:

              1. For all changes:
          • [x] Is there a JIRA ticket associated with this PR? Is it referenced
            in the commit message?
          • [x] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
          • [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
          • [x] Is your initial contribution a single, squashed commit?
              1. For code changes:
          • [x] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
          • [x] Have you written or updated unit tests to verify your changes?
          • [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
          • [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
          • [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
          • [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
              1. For documentation related changes:
          • [x] Have you ensured that format looks appropriate for the output in which it is rendered?
              1. Note:
                Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/ijokarumawak/nifi nifi-3414

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/nifi/pull/1496.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #1496


          commit a60fc5d88e1a1ab3190052f19e228908e996c818
          Author: Koji Kawamura <ijokarumawak@apache.org>
          Date: 2017-02-07T15:13:23Z

          NIFI-3414: Added EnforceOrder processor

          Use it with FirstInFirstOutPrioritizer, it can enforce original ordering
          of 'out-of-order' FlowFiles.

          nifi-mock is modified to support FlowFile assertion using Prioritizer.


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user ijokarumawak opened a pull request: https://github.com/apache/nifi/pull/1496 NIFI-3414 : Added EnforceOrder processor Use it with FirstInFirstOutPrioritizer, it can enforce original ordering of 'out-of-order' FlowFiles. nifi-mock is modified to support FlowFile assertion using Prioritizer. Thank you for submitting a contribution to Apache NiFi. In order to streamline the review of the contribution we ask you to ensure the following steps have been taken: For all changes: [x] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message? [x] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character. [x] Has your PR been rebased against the latest commit within the target branch (typically master)? [x] Is your initial contribution a single, squashed commit? For code changes: [x] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder? [x] Have you written or updated unit tests to verify your changes? [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0] ( http://www.apache.org/legal/resolved.html#category-a)? [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly? [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly? [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties? For documentation related changes: [x] Have you ensured that format looks appropriate for the output in which it is rendered? Note: Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijokarumawak/nifi nifi-3414 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/nifi/pull/1496.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1496 commit a60fc5d88e1a1ab3190052f19e228908e996c818 Author: Koji Kawamura <ijokarumawak@apache.org> Date: 2017-02-07T15:13:23Z NIFI-3414 : Added EnforceOrder processor Use it with FirstInFirstOutPrioritizer, it can enforce original ordering of 'out-of-order' FlowFiles. nifi-mock is modified to support FlowFile assertion using Prioritizer.
          Hide
          ijokarumawak Koji Kawamura added a comment -

          Thanks Matt for explaining detailed use-case. (I thought I wrote a reply last week but the comment wasn't saved properly..)

          I'm going to implement the processor, and add following capabilities as well:

          • To not update state storage too often, batch up input flow files max to 'Max Batch Count' and process those at once
          • Group incoming flow files by 'Group Identifier', and handle multiple groups
          • Add 'wait' queue to route flow files those didn't match with current order number, so that those leave from the incoming queue and make room for other flow files can be enqueued. Penalize those, too.
          • Track how long a flow file has been waiting. If it exceeds configured 'Timeout', routes it to 'timeout' relationship. Update the order number with the maximum + 1 of timeout flow files, so that if the skipped flow file arrives later, processor can detect it
          • If incoming file has an order number younger than the current, then route it to 'skipped' relationship
          Show
          ijokarumawak Koji Kawamura added a comment - Thanks Matt for explaining detailed use-case. (I thought I wrote a reply last week but the comment wasn't saved properly..) I'm going to implement the processor, and add following capabilities as well: To not update state storage too often, batch up input flow files max to 'Max Batch Count' and process those at once Group incoming flow files by 'Group Identifier', and handle multiple groups Add 'wait' queue to route flow files those didn't match with current order number, so that those leave from the incoming queue and make room for other flow files can be enqueued. Penalize those, too. Track how long a flow file has been waiting. If it exceeds configured 'Timeout', routes it to 'timeout' relationship. Update the order number with the maximum + 1 of timeout flow files, so that if the skipped flow file arrives later, processor can detect it If incoming file has an order number younger than the current, then route it to 'skipped' relationship
          Hide
          mattyb149 Matt Burgess added a comment -

          Certainly! Consider a set of incoming flow files that contain JSON objects, each with a field "status" with a value of "OK" or "ERROR". After extracting this code with EvaluateJsonPath, a RouteOnAttribute might follow, and there might be different processing for "OK" files vs "ERROR" files. Then if the two branches meet again by being connected to a funnel or a common processor, the original order of the flow files may not be preserved. This processor would use the Order Attribute property (and the flow would have to be configured with the correct Prioritizers as mentioned in the description) to ensure that the output of EnforceOrder retains the original order of the incoming flow files.

          Show
          mattyb149 Matt Burgess added a comment - Certainly! Consider a set of incoming flow files that contain JSON objects, each with a field "status" with a value of "OK" or "ERROR". After extracting this code with EvaluateJsonPath, a RouteOnAttribute might follow, and there might be different processing for "OK" files vs "ERROR" files. Then if the two branches meet again by being connected to a funnel or a common processor, the original order of the flow files may not be preserved. This processor would use the Order Attribute property (and the flow would have to be configured with the correct Prioritizers as mentioned in the description) to ensure that the output of EnforceOrder retains the original order of the incoming flow files.
          Hide
          ijokarumawak Koji Kawamura added a comment -

          Hi Matt Burgess,

          I'm interested in working on this. To understand the requirement properly, would you share some example NiFi flow that possibly violate desired ordering?

          Show
          ijokarumawak Koji Kawamura added a comment - Hi Matt Burgess , I'm interested in working on this. To understand the requirement properly, would you share some example NiFi flow that possibly violate desired ordering?

            People

            • Assignee:
              ijokarumawak Koji Kawamura
              Reporter:
              mattyb149 Matt Burgess
            • Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development