This is an automated email from the ASF dual-hosted git repository.
mattyb149 pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git
The following commit(s) were added to refs/heads/main by this push:
new 930e8d9 NIFI-7698: Added a merge.reason attribute for flowfiles
indicating why they were merged by MergeContent. Also updated logs to indicate
the reason and added additional documentation for processor
930e8d9 is described below
commit 930e8d9e0e59ff24f50d7e3a052870b1112f076a
Author: Mark Payne <[email protected]>
AuthorDate: Tue Jan 19 14:10:08 2021 -0500
NIFI-7698: Added a merge.reason attribute for flowfiles indicating why they
were merged by MergeContent. Also updated logs to indicate the reason and added
additional documentation for processor
Signed-off-by: Matthew Burgess <[email protected]>
This closes #4764
---
.../org/apache/nifi/processor/util/bin/Bin.java | 35 ++++-
.../apache/nifi/processor/util/bin/BinFiles.java | 25 +--
.../apache/nifi/processor/util/bin/BinManager.java | 15 +-
.../nifi/processor/util/bin/EvictionReason.java | 45 ++++++
.../nifi/processors/standard/MergeContent.java | 68 ++++----
.../additionalDetails.html | 171 +++++++++++++++++++++
.../additionalDetails.html | 22 +--
7 files changed, 319 insertions(+), 62 deletions(-)
diff --git
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/Bin.java
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/Bin.java
index f95c470..76175c9 100644
---
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/Bin.java
+++
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/Bin.java
@@ -16,6 +16,12 @@
*/
package org.apache.nifi.processor.util.bin;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.FragmentAttributes;
+import org.apache.nifi.processor.ProcessSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
@@ -24,12 +30,6 @@ import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.FragmentAttributes;
-import org.apache.nifi.processor.ProcessSession;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
/**
* Note: {@code Bin} objects are NOT thread safe. If multiple threads access a
{@code Bin}, the caller must synchronize
* access.
@@ -47,6 +47,7 @@ public class Bin {
private volatile int minimumEntries = 0;
private volatile int maximumEntries = Integer.MAX_VALUE;
private final String fileCountAttribute;
+ private volatile EvictionReason evictionReason = EvictionReason.UNSET;
private final List<FlowFile> binContents = new ArrayList<>();
private final Set<String> binIndexSet = new HashSet<>();
@@ -93,6 +94,20 @@ public class Bin {
|| (size >= maximumSizeBytes) || (binContents.size() >=
maximumEntries);
}
+ public EvictionReason determineFullness() {
+ if (size >= maximumSizeBytes) {
+ return EvictionReason.MAX_BYTES_THRESHOLD_REACHED;
+ }
+ if (binContents.size() >= maximumEntries) {
+ return EvictionReason.MAX_ENTRIES_THRESHOLD_REACHED;
+ }
+ if (size >= minimumSizeBytes && binContents.size() >= minimumEntries) {
+ return EvictionReason.MIN_THRESHOLDS_REACHED;
+ }
+
+ return null;
+ }
+
/**
* Indicates enough size exists to meet the minimum requirements
*
@@ -124,6 +139,14 @@ public class Bin {
return creationMomentEpochNs < other.creationMomentEpochNs;
}
+ public EvictionReason getEvictionReason() {
+ return evictionReason;
+ }
+
+ public void setEvictionReason(final EvictionReason evictionReason) {
+ this.evictionReason = evictionReason;
+ }
+
/**
* If this bin has enough room for the size of the given flow file then it
is added otherwise it is not
*
diff --git
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
index 7851654..583e8ff 100644
---
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
+++
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java
@@ -16,16 +16,6 @@
*/
package org.apache.nifi.processor.util.bin;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
@@ -42,6 +32,17 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
/**
* Base class for file-binning processors.
*
@@ -209,9 +210,11 @@ public abstract class BinFiles extends
AbstractSessionFactoryProcessor {
final Bin bin = binManager.removeOldestBin();
if (bin != null) {
added++;
+ bin.setEvictionReason(EvictionReason.BIN_MANAGER_FULL);
this.readyBins.add(bin);
}
}
+
return added;
}
@@ -242,7 +245,7 @@ public abstract class BinFiles extends
AbstractSessionFactoryProcessor {
// If this bin's session has been committed, move on.
if (!binProcessingResult.isCommitted()) {
final ProcessSession binSession = bin.getSession();
- bin.getContents().stream().forEach(ff ->
binSession.putAllAttributes(ff, binProcessingResult.getAttributes()));
+ bin.getContents().forEach(ff ->
binSession.putAllAttributes(ff, binProcessingResult.getAttributes()));
binSession.transfer(bin.getContents(), REL_ORIGINAL);
binSession.commit();
}
diff --git
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinManager.java
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinManager.java
index 60c2966..74356be 100644
---
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinManager.java
+++
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/BinManager.java
@@ -16,6 +16,10 @@
*/
package org.apache.nifi.processor.util.bin;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
@@ -30,10 +34,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.ProcessSessionFactory;
-
/**
* This class is thread safe
*
@@ -222,9 +222,14 @@ public class BinManager {
for (final Map.Entry<String, List<Bin>> group :
groupBinMap.entrySet()) {
final List<Bin> remainingBins = new ArrayList<>();
for (final Bin bin : group.getValue()) {
- if (relaxFullnessConstraint && (bin.isFullEnough() ||
bin.isOlderThan(maxBinAgeSeconds.get(), TimeUnit.SECONDS))) { //relaxed check
+ if (relaxFullnessConstraint && bin.isFullEnough()) {
+ bin.setEvictionReason(bin.determineFullness());
readyBins.add(bin);
} else if (!relaxFullnessConstraint && bin.isFull()) {
//strict check
+ bin.setEvictionReason(bin.determineFullness());
+ readyBins.add(bin);
+ } else if (relaxFullnessConstraint &&
bin.isOlderThan(maxBinAgeSeconds.get(), TimeUnit.SECONDS)) {
+ bin.setEvictionReason(EvictionReason.TIMEOUT);
readyBins.add(bin);
} else { //it isn't time yet...
remainingBins.add(bin);
diff --git
a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/EvictionReason.java
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/EvictionReason.java
new file mode 100644
index 0000000..ddafc20
--- /dev/null
+++
b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/bin/EvictionReason.java
@@ -0,0 +1,45 @@
+/*
+ * 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.processor.util.bin;
+
+public enum EvictionReason {
+ MAX_BYTES_THRESHOLD_REACHED("Maximum number of bytes (Max Group Size)
reached"),
+
+ MAX_ENTRIES_THRESHOLD_REACHED("Maximum number of entries reached"),
+
+ MIN_THRESHOLDS_REACHED("Minimum number of bytes (Min Group Size) and
minimum number of entries reached"),
+
+ TIMEOUT("Max Bin Age reached"),
+
+ BIN_MANAGER_FULL("The oldest Bin was removed because incoming FlowFile
could not be placed in an existing Bin, and the Maximum Number of Bins was
reached"),
+
+ UNSET("No reason was determined");
+
+ private final String explanation;
+ EvictionReason(final String explanation) {
+ this.explanation = explanation;
+ }
+
+ public String getExplanation() {
+ return explanation;
+ }
+
+ public String toString() {
+ return explanation;
+ }
+}
diff --git
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index 4bb9d4d..d1deca5 100644
---
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -16,33 +16,6 @@
*/
package org.apache.nifi.processors.standard;
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipException;
-import java.util.zip.ZipOutputStream;
import org.apache.avro.Schema;
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileConstants;
@@ -96,6 +69,34 @@ import org.apache.nifi.util.FlowFilePackagerV1;
import org.apache.nifi.util.FlowFilePackagerV2;
import org.apache.nifi.util.FlowFilePackagerV3;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipException;
+import java.util.zip.ZipOutputStream;
+
@SideEffectFree
@TriggerWhenEmpty
@InputRequirement(Requirement.INPUT_REQUIRED)
@@ -134,7 +135,11 @@ import org.apache.nifi.util.FlowFilePackagerV3;
@WritesAttribute(attribute = "merge.count", description = "The number of
FlowFiles that were merged into this bundle"),
@WritesAttribute(attribute = "merge.bin.age", description = "The age of
the bin, in milliseconds, when it was merged and output. Effectively "
+ "this is the greatest amount of time that any FlowFile in this
bundle remained waiting in this processor before it was output"),
- @WritesAttribute(attribute = "merge.uuid", description = "UUID of the
merged flow file that will be added to the original flow files attributes.")})
+ @WritesAttribute(attribute = "merge.uuid", description = "UUID of the
merged flow file that will be added to the original flow files attributes."),
+ @WritesAttribute(attribute = "merge.reason", description = "This processor
allows for several thresholds to be configured for merging FlowFiles. This
attribute indicates which of the Thresholds" +
+ " resulted in the FlowFiles being merged. For an explanation of each
of the possible values and their meanings, see the Processor's Usage /
documentation and see the 'Additional Details' " +
+ "page.")
+})
@SeeAlso({SegmentContent.class, MergeRecord.class})
@SystemResourceConsideration(resource = SystemResource.MEMORY, description =
"While content is not stored in memory, the FlowFiles' attributes are. " +
"The configuration of MergeContent (maximum bin size, maximum group
size, maximum bin age, max number of entries) will influence how much " +
@@ -234,6 +239,7 @@ public class MergeContent extends BinFiles {
public static final String MERGE_COUNT_ATTRIBUTE = "merge.count";
public static final String MERGE_BIN_AGE_ATTRIBUTE = "merge.bin.age";
public static final String MERGE_UUID_ATTRIBUTE = "merge.uuid";
+ public static final String REASON_FOR_MERGING = "merge.reason";
public static final PropertyDescriptor MERGE_STRATEGY = new
PropertyDescriptor.Builder()
.name("Merge Strategy")
@@ -520,10 +526,14 @@ public class MergeContent extends BinFiles {
bundleAttributes.put(MERGE_COUNT_ATTRIBUTE,
Integer.toString(contents.size()));
bundleAttributes.put(MERGE_BIN_AGE_ATTRIBUTE,
Long.toString(bin.getBinAge()));
+ bundleAttributes.put(REASON_FOR_MERGING,
bin.getEvictionReason().name());
+
bundle = binSession.putAllAttributes(bundle, bundleAttributes);
final String inputDescription = contents.size() < 10 ?
contents.toString() : contents.size() + " FlowFiles";
- getLogger().info("Merged {} into {}", new Object[]{inputDescription,
bundle});
+
+ getLogger().info("Merged {} into {}. Reason for merging: {}", new
Object[] {inputDescription, bundle, bin.getEvictionReason()});
+
binSession.transfer(bundle, REL_MERGED);
binProcessingResult.getAttributes().put(MERGE_UUID_ATTRIBUTE,
bundle.getAttribute(CoreAttributes.UUID.key()));
diff --git
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html
new file mode 100644
index 0000000..de6ab84
--- /dev/null
+++
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html
@@ -0,0 +1,171 @@
+<!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>MergeContent</title>
+
+ <link rel="stylesheet" href="../../../../../css/component-usage.css"
type="text/css" />
+</head>
+
+<body>
+
+<h3>Introduction</h3>
+<p>
+ The MergeContent Processor provides the ability to combine many FlowFiles
into a single FlowFile. There are many reasons that a dataflow designer
+ may want to do this. For example, it may be helpful to create batches of
data before sending to a downstream system, because the downstream system
+ is better optimized for large files than for many tiny files. NiFi itself
can also benefit from this, as NiFi operates best on "micro-batches," where
+ each FlowFile is several kilobytes to several megabytes in size.
+</p>
+
+<p>
+ The Processor creates several 'bins' to put the FlowFiles in. The maximum
number of bins to use is set to 5 by default, but this can be changed
+ by updating the value of the <Maximum number of Bins> property. The
number of bins is bound in order to avoid running out of Java heap space.
+ Note: while the contents of a FlowFile are stored in the Content
Repository and not in the Java heap space, the Processor must hold the FlowFile
+ objects themselves in memory. As a result, these FlowFiles with their
attributes can potentially take up a great deal of heap space and cause
+ OutOfMemoryError's to be thrown. In order to avoid this, if you expect to
merge many small FlowFiles together, it is advisable to instead use a
+ MergeContent that merges no more than say 1,000 FlowFiles into a bundle
and then use a second MergeContent to merges these small bundles into
+ larger bundles. For example, to merge 1,000,000 FlowFiles together, use
MergeContent that uses a <Maximum Number of Entries> of 1,000 and route
the
+ "merged" Relationship to a second MergeContent that also sets the
<Maximum Number of Entries> to 1,000. The second MergeContent will then
merge 1,000 bundles
+ of 1,000, which in effect produces bundles of 1,000,000.
+</p>
+
+
+
+<h3>How FlowFiles are Binned</h3>
+<p>
+ How the Processor determines which bin to place a FlowFile in depends on a
few different configuration options. Firstly, the Merge Strategy
+ is considered. The Merge Strategy can be set to one of two options: "Bin
Packing Algorithm," or "Defragment". When the goal is to simply combine
+ smaller FlowFiles into one larger FlowFile, the Bin Packing Algorithm
should be used. This algorithm picks a bin based on whether or not the FlowFile
+ can fit in the bin according to its size and the <Maximum Bin Size>
property and whether or not the FlowFile is 'like' the other FlowFiles in
+ the bin. What it means for two FlowFiles to be 'like FlowFiles' is
discussed at the end of this section.
+</p>
+
+<p>
+ The "Defragment" Merge Strategy can be used when FlowFiles need to be
explicitly assigned to the same bin. For example, if data is split apart using
+ the UnpackContent Processor, each unpacked FlowFile can be processed
independently and later merged back together using this Processor with the
+ Merge Strategy set to Defragment. In order for FlowFiles to be added to
the same bin when using this configuration, the FlowFiles must have the same
+ value for the "fragment.identifier" attribute. Each FlowFile with the same
identifier must also have the same value for the "fragment.count" attribute
+ (which indicates how many FlowFiles belong in the bin) and a unique value
for the "fragment.index" attribute so that the FlowFiles can be ordered
+ correctly. <b>NOTE:</b> while there are valid use cases for breaking apart
FlowFiles and later re-merging them, it is an anti-pattern to take a larger
FlowFile,
+ break it into a million tiny FlowFiles, and then re-merge them. Doing so
can result in using huge amounts of Java heap and can result in Out Of Memory
Errors.
+ Additionally, it adds large amounts of load to the NiFi framework. This
can result in increased CPU and disk utilization and often times can be an
order of magnitude
+ lower throughput and an order of magnitude higher latency. As an
alternative, whenever possible, dataflows should be built to make use of
Record-oriented processors,
+ such as QueryRecord, PartitionRecord, UpdateRecord, LookupRecord,
PublishKafkaRecord_2_6, etc.
+</p>
+
+<p>
+ In order to be added to the same bin, two FlowFiles must be 'like
FlowFiles.' In order for two FlowFiles to be like FlowFiles, they must have the
same
+ schema, and if the <Correlation Attribute Name> property is set,
they must have the same value for the specified attribute. For example, if the
+ <Correlation Attribute Name> is set to "filename" then two FlowFiles
must have the same value for the "filename" attribute in order to be binned
+ together. If more than one attribute is needed in order to correlate two
FlowFiles, it is recommended to use an UpdateAttribute processor before the
+ MergeContent processor and combine the attributes. For example, if the
goal is to bin together two FlowFiles only if they have the same value for the
+ "abc" attribute and the "xyz" attribute, then we could accomplish this by
using UpdateAttribute and adding a property with name "correlation.attribute"
+ and a value of "abc=${abc},xyz=${xyz}" and then setting MergeContent's
<Correlation Attribute Name> property to "correlation.attribute".
+</p>
+
+
+
+<h3>When a Bin is Merged</h3>
+<p>
+ Above, we discussed how a bin is chosen for a given FlowFile. Once a bin
has been created and FlowFiles added to it, we must have some way to determine
+ when a bin is "full" so that we can bin those FlowFiles together into a
"merged" FlowFile. There are a few criteria that are used to make a
determination as
+ to whether or not a bin should be merged.
+</p>
+
+<p>
+ If the <Merge Strategy> property is set to "Bin Packing Algorithm"
then then the following rules will be evaluated.
+</p>
+
+<p>
+ MergeContent exposes several different thresholds that can be used to
create bins that are of an ideal size. For example, the user can specify the
minimum number of FlowFiles
+ that must be packaged together before merging will be performed. The
minimum number of bytes can also be configured. Additionally, a maximum number
of FlowFiles and bytes may be
+ specified.
+</p>
+
+<p>
+ There are also two other conditions that will result in the contents of a
Bin being merged together. The Max Bin Age property specifies the maximum
amount of time that FlowFiles can
+ be binned together before the bin is merged. This property should almost
always be set, as it provides a means to set a timeout on a bin, so that even
if data stops flowing to the Processor
+ for a while (due to a problem with an upstream system, a source processor
being stopped, etc.) the FlowFiles won't remain stuck in the MergeContent
processor indefinitely. Additionally,
+ the processor exposes a property for the maximum number of Bins that
should be used. For some use cases, this won't matter much. However, if the
Correlation Attribute property is set,
+ this can be important. When an incoming FlowFile is to be placed in a Bin,
the processor must find an appropriate Bin to place the FlowFile into, or else
create a new one. If a Bin must be
+ created, and the number of Bins that exist is greater than or equal to the
value of the <Maximum Number of Bins> property, then the oldest Bin will
be merged together to make room for
+ the new one.
+</p>
+
+<p>
+ If the <Merge Strategy> property is set to "Defragment" then a bin
is full only when the number of FlowFiles in the bin is equal to the number
specified
+ by the "fragment.count" attribute of one of the FlowFiles in the bin. All
FlowFiles that have this attribute must have the same value for this attribute,
+ or else they will be routed to the "failure" relationship. It is not
necessary that all FlowFiles have this value, but at least one FlowFile in the
bin must have
+ this value or the bin will never be complete. If all of the necessary
FlowFiles are not binned together by the point at which the bin times amount
+ (as specified by the <Max Bin Age> property), then the FlowFiles
will all be routed to the 'failure' relationship instead of being merged
together.
+</p>
+
+<p>
+ Whenever the contents of a Bin are merged, an attribute with the name
"merge.reason" will be added to the merged FlowFile. The below table provides a
listing of all possible values for this
+ attribute with an explanation of each.
+</p>
+
+<table>
+ <thead>
+ <tr>
+ <th>Attribute Value</th>
+ <th>Explanation</th>
+ </tr>
+ </thead>
+ <tbody>
+ <tr>
+ <td>MAX_BYTES_THRESHOLD_REACHED</td>
+ <td>The bin has reached the maximum number of bytes, as configured by
the <Max Group Size> property. When this threshold is reached, the
contents of the Bin will be merged
+ together, even if the Minimum Number of Entries has not yet been
reached.</td>
+ </tr>
+ <tr>
+ <td>MAX_ENTRIES_THRESHOLD_REACHED</td>
+ <td>The bin has reached the maximum number of FlowFiles, as configured
by the <Maximum Number of Entries> property. When this threshold is
reached, the contents of the Bin will
+ be merged together, even if the minimum number of bytes (Min Group
Size) has not yet been reached.</td>
+ </tr>
+ <tr>
+ <td>MIN_THRESHOLDS_REACHED</td>
+ <td>The bin has reached both the minimum number of bytes, as
configured by the <Min Group Size> property, AND the minimum number of
FlowFiles, as configured by the
+ <Minimum Number of Entries> property. The bin has not
reached the maximum number of bytes (Max Group Size) OR the maximum number of
FlowFiles (Maximum Number of Entries).</td>
+ </tr>
+ <tr>
+ <td>TIMEOUT</td>
+ <td>The Bin has reached the maximum age, as configured by the <Max
Bin Age> property. If this threshold is reached, the contents of the Bin
will be merged together, even if the
+ Bin has not yet reached either of the minimum thresholds. Note
that the age here is determined by when the Bin was created, NOT the age of the
FlowFiles that reside within those
+ Bins. As a result, if the Processor is stopped until it has 1
million FlowFiles queued, each one being 10 days old, but the Max Bin Age is
set to "1 day," the Max Bin Age will not
+ be met for at least one full day, even though the FlowFiles
themselves are much older than this threshold. If the Processor is stopped and
restarted, all Bins are destroyed and
+ recreated, so the timer is reset.</td>
+ </tr>
+ <tr>
+ <td>BIN_MANAGER_FULL</td>
+ <td>If an incoming FlowFile does not fit into any of the existing Bins
(either due to the Maximum thresholds set, or due to the Correlation Attribute
being used, etc.) then a new Bin
+ must be created for the incoming FlowFiles. If the number of
active Bins is already equal to the <Maximum number of Bins> property,
the oldest Bin will be merged in order to
+ make room for the new Bin. In that case, the Bin Manager is said
to be full, and this value will be used.</td>
+ </tr>
+ </tbody>
+</table>
+
+<p>
+ Note that the attribute value is minimally named, while the textual
description is far more verbose. This is done for a few reasons. Firstly,
storing a large value for the attribute can be
+ more costly, utilizing more heap space and requiring more resources to
process. Secondly, it's more succinct, which makes it easier to talk about.
Most importantly, though, it means that a
+ processor such as RouteOnAttribute can be used, if necessary, to route
based on the value of the attribute. In this way, the explanation can be
further expanded or updated, without changing
+ the value of the attribute and without disturbing existing flows.
+</p>
+
+
+</body>
+</html>
\ No newline at end of file
diff --git
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeRecord/additionalDetails.html
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeRecord/additionalDetails.html
index ebd8856..4a34d3e 100644
---
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeRecord/additionalDetails.html
+++
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeRecord/additionalDetails.html
@@ -38,8 +38,8 @@
Note: while the contents of a FlowFile are stored in the
Content Repository and not in the Java heap space, the Processor must hold the
FlowFile
objects themselves in memory. As a result, these FlowFiles with
their attributes can potentially take up a great deal of heap space and cause
OutOfMemoryError's to be thrown. In order to avoid this, if you
expect to merge many small FlowFiles together, it is advisable to instead use a
- MergeContent that merges no more than say 1,000 FlowFiles into
a bundle and then use a second MergeContent to merges these small bundles into
- larger bundles. For example, to merge 1,000,000 FlowFiles
together, use MergeRecord that uses a <Maximum Number of Records> of
1,000 and route the
+ MergeRecord that merges no more than say 1,000 records into a
bundle and then use a second MergeRecord to merges these small bundles into
+ larger bundles. For example, to merge 1,000,000 records
together, use MergeRecord that uses a <Maximum Number of Records> of
1,000 and route the
"merged" Relationship to a second MergeRecord that also sets
the <Maximum Number of Records> to 1,000. The second MergeRecord will
then merge 1,000 bundles
of 1,000, which in effect produces bundles of 1,000,000.
</p>
@@ -54,7 +54,7 @@
can fit in the bin according to its size and the <Maximum Bin
Size> property and whether or not the FlowFile is 'like' the other FlowFiles
in
the bin. What it means for two FlowFiles to be 'like FlowFiles' is
discussed at the end of this section.
</p>
-
+
<p>
The "Defragment" Merge Strategy can be used when records need to be
explicitly assigned to the same bin. For example, if data is split apart using
the SplitRecord Processor, each 'split' can be processed
independently and later merged back together using this Processor with the
@@ -63,7 +63,7 @@
(which indicates how many FlowFiles belong in the bin) and a unique
value for the "fragment.index" attribute so that the FlowFiles can be ordered
correctly.
</p>
-
+
<p>
In order to be added to the same bin, two FlowFiles must be 'like
FlowFiles.' In order for two FlowFiles to be like FlowFiles, they must have the
same
schema, and if the <Correlation Attribute Name> property is
set, they must have the same value for the specified attribute. For example, if
the
@@ -73,7 +73,7 @@
"abc" attribute and the "xyz" attribute, then we could accomplish
this by using UpdateAttribute and adding a property with name
"correlation.attribute"
and a value of "abc=${abc},xyz=${xyz}" and then setting
MergeRecord's <Correlation Attribute Name> property to
"correlation.attribute".
</p>
-
+
<p>
It is often useful to bin together only Records that have the
same value for some field. For example, if we have point-of-sale data, perhaps
the desire
is to bin together records that belong to the same store, as
identified by the 'storeId' field. This can be accomplished by making use of
the PartitionRecord
@@ -84,7 +84,7 @@
- <h3>When a Bin is Merged</h3>
+ <h3>When a Bin is Merged</h3>
<p>
Above, we discussed how a bin is chosen for a given FlowFile. Once
a bin has been created and FlowFiles added to it, we must have some way to
determine
when a bin is "full" so that we can bin those FlowFiles together
into a "merged" FlowFile. There are a few criteria that are used to make a
determination as
@@ -134,11 +134,11 @@
then in a situation where the "failure" relationship is eventually
routed back to MergeRecord, we could end up continually duplicating those 100
successfully
processed records.
</p>
-
-
-
+
+
+
<h2>Examples</h2>
-
+
<p>
To better understand how this Processor works, we will lay out
a few examples. For the sake of simplicity of these examples, we will use
CSV-formatted data and
write the merged data as CSV-formatted data, but
@@ -149,7 +149,7 @@
<h3>Example 1 - Batching Together Many Small FlowFiles</h3>
-
+
<p>
When we want to batch together many small FlowFiles in order to
create one larger FlowFile, we will accomplish this by using the "Bin Packing
Algorithm"
Merge Strategy. The idea here is to bundle together as many
FlowFiles as we can within our minimum and maximum number of records and bin
size.