You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/11/03 17:23:01 UTC

[GitHub] [nifi] MikeThomsen opened a new pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

MikeThomsen opened a new pull request #4646:
URL: https://github.com/apache/nifi/pull/4646


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] 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.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] 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:
   - [ ] 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 GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r817130561



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")

Review comment:
       > If none were exactly the same (even if some had individually identical fields like lastName), it sent all to non-dupe.
   
   That's expected behavior.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] adamfisher commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
adamfisher commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1054540044


   > Revisiting this. @ottobackwards @mattyb149 @exceptionfactory going to work through any remaining comments and see if we can close this out this week.
   
   Thank you so much @MikeThomsen! Excited to have this processor block see the light of day. 🌞 🌞 🌞 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r614436894



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DeduplicateRecords/additionalDetails.html
##########
@@ -0,0 +1,53 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>DeduplicateRecords</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<!-- Processor Documentation ================================================== -->
+    <h1>Overview</h1>
+    <p>This processor provides deduplication across either a single record set file or across an entire data lake
+    using a DistributedMapCacheClient controller service. In the case of the former, it uses either a HashSet or a bloom
+    filter to provide extremely fast in-memory calculations with a high degree of accuracy. In the latter use case, it
+    will use the controller service to compare a generated hash against a map cache stored in one of the supported caching
+    options that Apache NiFi offers.</p>
+
+    <h2>Configuring single file deduplication</h2>
+    <p>Choose the "single file" option under the configuration property labeled "Deduplication Strategy." Then choose

Review comment:
       The size limits seems important.  But the specific property to set isn't mentioned as it is with the Deduplication Strategy.  I found myself going to read the code to find the property.  I think we should be explicit with the property names here.
   
   Also, an example of these settings might be very helpful ( along with some description of the record etc )




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] github-actions[bot] commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-819929633


   We're marking this PR as stale due to lack of updates in the past few months. If after another couple of weeks the stale label has not been removed this PR will be closed. This stale marker and eventual auto close does not indicate a judgement of the PR just lack of reviewer bandwidth and helps us keep the PR queue more manageable.  If you would like this PR re-opened you can do so and a committer can remove the stale tag.  Or you can open a new PR.  Try to help review other PRs to increase PR review bandwidth which in turn helps yours.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r531666660



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())

Review comment:
       Made that change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1057062266


   @exceptionfactory @ottobackwards @mattyb149 should be good to go now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1063290645


   @joewitt addressed your request.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r531611612



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +

Review comment:
       Did not realize that. Addressed that.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-721282868


   @adamfisher FYSA


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] adamfisher commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
adamfisher commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r629068847



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,687 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if(filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                }
+
+                filter.put(recordHash);

Review comment:
       I wonder if `filter.put(recordHash);` should be put into the `else` block above so we only put the record into the filter when it doesn't already contain it?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r528945585



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",

Review comment:
       Minor typo in `available`

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +

Review comment:
       Is this ignored when using the Single File strategy? If so, this has a dependency on the `DEDUPLICATION_STRATEGY` value of `OPTION_DATA_LAKE`, now that dependent properties are supported, you can add a `dependsOn` method here.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,

Review comment:
       Can the hashing be CPU-intensive?

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Data lake-level deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        useInMemoryStrategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);

Review comment:
       Might be able to do all these in a big try-with-resources clause?

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");

Review comment:
       Data Lake feels a little specific, maybe "Multiple Files" would work better, and the description could list a Data Lake as an example?

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()

Review comment:
       These two look Bloom-filter-specific, if so please add the `dependsOn` method to them.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()

Review comment:
       This looks like another dependent property

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "

Review comment:
       Does this description apply? I suppose the only way you'd get no output is either with empty input or a bad output schema...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] adamfisher commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
adamfisher commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r629068847



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,687 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if(filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                }
+
+                filter.put(recordHash);

Review comment:
       I wonder if this line should be put into the `else` block above so we only put the record into the filter when it doesn't already contain it?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1063503496


   +1 LGTM, tried with various happy and non-happy scenarios, verified the expected results. Thanks for sticking with this new feature! Merging to main


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r530678668



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())

Review comment:
       Good catch. FWIW, this PR combines two different implementations, so that slipped by me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r531615085



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(DeduplicateRecords.class);
+
+        // RECORD_READER, RECORD_WRITER
+        reader = new MockRecordParser();
+        writer = new MockRecordWriter("header", false);
+
+        runner.addControllerService("reader", reader);
+        runner.enableControllerService(reader);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(DeduplicateRecords.RECORD_READER, "reader");
+        runner.setProperty(DeduplicateRecords.RECORD_WRITER, "writer");
+
+        reader.addSchemaField("firstName", RecordFieldType.STRING);
+        reader.addSchemaField("middleName", RecordFieldType.STRING);
+        reader.addSchemaField("lastName", RecordFieldType.STRING);
+
+        // INCLUDE_ZERO_RECORD_FLOWFILES
+        runner.setProperty(DeduplicateRecords.INCLUDE_ZERO_RECORD_FLOWFILES, "true");
+
+        // CACHE_ENTRY_IDENTIFIER
+
+        runner.assertValid();
+    }
+
+    void commonEnqueue() {
+        final Map<String, String> props = new HashMap<>();
+        props.put("hash.value", "1000");
+        runner.enqueue(new byte[]{}, props);
+    }
+
+    @Test
+    public void testDetectDuplicatesHashSet() {
+        commonEnqueue();
+
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testDetectDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testNoDuplicatesHashSet() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testNoDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testAllDuplicates() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 2);
+    }
+
+    @Test
+    public void testAllUnique() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testCacheValueFromRecordPath() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+
+//        cache.assertContains("KEY", "VALUE"); // TODO: Get the tests running so you can see what the key/value is in serialized form

Review comment:
       It can probably be removed. I'm not sure what @adamfisher intended with that test in his original PR (as this is again, a consolidation of two competing implementations).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-732402943


   @mattyb149  thanks for taking a look. I'll try to carve out some time in the evening to address.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1055582737


   Using a DistributedMapCache client & server (rather than Cassandra, just to be different), I couldn't get any records/flowfiles on the duplicate relationship after sending 2 of the same flowfile into the DeduplicateRecords processor (with the config from your template above, Multiple Files e.g.). I had to enable the rest of the flow so the cache values would be populated by the PutDistributedMapCache processor. Is that intended? If so the docs should reflect that and if not, the processor itself should handle writing the cache identifier so no additional processor is needed, or perhaps it would at least be configurable.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r540950461



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DeduplicateRecords/additionalDetails.html
##########
@@ -0,0 +1,53 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>DeduplicateRecords</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<!-- Processor Documentation ================================================== -->
+    <h1>Overview</h1>
+    <p>This processor provides deduplication across either a single record set file or across an entire data lake

Review comment:
       A data lake is a good use case to mention, but to correspond to the changes to the property value(s) the doc should refer to the actual property name(s), even if they're intended for use on a data lake.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",

Review comment:
       It says `availble` instead of `available`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1055544885


   @mattyb149 @exceptionfactory @ottobackwards I think we're good new. See the attacked template and steps to setup Cassandra in docker to quickly test.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-820349280


   @adamfisher that was an automated message 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-820349761


   @MikeThomsen?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1055541043


   [DeduplicateRecords_TestWithCassandraDMC.xml.txt](https://github.com/apache/nifi/files/8162499/DeduplicateRecords_TestWithCassandraDMC.xml.txt)
   
   Here is a test flow.
   
   ```
   CREATE KEYSPACE nifi
         WITH REPLICATION = { 
          'class' : 'SimpleStrategy', 
          'replication_factor' : 1 
         };
   ```
   
   ```
   use nifi;
   ```
   
   ```
   create table map_cache (key blob, value blob, primary key(key));
   ```
   
   ```
   docker run -p 9042:9042 --name nifi-cassandra cassandra:4
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r816879732



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");

Review comment:
       Since some time has past since the original submission, recommend removing the `MD5` and `SHA-1` algorithms.  Given known possibilities for hash collisions, and the minimal cost of `SHA-256`, there does not seem to be any value in supporting these options.  It is worth noting that NiFi itself switched from MD5 to SHA-256 for NAR digest calculation over a year ago.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");

Review comment:
       There is a difference between `SHA-256` and `SHA3-256`.  Java 8 does not support SHA3 algorithms, so for compatibility and clarity, the `MessageDigestAlgorithms` references should reference the regular SHA_256 and SHA_512 algorithms, as opposed to the SHA3 versions.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';

Review comment:
       Minor stylistic issue: this static variable declaration should be moved prior to all method definitions.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;

Review comment:
       This looks an opportunity to update the test to use JUnit 5.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }

Review comment:
       Is it necessary to set enable this additional logging?

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))

Review comment:
       According to the [MessageDigest](https://docs.oracle.com/javase/8/docs/api/java/security/MessageDigest.html#digest-byte:A-) documentation, calling `digest(byte[] bytes)` updates the internal hash. This means that the digest value is cumulative, as opposed to just computing and returning the value based on the current bytes.
   
   To avoid having to create a new instance of `MessageDigest` for each record, the `digest(byte[] bytes, int offset, int length)` method can be used, as it resets the internal state so that the instance can be used again.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if (filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                    filter.put(recordHash);
+                }
+
+                index++;
+            }
+
+            // Route Non-Duplicates FlowFile
+            final WriteResult nonDuplicatesWriteResult = nonDuplicatesWriter.finishRecordSet();
+            // Route Duplicates FlowFile
+            final WriteResult duplicatesWriteResult = duplicatesWriter.finishRecordSet();
+
+            reader.close();
+            inputStream.close();
+            nonDuplicatesWriter.close();
+            nonDupeStream.close();
+            duplicatesWriter.close();
+            dupeStream.close();
+
+            final boolean includeZeroRecordFlowFiles = context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean();
+
+            session.adjustCounter("Records Processed",
+                    nonDuplicatesWriteResult.getRecordCount() + duplicatesWriteResult.getRecordCount(), false);
+
+            sendOrRemove(session, duplicatesFlowFile, REL_DUPLICATE, duplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, duplicatesWriteResult);
+
+            sendOrRemove(session, nonDuplicatesFlowFile, REL_NON_DUPLICATE, nonDuplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, nonDuplicatesWriteResult);
+
+            session.transfer(flowFile, REL_ORIGINAL);
+
+        } catch (final Exception e) {
+            logger.error("Failed in detecting duplicate records.", e);
+            session.remove(duplicatesFlowFile);
+            session.remove(nonDuplicatesFlowFile);
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    private void sendOrRemove(ProcessSession session,
+                              FlowFile outputFlowFile,
+                              Relationship targetRelationship,
+                              String mimeType,
+                              boolean includeZeroRecordFlowFiles,
+                              WriteResult writeResult) {
+        if(!includeZeroRecordFlowFiles && writeResult.getRecordCount() == 0) {
+            session.remove(outputFlowFile);
+        } else {
+            Map<String, String> attributes = new HashMap<>();
+            attributes.putAll(writeResult.getAttributes());
+            attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+            attributes.put(CoreAttributes.MIME_TYPE.key(), mimeType);
+            outputFlowFile = session.putAllAttributes(outputFlowFile, attributes);
+            if (getLogger().isDebugEnabled()) {
+                getLogger().debug("Successfully found {} unique records for {}",
+                        new Object[]{writeResult.getRecordCount(), outputFlowFile});
+            }

Review comment:
       The wrapping `isDebugEnabled()` check seems unnecessary since the arguments should not be expensive to provide to the logger method.  The wrapping `Object[]` can be removed:
   ```suggestion
               getLogger().debug("Processed [{}] unique records for {}", writeResult.getRecordCount(), outputFlowFile);
   ```

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if (filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                    filter.put(recordHash);
+                }
+
+                index++;
+            }
+
+            // Route Non-Duplicates FlowFile
+            final WriteResult nonDuplicatesWriteResult = nonDuplicatesWriter.finishRecordSet();
+            // Route Duplicates FlowFile
+            final WriteResult duplicatesWriteResult = duplicatesWriter.finishRecordSet();
+
+            reader.close();
+            inputStream.close();
+            nonDuplicatesWriter.close();
+            nonDupeStream.close();
+            duplicatesWriter.close();
+            dupeStream.close();
+
+            final boolean includeZeroRecordFlowFiles = context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean();
+
+            session.adjustCounter("Records Processed",
+                    nonDuplicatesWriteResult.getRecordCount() + duplicatesWriteResult.getRecordCount(), false);
+
+            sendOrRemove(session, duplicatesFlowFile, REL_DUPLICATE, duplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, duplicatesWriteResult);
+
+            sendOrRemove(session, nonDuplicatesFlowFile, REL_NON_DUPLICATE, nonDuplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, nonDuplicatesWriteResult);
+
+            session.transfer(flowFile, REL_ORIGINAL);
+
+        } catch (final Exception e) {
+            logger.error("Failed in detecting duplicate records.", e);
+            session.remove(duplicatesFlowFile);
+            session.remove(nonDuplicatesFlowFile);
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    private void sendOrRemove(ProcessSession session,
+                              FlowFile outputFlowFile,
+                              Relationship targetRelationship,
+                              String mimeType,
+                              boolean includeZeroRecordFlowFiles,
+                              WriteResult writeResult) {
+        if(!includeZeroRecordFlowFiles && writeResult.getRecordCount() == 0) {
+            session.remove(outputFlowFile);
+        } else {
+            Map<String, String> attributes = new HashMap<>();
+            attributes.putAll(writeResult.getAttributes());
+            attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+            attributes.put(CoreAttributes.MIME_TYPE.key(), mimeType);
+            outputFlowFile = session.putAllAttributes(outputFlowFile, attributes);
+            if (getLogger().isDebugEnabled()) {
+                getLogger().debug("Successfully found {} unique records for {}",
+                        new Object[]{writeResult.getRecordCount(), outputFlowFile});
+            }
+
+            session.transfer(outputFlowFile, targetRelationship);
+        }
+    }
+
+    private String executeDynamicRecordPaths(ProcessContext context, Record record, FlowFile flowFile) {
+        final List<String> fieldValues = new ArrayList<>();
+        for (final String recordPathText : recordPaths) {
+            final PropertyValue recordPathPropertyValue = context.getProperty(recordPathText);
+            final RecordPath recordPath = recordPathCache.getCompiled(recordPathText);
+            final RecordPathResult result = recordPath.evaluate(record);
+            final List<FieldValue> selectedFields = result.getSelectedFields().collect(Collectors.toList());
+
+            if (recordPathPropertyValue.isExpressionLanguagePresent()) {
+                final Map<String, String> fieldVariables = new HashMap<>();
+                selectedFields.forEach(fieldVal -> {
+                    fieldVariables.clear();
+                    fieldVariables.put(FIELD_NAME, fieldVal.getField().getFieldName());
+                    fieldVariables.put(FIELD_VALUE, DataTypeUtils.toString(fieldVal.getValue(), (String) null));
+                    fieldVariables.put(FIELD_TYPE, fieldVal.getField().getDataType().getFieldType().name());
+
+                    fieldValues.add(recordPathPropertyValue.evaluateAttributeExpressions(flowFile, fieldVariables).getValue());
+                });
+            } else {
+                fieldValues.add(recordPathPropertyValue.evaluateAttributeExpressions(flowFile).getValue());
+            }
+
+            fieldValues.addAll(selectedFields.stream()
+                    .map(f -> recordPathPropertyValue.evaluateAttributeExpressions(flowFile).getValue())
+                    .collect(toList())
+            );
+        }
+
+        return Joiner.on(JOIN_CHAR).join(fieldValues);
+    }
+
+    private abstract static class FilterWrapper {
+        public static FilterWrapper create(Object filter) {
+            if(filter instanceof HashSet) {
+                return new HashSetFilterWrapper((HashSet<String>) filter);
+            } else {
+                return new BloomFilterWrapper((BloomFilter<String>) filter);
+            }
+        }
+        public abstract boolean contains(String value);
+        public abstract void put(String value);
+    }
+
+    private static class HashSetFilterWrapper extends FilterWrapper {
+
+        private final HashSet<String> filter;
+
+        public HashSetFilterWrapper(HashSet<String> filter) {
+            this.filter = filter;
+        }
+
+        @Override
+        public boolean contains(String value) {
+            return filter.contains(value);
+        }
+
+        @Override
+        public void put(String value) {
+            filter.add(value);
+        }
+    }
+
+    private static class BloomFilterWrapper extends FilterWrapper {
+
+        private final BloomFilter<String> filter;
+
+        public BloomFilterWrapper(BloomFilter<String> filter) {
+            this.filter = filter;
+        }
+
+        @Override
+        public boolean contains(String value) {
+            return filter.mightContain(value);
+        }
+
+        @Override
+        public void put(String value) {
+            filter.put(value);
+        }
+    }
+
+    private static class DistributedMapCacheClientWrapper extends FilterWrapper {
+        private DistributedMapCacheClient client;
+
+        public DistributedMapCacheClientWrapper(DistributedMapCacheClient client) {
+            this.client = client;
+        }
+
+        @Override
+        public boolean contains(String value) {
+            try {
+                return client.containsKey(value, STRING_SERIALIZER);
+            } catch (IOException e) {
+                throw new ProcessException(e);

Review comment:
       It would be helpful to provide a message instead of just wrapping the exception:
   ```suggestion
                   throw new ProcessException("Distributed Map lookup failed", e);
   ```

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if (filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                    filter.put(recordHash);
+                }
+
+                index++;
+            }
+
+            // Route Non-Duplicates FlowFile
+            final WriteResult nonDuplicatesWriteResult = nonDuplicatesWriter.finishRecordSet();
+            // Route Duplicates FlowFile
+            final WriteResult duplicatesWriteResult = duplicatesWriter.finishRecordSet();
+
+            reader.close();
+            inputStream.close();
+            nonDuplicatesWriter.close();
+            nonDupeStream.close();
+            duplicatesWriter.close();
+            dupeStream.close();
+
+            final boolean includeZeroRecordFlowFiles = context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean();
+
+            session.adjustCounter("Records Processed",
+                    nonDuplicatesWriteResult.getRecordCount() + duplicatesWriteResult.getRecordCount(), false);
+
+            sendOrRemove(session, duplicatesFlowFile, REL_DUPLICATE, duplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, duplicatesWriteResult);
+
+            sendOrRemove(session, nonDuplicatesFlowFile, REL_NON_DUPLICATE, nonDuplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, nonDuplicatesWriteResult);
+
+            session.transfer(flowFile, REL_ORIGINAL);
+
+        } catch (final Exception e) {
+            logger.error("Failed in detecting duplicate records.", e);

Review comment:
       It might be helpful to include the current record index for troubleshooting:
   ```suggestion
               logger.error("Duplicate record detection failed: current record [{}]", index, e);
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r817143002



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")

Review comment:
       Supporting EL for the record path property might allow for some use case where a path expression has a dynamic predicate value, but I can also see it being confusing given that it would only work in the context of a single file. Removing the EL support for FlowFile attributes makes sense under the circumstances described.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r817742148



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(DeduplicateRecords.class);
+
+        // RECORD_READER, RECORD_WRITER
+        reader = new MockRecordParser();
+        writer = new MockRecordWriter("header", false);
+
+        runner.addControllerService("reader", reader);
+        runner.enableControllerService(reader);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(DeduplicateRecords.RECORD_READER, "reader");
+        runner.setProperty(DeduplicateRecords.RECORD_WRITER, "writer");
+        runner.setProperty(DeduplicateRecords.RECORD_HASHING_ALGORITHM, DeduplicateRecords.SHA1_ALGORITHM_VALUE);
+
+        reader.addSchemaField("firstName", RecordFieldType.STRING);
+        reader.addSchemaField("middleName", RecordFieldType.STRING);
+        reader.addSchemaField("lastName", RecordFieldType.STRING);
+
+        // INCLUDE_ZERO_RECORD_FLOWFILES
+        runner.setProperty(DeduplicateRecords.INCLUDE_ZERO_RECORD_FLOWFILES, "true");
+
+        runner.assertValid();
+    }
+
+    void commonEnqueue() {
+        final Map<String, String> props = new HashMap<>();
+        props.put("hash.value", "1000");
+        runner.enqueue(new byte[]{}, props);
+    }
+
+    @Test
+    public void testDetectDuplicatesHashSet() {
+        commonEnqueue();
+
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testDetectDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testNoDuplicatesHashSet() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testNoDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testAllDuplicates() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 2);
+    }
+
+    @Test
+    public void testAllUnique() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testCacheValueFromRecordPath() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    /*
+     * These are all related to NIFI-6014
+     */
+
+    @Test
+    public void testMultipleFileDeduplicationRequiresDMC() {
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_MULTIPLE_FILES.getValue());
+        runner.assertNotValid();
+    }
+
+    public static final String FIRST_KEY = "2875ba79836587028a920875a18ee5dceb837587";
+    public static final String SECOND_KEY = "6eeba6ecf9d263582f463890be339dbecbaf23c8";
+
+    @Test
+    public void testDeduplicateWithDMC() throws Exception {
+        DistributedMapCacheClient dmc = new MockCacheService<>();
+        runner.addControllerService("dmc", dmc);
+        runner.setProperty(DeduplicateRecords.DISTRIBUTED_MAP_CACHE, "dmc");
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_MULTIPLE_FILES.getValue());
+        runner.enableControllerService(dmc);
+        runner.assertValid();
+
+        dmc.put(FIRST_KEY, true, null, null);
+        dmc.put(SECOND_KEY, true, null, null);
+
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 3);
+    }
+
+    @Test
+    public void testDeduplicateWithDMCAndCacheIdentifier() throws Exception {
+        DistributedMapCacheClient dmc = new MockCacheService<>();
+        runner.addControllerService("dmc", dmc);
+        runner.setProperty(DeduplicateRecords.DISTRIBUTED_MAP_CACHE, "dmc");
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_MULTIPLE_FILES.getValue());
+        runner.setProperty(DeduplicateRecords.CACHE_IDENTIFIER, "concat('${user.name}', '${record.hash.value}')");
+        runner.enableControllerService(dmc);
+        runner.assertValid();
+
+        dmc.put(String.format("john.smith-%s", FIRST_KEY), true, null, null);
+        dmc.put(String.format("john.smith-%s", SECOND_KEY), true, null, null);
+
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        Map<String, String> attrs = new HashMap<>();
+        attrs.put("user.name", "john.smith-");
+
+        runner.enqueue("", attrs);
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 3);
+    }
+
+    void doCountTests(int failure, int original, int duplicates, int notDuplicates, int notDupeCount, int dupeCount) {
+        runner.assertTransferCount(DeduplicateRecords.REL_DUPLICATE, duplicates);
+        runner.assertTransferCount(DeduplicateRecords.REL_NON_DUPLICATE, notDuplicates);
+        runner.assertTransferCount(DeduplicateRecords.REL_ORIGINAL, original);
+        runner.assertTransferCount(DeduplicateRecords.REL_FAILURE, failure);
+
+        List<MockFlowFile> duplicateFlowFile = runner.getFlowFilesForRelationship(DeduplicateRecords.REL_DUPLICATE);
+        if (duplicateFlowFile != null) {
+            assertEquals(String.valueOf(dupeCount), duplicateFlowFile.get(0).getAttribute("record.count"));
+        }
+
+        List<MockFlowFile> nonDuplicateFlowFile = runner.getFlowFilesForRelationship(DeduplicateRecords.REL_NON_DUPLICATE);
+        if (nonDuplicateFlowFile != null) {
+            assertEquals(String.valueOf(notDupeCount), nonDuplicateFlowFile.get(0).getAttribute("record.count"));
+        }
+    }
+
+    private static final class MockCacheService<K, V> extends AbstractControllerService implements DistributedMapCacheClient {

Review comment:
       I'll add one, but it should only show the transaction rolling back and the flowfile going to failure.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r529080745



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(DeduplicateRecords.class);
+
+        // RECORD_READER, RECORD_WRITER
+        reader = new MockRecordParser();
+        writer = new MockRecordWriter("header", false);
+
+        runner.addControllerService("reader", reader);
+        runner.enableControllerService(reader);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(DeduplicateRecords.RECORD_READER, "reader");
+        runner.setProperty(DeduplicateRecords.RECORD_WRITER, "writer");
+
+        reader.addSchemaField("firstName", RecordFieldType.STRING);
+        reader.addSchemaField("middleName", RecordFieldType.STRING);
+        reader.addSchemaField("lastName", RecordFieldType.STRING);
+
+        // INCLUDE_ZERO_RECORD_FLOWFILES
+        runner.setProperty(DeduplicateRecords.INCLUDE_ZERO_RECORD_FLOWFILES, "true");
+
+        // CACHE_ENTRY_IDENTIFIER
+
+        runner.assertValid();
+    }
+
+    void commonEnqueue() {
+        final Map<String, String> props = new HashMap<>();
+        props.put("hash.value", "1000");
+        runner.enqueue(new byte[]{}, props);
+    }
+
+    @Test
+    public void testDetectDuplicatesHashSet() {
+        commonEnqueue();
+
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testDetectDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testNoDuplicatesHashSet() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testNoDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testAllDuplicates() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 2);
+    }
+
+    @Test
+    public void testAllUnique() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testCacheValueFromRecordPath() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+
+//        cache.assertContains("KEY", "VALUE"); // TODO: Get the tests running so you can see what the key/value is in serialized form

Review comment:
       Should this comment be retained, or is the purpose to implement cache checking in a future test?

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())

Review comment:
       Is there a reason for making SHA-1 the default value?  The [NIST Policy on Hash Functions](https://csrc.nist.gov/Projects/Hash-Functions/NIST-Policy-on-Hash-Functions) and other sources recommend against using SHA-1 for security purposes.  Although this processor may not be strictly targeted for security purposes, defaulting to SHA-256 seems like a better option.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Data lake-level deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        useInMemoryStrategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on('~').join(record.getValues());

Review comment:
       Recommend declaring a static class variable for the joining character since it is used in multiple places.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(DeduplicateRecords.class);
+
+        // RECORD_READER, RECORD_WRITER
+        reader = new MockRecordParser();
+        writer = new MockRecordWriter("header", false);
+
+        runner.addControllerService("reader", reader);
+        runner.enableControllerService(reader);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(DeduplicateRecords.RECORD_READER, "reader");
+        runner.setProperty(DeduplicateRecords.RECORD_WRITER, "writer");
+
+        reader.addSchemaField("firstName", RecordFieldType.STRING);
+        reader.addSchemaField("middleName", RecordFieldType.STRING);
+        reader.addSchemaField("lastName", RecordFieldType.STRING);
+
+        // INCLUDE_ZERO_RECORD_FLOWFILES
+        runner.setProperty(DeduplicateRecords.INCLUDE_ZERO_RECORD_FLOWFILES, "true");
+
+        // CACHE_ENTRY_IDENTIFIER
+
+        runner.assertValid();
+    }
+
+    void commonEnqueue() {
+        final Map<String, String> props = new HashMap<>();
+        props.put("hash.value", "1000");
+        runner.enqueue(new byte[]{}, props);
+    }
+
+    @Test
+    public void testDetectDuplicatesHashSet() {
+        commonEnqueue();
+
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testDetectDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testNoDuplicatesHashSet() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testNoDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testAllDuplicates() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 2);
+    }
+
+    @Test
+    public void testAllUnique() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testCacheValueFromRecordPath() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+
+//        cache.assertContains("KEY", "VALUE"); // TODO: Get the tests running so you can see what the key/value is in serialized form
+    }
+
+    /*
+     * These are all related to NIFI-6014
+     */
+
+    @Test
+    public void testDataLakeDeduplicationRequiresDMC() {
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_DATA_LAKE.getValue());
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testDeduplicateWithDMC() throws Exception {
+        DistributedMapCacheClient dmc = new MockCacheService<>();
+        runner.addControllerService("dmc", dmc);
+        runner.setProperty(DeduplicateRecords.DISTRIBUTED_MAP_CACHE, "dmc");
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_DATA_LAKE.getValue());
+        runner.enableControllerService(dmc);
+        runner.assertValid();
+
+        dmc.put("2875ba79836587028a920875a18ee5dceb837587", true, null, null);
+        dmc.put("6eeba6ecf9d263582f463890be339dbecbaf23c8", true, null, null);

Review comment:
       Recommend declaring static variables for these identifiers so that they can be reused in the following test method.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] adamfisher commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
adamfisher commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-991195276


   If there is something I can do to get this across the finish line I would love to help. I'm just not familiar with the roadblocks I faced in the git process. I put a lot of work into it and I think it would be a very useful processor block for deduping records.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] adamfisher commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
adamfisher commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-819955632


   I would prefer not to see this go away. I put a lot of time into making it generic and robust. Really I just ran into problems near the end when I had to get it merged in properly and Mike is the git ninja. I thought this was almost across the finish line?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r614438593



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DeduplicateRecords/additionalDetails.html
##########
@@ -0,0 +1,53 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>DeduplicateRecords</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<!-- Processor Documentation ================================================== -->
+    <h1>Overview</h1>
+    <p>This processor provides deduplication across either a single record set file or across an entire data lake
+    using a DistributedMapCacheClient controller service. In the case of the former, it uses either a HashSet or a bloom
+    filter to provide extremely fast in-memory calculations with a high degree of accuracy. In the latter use case, it
+    will use the controller service to compare a generated hash against a map cache stored in one of the supported caching
+    options that Apache NiFi offers.</p>
+
+    <h2>Configuring single file deduplication</h2>
+    <p>Choose the "single file" option under the configuration property labeled "Deduplication Strategy." Then choose

Review comment:
       if these *are* the proper display names, and they are just lowercase etc, then they should be bold maybe?    in markdown I would `backquote` them




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r614437577



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(DeduplicateRecords.class);
+
+        // RECORD_READER, RECORD_WRITER
+        reader = new MockRecordParser();
+        writer = new MockRecordWriter("header", false);
+
+        runner.addControllerService("reader", reader);
+        runner.enableControllerService(reader);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(DeduplicateRecords.RECORD_READER, "reader");
+        runner.setProperty(DeduplicateRecords.RECORD_WRITER, "writer");
+        runner.setProperty(DeduplicateRecords.RECORD_HASHING_ALGORITHM, DeduplicateRecords.SHA1_ALGORITHM_VALUE);
+
+        reader.addSchemaField("firstName", RecordFieldType.STRING);
+        reader.addSchemaField("middleName", RecordFieldType.STRING);
+        reader.addSchemaField("lastName", RecordFieldType.STRING);
+
+        // INCLUDE_ZERO_RECORD_FLOWFILES
+        runner.setProperty(DeduplicateRecords.INCLUDE_ZERO_RECORD_FLOWFILES, "true");
+
+        // CACHE_ENTRY_IDENTIFIER

Review comment:
       Is there supposed to be a CACHE_ENTRY_IDENTIFIER?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r614440190



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,687 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if(filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                }
+
+                filter.put(recordHash);
+                index++;
+            }
+
+            reader.close();
+            inputStream.close();
+            nonDuplicatesWriter.close();
+            nonDupeStream.close();
+            duplicatesWriter.close();
+            dupeStream.close();
+
+            final boolean includeZeroRecordFlowFiles = context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean();
+
+            // Route Non-Duplicates FlowFile
+            final WriteResult nonDuplicatesWriteResult = nonDuplicatesWriter.finishRecordSet();
+            // Route Duplicates FlowFile
+            final WriteResult duplicatesWriteResult = duplicatesWriter.finishRecordSet();
+
+            session.adjustCounter("Records Processed",
+                    nonDuplicatesWriteResult.getRecordCount() + duplicatesWriteResult.getRecordCount(), false);
+
+            sendOrRemove(session, duplicatesFlowFile, REL_DUPLICATE, duplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, duplicatesWriteResult);
+
+            sendOrRemove(session, nonDuplicatesFlowFile, REL_NON_DUPLICATE, nonDuplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, nonDuplicatesWriteResult);
+
+            session.transfer(flowFile, REL_ORIGINAL);
+
+        } catch (final Exception e) {
+            logger.error("Failed in detecting duplicate records.", e);
+            session.remove(duplicatesFlowFile);
+            session.remove(nonDuplicatesFlowFile);
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    private void sendOrRemove(ProcessSession session,
+                              FlowFile outputFlowFile,
+                              Relationship targetRelationship,
+                              String mimeType,
+                              boolean includeZeroRecordFlowFiles,
+                              WriteResult writeResult) {
+        if(!includeZeroRecordFlowFiles && writeResult.getRecordCount() == 0) {
+            session.remove(outputFlowFile);
+        } else {
+            Map<String, String> attributes = new HashMap<>();
+            attributes.putAll(writeResult.getAttributes());
+            attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+            attributes.put(CoreAttributes.MIME_TYPE.key(), mimeType);
+            outputFlowFile = session.putAllAttributes(outputFlowFile, attributes);

Review comment:
       This might be too much for INFO level logging




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-820737577


   I think it is close to being done, but it fell off everyone's radars (including @adamfisher). @ottobackwards  you can pick up the review now if @mattyb149 is low on time :-D


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r817088376



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))

Review comment:
       `reset()` does something equivalent so I went with that.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1059125322


   @exceptionfactory @mattyb149 any chance we could close this out?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 closed pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
mattyb149 closed pull request #4646:
URL: https://github.com/apache/nifi/pull/4646


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r614437286



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DeduplicateRecords/additionalDetails.html
##########
@@ -0,0 +1,53 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>DeduplicateRecords</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<!-- Processor Documentation ================================================== -->
+    <h1>Overview</h1>
+    <p>This processor provides deduplication across either a single record set file or across an entire data lake
+    using a DistributedMapCacheClient controller service. In the case of the former, it uses either a HashSet or a bloom
+    filter to provide extremely fast in-memory calculations with a high degree of accuracy. In the latter use case, it
+    will use the controller service to compare a generated hash against a map cache stored in one of the supported caching
+    options that Apache NiFi offers.</p>
+
+    <h2>Configuring single file deduplication</h2>
+    <p>Choose the "single file" option under the configuration property labeled "Deduplication Strategy." Then choose

Review comment:
       I think you should use specific property names through out this document




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r817076550



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Multiple files deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        String strategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue();
+
+        useInMemoryStrategy = strategy.equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    public static final char JOIN_CHAR = '~';
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);
+
+            final RecordReader reader = readerFactory.createRecordReader(flowFile, inputStream, logger);
+
+            final RecordSchema writeSchema = writerFactory.getSchema(flowFile.getAttributes(), reader.getSchema());
+
+            final OutputStream nonDupeStream = session.write(nonDuplicatesFlowFile);
+            final OutputStream dupeStream = session.write(duplicatesFlowFile);
+
+            final RecordSetWriter nonDuplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, nonDupeStream, nonDuplicatesFlowFile);
+            final RecordSetWriter duplicatesWriter = writerFactory.createWriter(getLogger(), writeSchema, dupeStream, duplicatesFlowFile);
+
+            nonDuplicatesWriter.beginRecordSet();
+            duplicatesWriter.beginRecordSet();
+            Record record;
+
+            long index = 0;
+            while ((record = reader.nextRecord()) != null) {
+                String recordValue;
+
+                if (matchWholeRecord) {
+                    recordValue = Joiner.on(JOIN_CHAR).join(record.getValues());
+                } else {
+                    recordValue = executeDynamicRecordPaths(context, record, flowFile);
+                }
+
+                String recordHash = messageDigest != null
+                        ? Hex.encodeHexString(messageDigest.digest(getBytesUtf8(recordValue)))
+                        : recordValue;
+
+                if (!useInMemoryStrategy && context.getProperty(CACHE_IDENTIFIER).isSet()) {
+                    Map<String, String> additional = new HashMap<>();
+                    additional.put("record.hash.value", recordHash);
+                    String rawPath = context.getProperty(CACHE_IDENTIFIER).evaluateAttributeExpressions(flowFile, additional).getValue();
+                    RecordPath compiled = recordPathCache.getCompiled(rawPath);
+                    RecordPathResult result = compiled.evaluate(record);
+                    FieldValue fieldValue = result.getSelectedFields().findFirst().get();
+                    if (fieldValue.getValue() == null) {
+                        throw new ProcessException(String.format("The path \"%s\" failed to create an ID value at record index %d", rawPath, index));
+                    }
+
+                    recordHash = fieldValue.getValue().toString();
+                }
+
+                if (filter.contains(recordHash)) {
+                    duplicatesWriter.write(record);
+                } else {
+                    nonDuplicatesWriter.write(record);
+                    filter.put(recordHash);
+                }
+
+                index++;
+            }
+
+            // Route Non-Duplicates FlowFile
+            final WriteResult nonDuplicatesWriteResult = nonDuplicatesWriter.finishRecordSet();
+            // Route Duplicates FlowFile
+            final WriteResult duplicatesWriteResult = duplicatesWriter.finishRecordSet();
+
+            reader.close();
+            inputStream.close();
+            nonDuplicatesWriter.close();
+            nonDupeStream.close();
+            duplicatesWriter.close();
+            dupeStream.close();
+
+            final boolean includeZeroRecordFlowFiles = context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean();
+
+            session.adjustCounter("Records Processed",
+                    nonDuplicatesWriteResult.getRecordCount() + duplicatesWriteResult.getRecordCount(), false);
+
+            sendOrRemove(session, duplicatesFlowFile, REL_DUPLICATE, duplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, duplicatesWriteResult);
+
+            sendOrRemove(session, nonDuplicatesFlowFile, REL_NON_DUPLICATE, nonDuplicatesWriter.getMimeType(),
+                    includeZeroRecordFlowFiles, nonDuplicatesWriteResult);
+
+            session.transfer(flowFile, REL_ORIGINAL);
+
+        } catch (final Exception e) {
+            logger.error("Failed in detecting duplicate records.", e);
+            session.remove(duplicatesFlowFile);
+            session.remove(nonDuplicatesFlowFile);
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    private void sendOrRemove(ProcessSession session,
+                              FlowFile outputFlowFile,
+                              Relationship targetRelationship,
+                              String mimeType,
+                              boolean includeZeroRecordFlowFiles,
+                              WriteResult writeResult) {
+        if(!includeZeroRecordFlowFiles && writeResult.getRecordCount() == 0) {
+            session.remove(outputFlowFile);
+        } else {
+            Map<String, String> attributes = new HashMap<>();
+            attributes.putAll(writeResult.getAttributes());
+            attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+            attributes.put(CoreAttributes.MIME_TYPE.key(), mimeType);
+            outputFlowFile = session.putAllAttributes(outputFlowFile, attributes);
+            if (getLogger().isDebugEnabled()) {
+                getLogger().debug("Successfully found {} unique records for {}",
+                        new Object[]{writeResult.getRecordCount(), outputFlowFile});
+            }

Review comment:
       Going to leave the isDebugEnabled in there because while the object creation is cheap, it's also unnecessary.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1055589508


   Good feedback. Will incorporate.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] joewitt commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1063106976


   Please don't use plural form on processor naming.
   
   DeduplicateRecord


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1054411116


   Revisiting this. @ottobackwards @mattyb149 @exceptionfactory going to work through any remaining comments and see if we can close this out this week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r529984319



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");

Review comment:
       Good point.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-825221203


   Thanks @ottobackwards. I'll try to make some time to knock these out tomorrow.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r529979791



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,

Review comment:
       I don't think any of the supported algorithms are likely to be that bad, but might want to put a note in case someone plans to go wild with SHA-512.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r531615707



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",

Review comment:
       Maybe I just have "holiday brain," but I'm not seeing the typo.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r531612436



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "

Review comment:
       It would apply in two cases:
   
   1. Every input record was a duplicate.
   2. There are absolutely not duplicates.
   
   In either of those cases, you'd have a 0 record record set.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ottobackwards commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
ottobackwards commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-820784335


   Left a couple of small comments.  Also, I think for maintainability, it would be nice if there were some comments and javadoc in the processor, as to the overall logic/process, and what the methods are doing / returning.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r816943197



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");

Review comment:
       Are these reset between test classes? If not we should save off the current property values and restore them in an `@AfterClass`. Also consider upgrading to JUnit 5

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDeduplicateRecords.java
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.controller.AbstractControllerService;
+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.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeduplicateRecords {
+
+    private TestRunner runner;
+    private MockRecordParser reader;
+    private MockRecordWriter writer;
+
+    @BeforeClass
+    public static void beforeClass() {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DeduplicateRecords", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestDeduplicateRecords", "debug");
+    }
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(DeduplicateRecords.class);
+
+        // RECORD_READER, RECORD_WRITER
+        reader = new MockRecordParser();
+        writer = new MockRecordWriter("header", false);
+
+        runner.addControllerService("reader", reader);
+        runner.enableControllerService(reader);
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        runner.setProperty(DeduplicateRecords.RECORD_READER, "reader");
+        runner.setProperty(DeduplicateRecords.RECORD_WRITER, "writer");
+        runner.setProperty(DeduplicateRecords.RECORD_HASHING_ALGORITHM, DeduplicateRecords.SHA1_ALGORITHM_VALUE);
+
+        reader.addSchemaField("firstName", RecordFieldType.STRING);
+        reader.addSchemaField("middleName", RecordFieldType.STRING);
+        reader.addSchemaField("lastName", RecordFieldType.STRING);
+
+        // INCLUDE_ZERO_RECORD_FLOWFILES
+        runner.setProperty(DeduplicateRecords.INCLUDE_ZERO_RECORD_FLOWFILES, "true");
+
+        runner.assertValid();
+    }
+
+    void commonEnqueue() {
+        final Map<String, String> props = new HashMap<>();
+        props.put("hash.value", "1000");
+        runner.enqueue(new byte[]{}, props);
+    }
+
+    @Test
+    public void testDetectDuplicatesHashSet() {
+        commonEnqueue();
+
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testDetectDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    @Test
+    public void testNoDuplicatesHashSet() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.HASH_SET_VALUE);
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testNoDuplicatesBloomFilter() {
+        commonEnqueue();
+        runner.setProperty(DeduplicateRecords.FILTER_TYPE, DeduplicateRecords.BLOOM_FILTER_VALUE);
+        runner.setProperty(DeduplicateRecords.BLOOM_FILTER_FPP, "0.10");
+        runner.setProperty("/middleName", "${field.value}");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testAllDuplicates() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("John", "Q", "Smith");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 2);
+    }
+
+    @Test
+    public void testAllUnique() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 3, 0);
+    }
+
+    @Test
+    public void testCacheValueFromRecordPath() {
+        commonEnqueue();
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 2, 1);
+    }
+
+    /*
+     * These are all related to NIFI-6014
+     */
+
+    @Test
+    public void testMultipleFileDeduplicationRequiresDMC() {
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_MULTIPLE_FILES.getValue());
+        runner.assertNotValid();
+    }
+
+    public static final String FIRST_KEY = "2875ba79836587028a920875a18ee5dceb837587";
+    public static final String SECOND_KEY = "6eeba6ecf9d263582f463890be339dbecbaf23c8";
+
+    @Test
+    public void testDeduplicateWithDMC() throws Exception {
+        DistributedMapCacheClient dmc = new MockCacheService<>();
+        runner.addControllerService("dmc", dmc);
+        runner.setProperty(DeduplicateRecords.DISTRIBUTED_MAP_CACHE, "dmc");
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_MULTIPLE_FILES.getValue());
+        runner.enableControllerService(dmc);
+        runner.assertValid();
+
+        dmc.put(FIRST_KEY, true, null, null);
+        dmc.put(SECOND_KEY, true, null, null);
+
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        runner.enqueue("");
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 3);
+    }
+
+    @Test
+    public void testDeduplicateWithDMCAndCacheIdentifier() throws Exception {
+        DistributedMapCacheClient dmc = new MockCacheService<>();
+        runner.addControllerService("dmc", dmc);
+        runner.setProperty(DeduplicateRecords.DISTRIBUTED_MAP_CACHE, "dmc");
+        runner.setProperty(DeduplicateRecords.DEDUPLICATION_STRATEGY, DeduplicateRecords.OPTION_MULTIPLE_FILES.getValue());
+        runner.setProperty(DeduplicateRecords.CACHE_IDENTIFIER, "concat('${user.name}', '${record.hash.value}')");
+        runner.enableControllerService(dmc);
+        runner.assertValid();
+
+        dmc.put(String.format("john.smith-%s", FIRST_KEY), true, null, null);
+        dmc.put(String.format("john.smith-%s", SECOND_KEY), true, null, null);
+
+        reader.addRecord("John", "Q", "Smith");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jack", "Z", "Brown");
+        reader.addRecord("Jane", "X", "Doe");
+
+        Map<String, String> attrs = new HashMap<>();
+        attrs.put("user.name", "john.smith-");
+
+        runner.enqueue("", attrs);
+        runner.run();
+
+        doCountTests(0, 1, 1, 1, 1, 3);
+    }
+
+    void doCountTests(int failure, int original, int duplicates, int notDuplicates, int notDupeCount, int dupeCount) {
+        runner.assertTransferCount(DeduplicateRecords.REL_DUPLICATE, duplicates);
+        runner.assertTransferCount(DeduplicateRecords.REL_NON_DUPLICATE, notDuplicates);
+        runner.assertTransferCount(DeduplicateRecords.REL_ORIGINAL, original);
+        runner.assertTransferCount(DeduplicateRecords.REL_FAILURE, failure);
+
+        List<MockFlowFile> duplicateFlowFile = runner.getFlowFilesForRelationship(DeduplicateRecords.REL_DUPLICATE);
+        if (duplicateFlowFile != null) {
+            assertEquals(String.valueOf(dupeCount), duplicateFlowFile.get(0).getAttribute("record.count"));
+        }
+
+        List<MockFlowFile> nonDuplicateFlowFile = runner.getFlowFilesForRelationship(DeduplicateRecords.REL_NON_DUPLICATE);
+        if (nonDuplicateFlowFile != null) {
+            assertEquals(String.valueOf(notDupeCount), nonDuplicateFlowFile.get(0).getAttribute("record.count"));
+        }
+    }
+
+    private static final class MockCacheService<K, V> extends AbstractControllerService implements DistributedMapCacheClient {

Review comment:
       Maybe a unit test illustrating a cache failure to show some non-happy-path scenarios(s)?

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")

Review comment:
       The doc for this and the dynamic properties should explain in more detail how they work together. The dependent property stuff helps but you can't really tell about the dynamic properties until you add one.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,675 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availble: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_DATA_LAKE = new AllowableValue("dataLake", "Data Lake");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter data lake " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_DATA_LAKE)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'across data lake.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA1_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathPropertyNameValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext context) {
+        RecordPathValidator recordPathValidator = new RecordPathValidator();
+        final List<ValidationResult> validationResults = context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(property -> recordPathValidator.validate(
+                        "User-defined Properties",
+                        property.getName(),
+                        context
+                )).collect(Collectors.toList());
+
+        boolean useSingleFile = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+
+        if (useSingleFile && context.getProperty(BLOOM_FILTER_FPP).isSet()) {
+            final double falsePositiveProbability = context.getProperty(BLOOM_FILTER_FPP).asDouble();
+            if (falsePositiveProbability < 0 || falsePositiveProbability > 1) {
+                validationResults.add(
+                        new ValidationResult.Builder()
+                                .subject(BLOOM_FILTER_FPP.getName() + " out of range.")
+                                .input(String.valueOf(falsePositiveProbability))
+                                .explanation("Valid values are 0.0 - 1.0 inclusive")
+                                .valid(false).build());
+            }
+        } else if (!useSingleFile) {
+            if (!context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+                validationResults.add(new ValidationResult.Builder()
+                        .subject(DISTRIBUTED_MAP_CACHE.getName())
+                        .explanation("Data lake-level deduplication was chosen, but a distributed map cache client was " +
+                                "not configured")
+                        .valid(false).build());
+            }
+        }
+
+        return validationResults;
+    }
+
+    private DistributedMapCacheClient mapCacheClient;
+    private RecordReaderFactory readerFactory;
+    private RecordSetWriterFactory writerFactory;
+
+    private boolean useInMemoryStrategy;
+
+    @OnScheduled
+    public void compileRecordPaths(final ProcessContext context) {
+        final List<String> recordPaths = new ArrayList<>();
+
+        recordPaths.addAll(context.getProperties().keySet().stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .map(PropertyDescriptor::getName)
+                .collect(toList()));
+
+        recordPathCache = new RecordPathCache(recordPaths.size());
+        this.recordPaths = recordPaths;
+
+        if (context.getProperty(DISTRIBUTED_MAP_CACHE).isSet()) {
+            mapCacheClient = context.getProperty(DISTRIBUTED_MAP_CACHE).asControllerService(DistributedMapCacheClient.class);
+        }
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        useInMemoryStrategy = context.getProperty(DEDUPLICATION_STRATEGY).getValue().equals(OPTION_SINGLE_FILE.getValue());
+    }
+
+    private FilterWrapper getFilter(ProcessContext context) {
+        if (useInMemoryStrategy) {
+            boolean useHashSet = context.getProperty(FILTER_TYPE).getValue()
+                    .equals(context.getProperty(HASH_SET_VALUE.getValue()));
+            final int filterCapacity = context.getProperty(FILTER_CAPACITY_HINT).asInteger();
+            return useHashSet
+                ? new HashSetFilterWrapper(new HashSet<>(filterCapacity))
+                : new BloomFilterWrapper(BloomFilter.create(
+                    Funnels.stringFunnel(Charset.defaultCharset()),
+                    filterCapacity,
+                    context.getProperty(BLOOM_FILTER_FPP).asDouble()
+                ));
+        } else {
+            return new DistributedMapCacheClientWrapper(mapCacheClient);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+
+        FlowFile nonDuplicatesFlowFile = session.create(flowFile);
+        FlowFile duplicatesFlowFile = session.create(flowFile);
+
+        try {
+            final long now = System.currentTimeMillis();
+
+            final FilterWrapper filter = getFilter(context);
+
+            final String recordHashingAlgorithm = context.getProperty(RECORD_HASHING_ALGORITHM).getValue();
+            final MessageDigest messageDigest = recordHashingAlgorithm.equals(NONE_ALGORITHM_VALUE.getValue())
+                    ? null
+                    : DigestUtils.getDigest(recordHashingAlgorithm);
+            final Boolean matchWholeRecord = context.getProperties().keySet().stream().noneMatch(p -> p.isDynamic());
+
+            final InputStream inputStream = session.read(flowFile);

Review comment:
       I think the try-with-resources is important here, currently if there is an error creating/using the reader/writer, the processor fails with an `InputStream not closed` error and rolls back.

##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")

Review comment:
       Maybe mention that this is dependent on `Single File`, and for the sake of the Usage doc (since it will be moot once a dynamic property is added to the dialog) maybe mention what happens if no dynamic property is provided.
   
   I was also getting some weird behavior with dynamic properties. It looks like if no dynamic properties are provided, all fields are used to determine duplication. If (in my example with 4 records) all records were exactly the same, it sent 1 to non-dupe and 3 to dupe. If none were exactly the same (even if some had individually identical fields like lastName), it sent all to non-dupe. If I set a dynamic property `/lastName` to `Doe` (using your sample data), the `John Smith` record was sent to non-dupe and the others (including the one with lastName = Doe) to dupe. If I awr `/lastName` to `Smith` it did the exact same thing. Only when I set `/lastName` to `${field.value}` did it behave as expected (the first `Smith` and the `Doe` to non-dupe and the second `Smith` to dupe).
   
   Also when I put in a record path for a missing field `/lastname` (lowercased), it behaved as if none had been provided. I wonder if we want to have an error/failure condition there as the user may not be aware that the behavior will not be as expected.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on a change in pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on a change in pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#discussion_r817134950



##########
File path: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DeduplicateRecords.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.hash.BloomFilter;
+import com.google.common.hash.Funnels;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.MessageDigestAlgorithms;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+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.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+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.Validator;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathPropertyNameValidator;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Collection;
+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 static java.util.stream.Collectors.toList;
+import static org.apache.commons.codec.binary.StringUtils.getBytesUtf8;
+
+@EventDriven
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@SystemResourceConsideration(resource = SystemResource.MEMORY,
+    description = "The HashSet filter type will grow memory space proportionate to the number of unique records processed. " +
+        "The BloomFilter type will use constant memory regardless of the number of records processed.")
+@SystemResourceConsideration(resource = SystemResource.CPU,
+    description =  "If a more advanced hash algorithm is chosen, the amount of time required to hash any particular " +
+            "record could increase substantially."
+)
+@Tags({"text", "record", "update", "change", "replace", "modify", "distinct", "unique",
+    "filter", "hash", "dupe", "duplicate", "dedupe"})
+@CapabilityDescription("This processor attempts to deduplicate a record set in memory using either a hashset or a bloom filter. " +
+        "It operates on a per-file basis rather than across an entire data set that spans multiple files.")
+@WritesAttribute(attribute = "record.count", description = "The number of records processed.")
+@DynamicProperty(
+    name = "RecordPath",
+    value = "An expression language statement used to determine how the RecordPath is resolved. " +
+            "The following variables are availible: ${field.name}, ${field.value}, ${field.type}",
+    description = "The name of each user-defined property must be a valid RecordPath.")
+@SeeAlso(classNames = {
+    "org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService",
+    "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer",
+    "org.apache.nifi.processors.standard.DetectDuplicate"
+})
+public class DeduplicateRecords extends AbstractProcessor {
+
+    private static final String FIELD_NAME = "field.name";
+    private static final String FIELD_VALUE = "field.value";
+    private static final String FIELD_TYPE = "field.type";
+
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<String> recordPaths;
+
+    // VALUES
+
+    static final AllowableValue NONE_ALGORITHM_VALUE = new AllowableValue("none", "None",
+            "Do not use a hashing algorithm. The value of resolved RecordPaths will be combined with tildes (~) to form the unique record key. " +
+                    "This may use significantly more storage depending on the size and shape or your data.");
+    static final AllowableValue MD5_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.MD5, "MD5",
+            "The MD5 message-digest algorithm.");
+    static final AllowableValue SHA1_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA_1, "SHA-1",
+            "The SHA-1 cryptographic hash algorithm.");
+    static final AllowableValue SHA256_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_256, "SHA-256",
+            "The SHA-256 cryptographic hash algorithm.");
+    static final AllowableValue SHA512_ALGORITHM_VALUE = new AllowableValue(MessageDigestAlgorithms.SHA3_512, "SHA-512",
+            "The SHA-512 cryptographic hash algorithm.");
+
+    static final AllowableValue HASH_SET_VALUE = new AllowableValue("hash-set", "HashSet",
+            "Exactly matches records seen before with 100% accuracy at the expense of more storage usage. " +
+                    "Stores the filter data in a single cache entry in the distributed cache, and is loaded entirely into memory during duplicate detection. " +
+                    "This filter is preferred for small to medium data sets and offers high performance  loaded into memory when this processor is running.");
+    static final AllowableValue BLOOM_FILTER_VALUE = new AllowableValue("bloom-filter", "BloomFilter",
+            "Space-efficient data structure ideal for large data sets using probability to determine if a record was seen previously. " +
+                    "False positive matches are possible, but false negatives are not – in other words, a query returns either \"possibly in the set\" or \"definitely not in the set\". " +
+                    "You should use this option if the FlowFile content is large and you can tolerate some duplication in the data. Uses constant storage space regardless of the record set size.");
+
+    // PROPERTIES
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Specifies the Controller Service to use for writing out the records")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final AllowableValue OPTION_SINGLE_FILE = new AllowableValue("single", "Single File");
+    static final AllowableValue OPTION_MULTIPLE_FILES = new AllowableValue("multiple", "Multiple Files");
+
+    static final PropertyDescriptor DEDUPLICATION_STRATEGY = new PropertyDescriptor.Builder()
+            .name("deduplication-strategy")
+            .displayName("Deduplication Strategy")
+            .description("The strategy to use for detecting and isolating duplicate records. The option for doing it " +
+                    "across a single data file will operate in memory, whereas the one for going across the enter repository " +
+                    "will require a distributed map cache.")
+            .allowableValues(OPTION_SINGLE_FILE, OPTION_MULTIPLE_FILES)
+            .defaultValue(OPTION_SINGLE_FILE.getValue())
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor DISTRIBUTED_MAP_CACHE = new PropertyDescriptor.Builder()
+            .name("distributed-map-cache")
+            .displayName("Distributed Map Cache client")
+            .description("This configuration is required when the deduplication strategy is set to 'multiple files.' The map " +
+                    "cache will be used to check a data source such as HBase or Redis for entries indicating that a record has " +
+                    "been processed before. This option requires a downstream process that uses PutDistributedMapCache to write " +
+                    "an entry to the cache data source once the record has been processed to indicate that it has been handled before.")
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor CACHE_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("cache-identifier")
+            .displayName("Cache Identifier")
+            .description("This option defines a record path operation to use for defining the cache identifier. It can be used " +
+                    "in addition to the hash settings. This field will have the expression language attribute \"record.hash.value\" " +
+                    "available to it to use with it to generate the record path operation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(Validator.VALID)
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_MULTIPLE_FILES)
+            .build();
+
+    static final PropertyDescriptor INCLUDE_ZERO_RECORD_FLOWFILES = new PropertyDescriptor.Builder()
+            .name("include-zero-record-flowfiles")
+            .displayName("Include Zero Record FlowFiles")
+            .description("When converting an incoming FlowFile, if the conversion results in no data, "
+                    + "this property specifies whether or not a FlowFile will be sent to the corresponding relationship")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_HASHING_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("record-hashing-algorithm")
+            .displayName("Record Hashing Algorithm")
+            .description("The algorithm used to hash the combined set of resolved RecordPath values for cache storage.")
+            .allowableValues(
+                    NONE_ALGORITHM_VALUE,
+                    MD5_ALGORITHM_VALUE,
+                    SHA1_ALGORITHM_VALUE,
+                    SHA256_ALGORITHM_VALUE,
+                    SHA512_ALGORITHM_VALUE
+            )
+            .defaultValue(SHA256_ALGORITHM_VALUE.getValue())
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_TYPE = new PropertyDescriptor.Builder()
+            .name("filter-type")
+            .displayName("Filter Type")
+            .description("The filter used to determine whether a record has been seen before based on the matching RecordPath criteria.")
+            .allowableValues(
+                    HASH_SET_VALUE,
+                    BLOOM_FILTER_VALUE
+            )
+            .defaultValue(HASH_SET_VALUE.getValue())
+            .dependsOn(DEDUPLICATION_STRATEGY, OPTION_SINGLE_FILE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor FILTER_CAPACITY_HINT = new PropertyDescriptor.Builder()
+            .name("filter-capacity-hint")
+            .displayName("Filter Capacity Hint")
+            .description("An estimation of the total number of unique records to be processed. " +
+                    "The more accurate this number is will lead to fewer false negatives on a BloomFilter.")
+            .defaultValue("25000")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .dependsOn(FILTER_TYPE, BLOOM_FILTER_VALUE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BLOOM_FILTER_FPP = new PropertyDescriptor.Builder()
+            .name("bloom-filter-certainty")
+            .displayName("Bloom Filter Certainty")
+            .description("The desired false positive probability when using the BloomFilter type. " +
+                    "Using a value of .05 for example, guarantees a five-percent probability that the result is a false positive. " +
+                    "The closer to 1 this value is set, the more precise the result at the expense of more storage space utilization.")
+            .defaultValue("0.10")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .required(false)
+            .build();
+
+
+    // RELATIONSHIPS
+
+    static final Relationship REL_DUPLICATE = new Relationship.Builder()
+            .name("duplicate")
+            .description("Records detected as duplicates in the FlowFile content will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_NON_DUPLICATE = new Relationship.Builder()
+            .name("non-duplicate")
+            .description("If the record was not found in the cache, it will be routed to this relationship")
+            .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input FlowFile is sent to this relationship unless there is a fatal error in the processing.")
+            .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If unable to communicate with the cache, the FlowFile will be penalized and routed to this relationship")
+            .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(DEDUPLICATION_STRATEGY);
+        descriptors.add(DISTRIBUTED_MAP_CACHE);
+        descriptors.add(CACHE_IDENTIFIER);
+        descriptors.add(RECORD_READER);
+        descriptors.add(RECORD_WRITER);
+        descriptors.add(INCLUDE_ZERO_RECORD_FLOWFILES);
+        descriptors.add(RECORD_HASHING_ALGORITHM);
+        descriptors.add(FILTER_TYPE);
+        descriptors.add(FILTER_CAPACITY_HINT);
+        descriptors.add(BLOOM_FILTER_FPP);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_DUPLICATE);
+        relationships.add(REL_NON_DUPLICATE);
+        relationships.add(REL_ORIGINAL);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .description("Specifies a value to use from the record that matches the RecordPath: '" +
+                        propertyDescriptorName + "' which is used together with other specified " +
+                        "record path values to determine the uniqueness of a record. " +
+                        "Expression Language may reference variables 'field.name', 'field.type', and 'field.value' " +
+                        "to access information about the field and the value of the field being evaluated.")

Review comment:
       You brought up a good point about the EL stuff there. I think that's something I missed when merging @adamfisher's PR with mine. Now that I think about it, I think the better way to do this property stuff would be to do it like this:
   
   <human_readable> => record path
   
   Ex. `firstName => /firstName`
   
   That would make it easier to say "hey, we didn't find firstName" instead of throwing a record path statement (which could be long) into the error logs.
   
   EL should only be supported on the cache identifier. I can't honestly think of a good use case to give users the flexibility to dynamic define property values on this part with EL. Thoughts?
   
   @exceptionfactory what do you think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] MikeThomsen commented on pull request #4646: NIFI-6047 Add DeduplicateRecords (combines 6047 and 6014)

Posted by GitBox <gi...@apache.org>.
MikeThomsen commented on pull request #4646:
URL: https://github.com/apache/nifi/pull/4646#issuecomment-1055628577


   @mattyb149
   
   > I had to enable the rest of the flow so the cache values would be populated by the PutDistributedMapCache processor. Is that intended? If so the docs should reflect that and if not, the processor itself should handle writing the cache identifier so no additional processor is needed, or perhaps it would at least be configurable.
   
   I'll update the docs. We can't have this processor updating the DMC because otherwise it's going to be such a thing upstream of the operations that need to complete before a DMC entry is written.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org