You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by gresockj <gi...@git.apache.org> on 2016/12/14 15:37:57 UTC

[GitHub] nifi pull request #1329: NIFI-190: Initial commit of Wait and Notify process...

GitHub user gresockj opened a pull request:

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

    NIFI-190: Initial commit of Wait and Notify processors

    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?
    - [x] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [ ] 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/gresockj/nifi NIFI-190

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

    https://github.com/apache/nifi/pull/1329.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 #1329
    
----
commit 12d686952e220f74862fc031b3495b6acc8eb0ff
Author: Joe Gresock <jo...@lmco.com>
Date:   2016-12-14T15:32:16Z

    NIFI-190: Initial commit of Wait and Notify processors

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1329: NIFI-190: Initial commit of Wait and Notify processors

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi/pull/1329
  
    Reviewing...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1329: NIFI-190: Initial commit of Wait and Notify processors

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi/pull/1329
  
    That makes sense, I think we can add that functionality later. I created this JIRA to track the changes for the DistributedMapCacheClient https://issues.apache.org/jira/browse/NIFI-3214, then we can eventually use that to determine if there are N keys in the cache matching a pattern, and release based on that.
    
    I'm a +1 on this PR as is and will merge shortly, thanks for reviving this ticket! 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1329: NIFI-190: Initial commit of Wait and Notify process...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1329#discussion_r92808940
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java ---
    @@ -0,0 +1,191 @@
    +/*
    + * 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 java.io.IOException;
    +import java.io.OutputStream;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang3.StringUtils;
    +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.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
    +import org.apache.nifi.distributed.cache.client.Serializer;
    +import org.apache.nifi.distributed.cache.client.exception.SerializationException;
    +import org.apache.nifi.expression.AttributeExpression.ResultType;
    +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 org.apache.nifi.processors.standard.util.FlowFileAttributesSerializer;
    +
    +@EventDriven
    +@SupportsBatching
    +@Tags({"map", "cache", "notify", "distributed", "signal", "release"})
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Caches a release signal identifier in the distributed cache, optionally along with "
    +        + "the FlowFile's attributes.  Any flow files held at a corresponding Wait processor will be "
    +        + "released once this signal in the cache is discovered.")
    +@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
    +        "org.apache.nifi.processors.standard.Wait"})
    +public class Notify extends AbstractProcessor {
    +
    +    // Identifies the distributed map cache client
    +    public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
    +        .name("Distributed Cache Service")
    +        .description("The Controller Service that is used to check for release signals from a corresponding Notify processor")
    +        .required(true)
    +        .identifiesControllerService(DistributedMapCacheClient.class)
    +        .build();
    +
    +    // Selects the FlowFile attribute or expression, whose value is used as cache key
    +    public static final PropertyDescriptor RELEASE_SIGNAL_IDENTIFIER = new PropertyDescriptor.Builder()
    +        .name("Release Signal Identifier")
    +        .description("A value, or the results of an Attribute Expression Language statement, which will " +
    +            "be evaluated against a FlowFile in order to determine the release signal cache key")
    +        .required(true)
    +        .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
    +        .expressionLanguageSupported(true)
    +        .build();
    +
    +    // Specifies an optional regex used to identify which attributes to cache
    +    public static final PropertyDescriptor ATTRIBUTE_CACHE_REGEX = new PropertyDescriptor.Builder()
    +        .name("Attribute Cache Regex")
    +        .description("Any attributes whose names match this regex will be stored in the distributed cache to be "
    +                + "copied to any FlowFiles released from a corresponding Wait processor.  Note that the "
    +                + "uuid attribute will not be cached regardless of this value.  If blank, no attributes "
    +                + "will be cached.")
    +        .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +        .expressionLanguageSupported(false)
    +        .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("A FlowFile with a matching release signal in the cache will be routed to this relationship")
    --- End diff --
    
    Minor point, but should this be something like "All FlowFiles where the release signal has been successfully entered in the cache will be routed to this relationship"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1329: NIFI-190: Initial commit of Wait and Notify process...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1329#discussion_r92809224
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java ---
    @@ -0,0 +1,266 @@
    +/*
    + * 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 java.io.IOException;
    +import java.io.OutputStream;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.lang3.StringUtils;
    +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.SupportsBatching;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.distributed.cache.client.Deserializer;
    +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
    +import org.apache.nifi.distributed.cache.client.Serializer;
    +import org.apache.nifi.distributed.cache.client.exception.SerializationException;
    +import org.apache.nifi.expression.AttributeExpression.ResultType;
    +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 org.apache.nifi.processors.standard.util.FlowFileAttributesSerializer;
    +
    +@EventDriven
    +@SupportsBatching
    +@Tags({"map", "cache", "wait", "hold", "distributed", "signal", "release"})
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Routes incoming FlowFiles to the 'wait' relationship until a matching release signal "
    +        + "is stored in the distributed cache from a corresponding Notify processor.  At this point, a waiting FlowFile is routed to "
    +        + "the 'success' relationship, with attributes copied from the FlowFile that produced "
    +        + "the release signal from the Notify processor.  The release signal entry is then removed from "
    +        + "the cache.  Waiting FlowFiles will be routed to 'expired' if they exceed the Expiration Duration.")
    +@WritesAttribute(attribute = "wait.start.timestamp", description = "All FlowFiles will have an attribute 'wait.start.timestamp', which sets the "
    +        + "initial epoch timestamp when the file first entered this processor.  This is used to determine the expiration time of the FlowFile.")
    +@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
    +        "org.apache.nifi.processors.standard.Notify"})
    +public class Wait extends AbstractProcessor {
    +
    +    public static final String WAIT_START_TIMESTAMP = "wait.start.timestamp";
    +
    +    // Identifies the distributed map cache client
    +    public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
    +        .name("Distributed Cache Service")
    +        .description("The Controller Service that is used to check for release signals from a corresponding Notify processor")
    +        .required(true)
    +        .identifiesControllerService(DistributedMapCacheClient.class)
    +        .build();
    +
    +    // Selects the FlowFile attribute or expression, whose value is used as cache key
    +    public static final PropertyDescriptor RELEASE_SIGNAL_IDENTIFIER = new PropertyDescriptor.Builder()
    +        .name("Release Signal Identifier")
    +        .description("A value, or the results of an Attribute Expression Language statement, which will " +
    +            "be evaluated against a FlowFile in order to determine the release signal cache key")
    +        .required(true)
    +        .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
    +        .expressionLanguageSupported(true)
    +        .build();
    +
    +    // Selects the FlowFile attribute or expression, whose value is used as cache key
    +    public static final PropertyDescriptor EXPIRATION_DURATION = new PropertyDescriptor.Builder()
    +        .name("Expiration Duration")
    +        .description("Indicates the duration after which waiting flow files will be routed to the 'expired' relationship")
    +        .required(true)
    +        .defaultValue("10 min")
    +        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +        .expressionLanguageSupported(false)
    +        .build();
    +
    +    public static final AllowableValue ATTRIBUTE_COPY_REPLACE = new AllowableValue("replace", "Replace if present",
    +            "When cached attributes are copied onto released FlowFiles, they replace any matching attributes.");
    +
    +    public static final AllowableValue ATTRIBUTE_COPY_KEEP_ORIGINAL = new AllowableValue("keeporiginal", "Keep original",
    +            "Attributes on released FlowFiles are not overwritten by copied cached attributes.");
    +
    +    public static final PropertyDescriptor ATTRIBUTE_COPY_MODE = new PropertyDescriptor.Builder()
    +        .name("Attribute Copy Mode")
    +        .description("Specifies how to handle attributes copied from flow files entering the Notify processor")
    +        .defaultValue(ATTRIBUTE_COPY_KEEP_ORIGINAL.getValue())
    +        .required(true)
    +        .allowableValues(ATTRIBUTE_COPY_REPLACE, ATTRIBUTE_COPY_KEEP_ORIGINAL)
    +        .expressionLanguageSupported(false)
    +        .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("A FlowFile with a matching release signal in the cache will be routed to this relationship")
    +        .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +        .name("failure")
    +        .description("When the cache cannot be reached, or if the Release Signal Identifier evaluates to null or empty, FlowFiles will be routed to this relationship")
    +        .build();
    +
    +    public static final Relationship REL_WAIT = new Relationship.Builder()
    +        .name("wait")
    +        .description("A FlowFile with no matching release signal in the cache will be routed to this relationship")
    +        .build();
    +
    +    public static final Relationship REL_EXPIRED = new Relationship.Builder()
    +        .name("expired")
    +        .description("A FlowFile that has exceeded the configured Expiration Duration will be routed to this relationship")
    +        .build();
    +    private final Set<Relationship> relationships;
    +
    +    private final Serializer<String> keySerializer = new StringSerializer();
    +    private final Deserializer<Map<String, String>> valueDeserializer = new FlowFileAttributesSerializer();
    +
    +    public Wait() {
    +        final Set<Relationship> rels = new HashSet<>();
    +        rels.add(REL_SUCCESS);
    +        rels.add(REL_WAIT);
    +        rels.add(REL_EXPIRED);
    +        rels.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(rels);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<>();
    +        descriptors.add(RELEASE_SIGNAL_IDENTIFIER);
    +        descriptors.add(EXPIRATION_DURATION);
    +        descriptors.add(DISTRIBUTED_CACHE_SERVICE);
    +        descriptors.add(ATTRIBUTE_COPY_MODE);
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        FlowFile flowFile = session.get();
    +        if (flowFile == null) {
    +            return;
    +        }
    +
    +        final ComponentLog logger = getLogger();
    +
    +        // cache key is computed from attribute 'RELEASE_SIGNAL_IDENTIFIER' with expression language support
    +        final String cacheKey = context.getProperty(RELEASE_SIGNAL_IDENTIFIER).evaluateAttributeExpressions(flowFile).getValue();
    +
    +        // if the computed value is null, or empty, we transfer the flow file to failure relationship
    +        if (StringUtils.isBlank(cacheKey)) {
    +            logger.error("FlowFile {} has no attribute for given Release Signal Identifier", new Object[] {flowFile});
    +            flowFile = session.penalize(flowFile);
    +            session.transfer(flowFile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // the cache client used to interact with the distributed cache
    +        final DistributedMapCacheClient cache = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
    +
    +        try {
    +            // check for expiration
    +            String waitStartTimestamp = flowFile.getAttribute(WAIT_START_TIMESTAMP);
    +            if (waitStartTimestamp == null) {
    +                waitStartTimestamp = String.valueOf(System.currentTimeMillis());
    +                flowFile = session.putAttribute(flowFile, WAIT_START_TIMESTAMP, waitStartTimestamp);
    +            }
    +
    +            long lWaitStartTimestamp = 0L;
    +            try {
    +                lWaitStartTimestamp = Long.parseLong(waitStartTimestamp);
    +            } catch (NumberFormatException nfe) {
    +                logger.error("{} has an invalid value '{}' on FlowFile {}", new Object[] {WAIT_START_TIMESTAMP, waitStartTimestamp, flowFile});
    +                flowFile = session.penalize(flowFile);
    +                session.transfer(flowFile, REL_FAILURE);
    +                return;
    +            }
    +            long expirationDuration = context.getProperty(EXPIRATION_DURATION)
    +                    .asTimePeriod(TimeUnit.MILLISECONDS);
    +            long now = System.currentTimeMillis();
    +            if (now > (lWaitStartTimestamp + expirationDuration)) {
    +                logger.warn("FlowFile {} expired after {}ms", new Object[] {flowFile, (now - lWaitStartTimestamp)});
    +                session.transfer(flowFile, REL_EXPIRED);
    +                return;
    +            }
    +
    +            // get notifying flow file attributes
    +            Map<String, String> cachedAttributes = cache.get(cacheKey, keySerializer, valueDeserializer);
    +
    +            if (cachedAttributes == null) {
    +                if (logger.isDebugEnabled()) {
    +                    logger.debug("No release signal yet for {} on FlowFile {}", new Object[] {cacheKey, flowFile});
    +                }
    +                session.transfer(flowFile, REL_WAIT);
    --- End diff --
    
    Since the wait relationship is going to be looped back to itself and continually retrying, I'm wondering if there should be some way to throttle how fast it is retrying? The only thing I can think of would be to penalize the flow file, but then we are using the penalty duration for two different scenarios, but maybe that is fine. Thoughts?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1329: NIFI-190: Initial commit of Wait and Notify process...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1329: NIFI-190: Initial commit of Wait and Notify process...

Posted by gresockj <gi...@git.apache.org>.
Github user gresockj commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1329#discussion_r92813543
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Notify.java ---
    @@ -0,0 +1,191 @@
    +/*
    + * 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 java.io.IOException;
    +import java.io.OutputStream;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +import org.apache.commons.lang3.StringUtils;
    +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.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
    +import org.apache.nifi.distributed.cache.client.Serializer;
    +import org.apache.nifi.distributed.cache.client.exception.SerializationException;
    +import org.apache.nifi.expression.AttributeExpression.ResultType;
    +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 org.apache.nifi.processors.standard.util.FlowFileAttributesSerializer;
    +
    +@EventDriven
    +@SupportsBatching
    +@Tags({"map", "cache", "notify", "distributed", "signal", "release"})
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Caches a release signal identifier in the distributed cache, optionally along with "
    +        + "the FlowFile's attributes.  Any flow files held at a corresponding Wait processor will be "
    +        + "released once this signal in the cache is discovered.")
    +@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
    +        "org.apache.nifi.processors.standard.Wait"})
    +public class Notify extends AbstractProcessor {
    +
    +    // Identifies the distributed map cache client
    +    public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
    +        .name("Distributed Cache Service")
    +        .description("The Controller Service that is used to check for release signals from a corresponding Notify processor")
    +        .required(true)
    +        .identifiesControllerService(DistributedMapCacheClient.class)
    +        .build();
    +
    +    // Selects the FlowFile attribute or expression, whose value is used as cache key
    +    public static final PropertyDescriptor RELEASE_SIGNAL_IDENTIFIER = new PropertyDescriptor.Builder()
    +        .name("Release Signal Identifier")
    +        .description("A value, or the results of an Attribute Expression Language statement, which will " +
    +            "be evaluated against a FlowFile in order to determine the release signal cache key")
    +        .required(true)
    +        .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
    +        .expressionLanguageSupported(true)
    +        .build();
    +
    +    // Specifies an optional regex used to identify which attributes to cache
    +    public static final PropertyDescriptor ATTRIBUTE_CACHE_REGEX = new PropertyDescriptor.Builder()
    +        .name("Attribute Cache Regex")
    +        .description("Any attributes whose names match this regex will be stored in the distributed cache to be "
    +                + "copied to any FlowFiles released from a corresponding Wait processor.  Note that the "
    +                + "uuid attribute will not be cached regardless of this value.  If blank, no attributes "
    +                + "will be cached.")
    +        .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +        .expressionLanguageSupported(false)
    +        .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("A FlowFile with a matching release signal in the cache will be routed to this relationship")
    --- End diff --
    
    Technically it would be: "The first FlowFile with a matching release signal in the cache...", since Wait removes the cache entry once it has successfully transferred the flow file to success.  
    
    Do you think there should be a use case where the cache value is not removed?  If so, we could add a property like "Remove Successful Signals?"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1329: NIFI-190: Initial commit of Wait and Notify process...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1329#discussion_r92815826
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Wait.java ---
    @@ -0,0 +1,266 @@
    +/*
    + * 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 java.io.IOException;
    +import java.io.OutputStream;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.commons.lang3.StringUtils;
    +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.SupportsBatching;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.distributed.cache.client.Deserializer;
    +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
    +import org.apache.nifi.distributed.cache.client.Serializer;
    +import org.apache.nifi.distributed.cache.client.exception.SerializationException;
    +import org.apache.nifi.expression.AttributeExpression.ResultType;
    +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 org.apache.nifi.processors.standard.util.FlowFileAttributesSerializer;
    +
    +@EventDriven
    +@SupportsBatching
    +@Tags({"map", "cache", "wait", "hold", "distributed", "signal", "release"})
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Routes incoming FlowFiles to the 'wait' relationship until a matching release signal "
    +        + "is stored in the distributed cache from a corresponding Notify processor.  At this point, a waiting FlowFile is routed to "
    +        + "the 'success' relationship, with attributes copied from the FlowFile that produced "
    +        + "the release signal from the Notify processor.  The release signal entry is then removed from "
    +        + "the cache.  Waiting FlowFiles will be routed to 'expired' if they exceed the Expiration Duration.")
    +@WritesAttribute(attribute = "wait.start.timestamp", description = "All FlowFiles will have an attribute 'wait.start.timestamp', which sets the "
    +        + "initial epoch timestamp when the file first entered this processor.  This is used to determine the expiration time of the FlowFile.")
    +@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
    +        "org.apache.nifi.processors.standard.Notify"})
    +public class Wait extends AbstractProcessor {
    +
    +    public static final String WAIT_START_TIMESTAMP = "wait.start.timestamp";
    +
    +    // Identifies the distributed map cache client
    +    public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
    +        .name("Distributed Cache Service")
    +        .description("The Controller Service that is used to check for release signals from a corresponding Notify processor")
    +        .required(true)
    +        .identifiesControllerService(DistributedMapCacheClient.class)
    +        .build();
    +
    +    // Selects the FlowFile attribute or expression, whose value is used as cache key
    +    public static final PropertyDescriptor RELEASE_SIGNAL_IDENTIFIER = new PropertyDescriptor.Builder()
    +        .name("Release Signal Identifier")
    +        .description("A value, or the results of an Attribute Expression Language statement, which will " +
    +            "be evaluated against a FlowFile in order to determine the release signal cache key")
    +        .required(true)
    +        .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
    +        .expressionLanguageSupported(true)
    +        .build();
    +
    +    // Selects the FlowFile attribute or expression, whose value is used as cache key
    +    public static final PropertyDescriptor EXPIRATION_DURATION = new PropertyDescriptor.Builder()
    +        .name("Expiration Duration")
    +        .description("Indicates the duration after which waiting flow files will be routed to the 'expired' relationship")
    +        .required(true)
    +        .defaultValue("10 min")
    +        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +        .expressionLanguageSupported(false)
    +        .build();
    +
    +    public static final AllowableValue ATTRIBUTE_COPY_REPLACE = new AllowableValue("replace", "Replace if present",
    +            "When cached attributes are copied onto released FlowFiles, they replace any matching attributes.");
    +
    +    public static final AllowableValue ATTRIBUTE_COPY_KEEP_ORIGINAL = new AllowableValue("keeporiginal", "Keep original",
    +            "Attributes on released FlowFiles are not overwritten by copied cached attributes.");
    +
    +    public static final PropertyDescriptor ATTRIBUTE_COPY_MODE = new PropertyDescriptor.Builder()
    +        .name("Attribute Copy Mode")
    +        .description("Specifies how to handle attributes copied from flow files entering the Notify processor")
    +        .defaultValue(ATTRIBUTE_COPY_KEEP_ORIGINAL.getValue())
    +        .required(true)
    +        .allowableValues(ATTRIBUTE_COPY_REPLACE, ATTRIBUTE_COPY_KEEP_ORIGINAL)
    +        .expressionLanguageSupported(false)
    +        .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +        .name("success")
    +        .description("A FlowFile with a matching release signal in the cache will be routed to this relationship")
    +        .build();
    +
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +        .name("failure")
    +        .description("When the cache cannot be reached, or if the Release Signal Identifier evaluates to null or empty, FlowFiles will be routed to this relationship")
    +        .build();
    +
    +    public static final Relationship REL_WAIT = new Relationship.Builder()
    +        .name("wait")
    +        .description("A FlowFile with no matching release signal in the cache will be routed to this relationship")
    +        .build();
    +
    +    public static final Relationship REL_EXPIRED = new Relationship.Builder()
    +        .name("expired")
    +        .description("A FlowFile that has exceeded the configured Expiration Duration will be routed to this relationship")
    +        .build();
    +    private final Set<Relationship> relationships;
    +
    +    private final Serializer<String> keySerializer = new StringSerializer();
    +    private final Deserializer<Map<String, String>> valueDeserializer = new FlowFileAttributesSerializer();
    +
    +    public Wait() {
    +        final Set<Relationship> rels = new HashSet<>();
    +        rels.add(REL_SUCCESS);
    +        rels.add(REL_WAIT);
    +        rels.add(REL_EXPIRED);
    +        rels.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(rels);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<>();
    +        descriptors.add(RELEASE_SIGNAL_IDENTIFIER);
    +        descriptors.add(EXPIRATION_DURATION);
    +        descriptors.add(DISTRIBUTED_CACHE_SERVICE);
    +        descriptors.add(ATTRIBUTE_COPY_MODE);
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +
    +        FlowFile flowFile = session.get();
    +        if (flowFile == null) {
    +            return;
    +        }
    +
    +        final ComponentLog logger = getLogger();
    +
    +        // cache key is computed from attribute 'RELEASE_SIGNAL_IDENTIFIER' with expression language support
    +        final String cacheKey = context.getProperty(RELEASE_SIGNAL_IDENTIFIER).evaluateAttributeExpressions(flowFile).getValue();
    +
    +        // if the computed value is null, or empty, we transfer the flow file to failure relationship
    +        if (StringUtils.isBlank(cacheKey)) {
    +            logger.error("FlowFile {} has no attribute for given Release Signal Identifier", new Object[] {flowFile});
    +            flowFile = session.penalize(flowFile);
    +            session.transfer(flowFile, REL_FAILURE);
    +            return;
    +        }
    +
    +        // the cache client used to interact with the distributed cache
    +        final DistributedMapCacheClient cache = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
    +
    +        try {
    +            // check for expiration
    +            String waitStartTimestamp = flowFile.getAttribute(WAIT_START_TIMESTAMP);
    +            if (waitStartTimestamp == null) {
    +                waitStartTimestamp = String.valueOf(System.currentTimeMillis());
    +                flowFile = session.putAttribute(flowFile, WAIT_START_TIMESTAMP, waitStartTimestamp);
    +            }
    +
    +            long lWaitStartTimestamp = 0L;
    +            try {
    +                lWaitStartTimestamp = Long.parseLong(waitStartTimestamp);
    +            } catch (NumberFormatException nfe) {
    +                logger.error("{} has an invalid value '{}' on FlowFile {}", new Object[] {WAIT_START_TIMESTAMP, waitStartTimestamp, flowFile});
    +                flowFile = session.penalize(flowFile);
    +                session.transfer(flowFile, REL_FAILURE);
    +                return;
    +            }
    +            long expirationDuration = context.getProperty(EXPIRATION_DURATION)
    +                    .asTimePeriod(TimeUnit.MILLISECONDS);
    +            long now = System.currentTimeMillis();
    +            if (now > (lWaitStartTimestamp + expirationDuration)) {
    +                logger.warn("FlowFile {} expired after {}ms", new Object[] {flowFile, (now - lWaitStartTimestamp)});
    +                session.transfer(flowFile, REL_EXPIRED);
    +                return;
    +            }
    +
    +            // get notifying flow file attributes
    +            Map<String, String> cachedAttributes = cache.get(cacheKey, keySerializer, valueDeserializer);
    +
    +            if (cachedAttributes == null) {
    +                if (logger.isDebugEnabled()) {
    +                    logger.debug("No release signal yet for {} on FlowFile {}", new Object[] {cacheKey, flowFile});
    +                }
    +                session.transfer(flowFile, REL_WAIT);
    --- End diff --
    
    After I wrote this comment I also started thinking about using the scheduling to control it, that seems good to me, so no need to do anything else here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1329: NIFI-190: Initial commit of Wait and Notify processors

Posted by gresockj <gi...@git.apache.org>.
Github user gresockj commented on the issue:

    https://github.com/apache/nifi/pull/1329
  
    I like the idea about waiting for N signals.  I think it can be done by adding a getKeys(Pattern) method to the distributed cache, so maybe it's best captured in a separate ticket.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---