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 [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to