http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java
index 53ed961..46629fe 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java
@@ -62,35 +62,35 @@ public class ScanAttribute extends AbstractProcessor {
     public static final String MATCH_CRITERIA_ALL = "All Must Match";
     public static final String MATCH_CRITERIA_ANY = "At Least 1 Must Match";
 
-    public static final PropertyDescriptor MATCHING_CRITERIA = new 
PropertyDescriptor.Builder().
-            name("Match Criteria").
-            description("If set to All Must Match, then FlowFiles will be 
routed to 'matched' only if all specified "
+    public static final PropertyDescriptor MATCHING_CRITERIA = new 
PropertyDescriptor.Builder()
+            .name("Match Criteria")
+            .description("If set to All Must Match, then FlowFiles will be 
routed to 'matched' only if all specified "
                     + "attributes' values are found in the dictionary. If set 
to At Least 1 Must Match, FlowFiles will "
-                    + "be routed to 'matched' if any attribute specified is 
found in the dictionary").
-            required(true).
-            allowableValues(MATCH_CRITERIA_ANY, MATCH_CRITERIA_ALL).
-            defaultValue(MATCH_CRITERIA_ANY).
-            build();
-    public static final PropertyDescriptor ATTRIBUTE_PATTERN = new 
PropertyDescriptor.Builder().
-            name("Attribute Pattern").
-            description("Regular Expression that specifies the names of 
attributes whose values will be matched against the terms in the dictionary").
-            required(true).
-            addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR).
-            defaultValue(".*").
-            build();
-    public static final PropertyDescriptor DICTIONARY_FILE = new 
PropertyDescriptor.Builder().
-            name("Dictionary File").
-            description("A new-line-delimited text file that includes the 
terms that should trigger a match. Empty lines are ignored.").
-            required(true).
-            addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).
-            build();
-    public static final PropertyDescriptor DICTIONARY_FILTER = new 
PropertyDescriptor.Builder().
-            name("Dictionary Filter Pattern").
-            description("A Regular Expression that will be applied to each 
line in the dictionary file. If the regular expression does not match the line, 
the line will not be included in the list of terms to search for. If a Matching 
Group is specified, only the portion of the term that matches that Matching 
Group will be used instead of the entire term. If not specified, all terms in 
the dictionary will be used and each term will consist of the text of the 
entire line in the file").
-            required(false).
-            addValidator(StandardValidators.createRegexValidator(0, 1, false)).
-            defaultValue(null).
-            build();
+                    + "be routed to 'matched' if any attribute specified is 
found in the dictionary")
+            .required(true)
+            .allowableValues(MATCH_CRITERIA_ANY, MATCH_CRITERIA_ALL)
+            .defaultValue(MATCH_CRITERIA_ANY)
+            .build();
+    public static final PropertyDescriptor ATTRIBUTE_PATTERN = new 
PropertyDescriptor.Builder()
+            .name("Attribute Pattern")
+            .description("Regular Expression that specifies the names of 
attributes whose values will be matched against the terms in the dictionary")
+            .required(true)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .defaultValue(".*")
+            .build();
+    public static final PropertyDescriptor DICTIONARY_FILE = new 
PropertyDescriptor.Builder()
+            .name("Dictionary File")
+            .description("A new-line-delimited text file that includes the 
terms that should trigger a match. Empty lines are ignored.")
+            .required(true)
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor DICTIONARY_FILTER = new 
PropertyDescriptor.Builder()
+            .name("Dictionary Filter Pattern")
+            .description("A Regular Expression that will be applied to each 
line in the dictionary file. If the regular expression does not match the line, 
the line will not be included in the list of terms to search for. If a Matching 
Group is specified, only the portion of the term that matches that Matching 
Group will be used instead of the entire term. If not specified, all terms in 
the dictionary will be used and each term will consist of the text of the 
entire line in the file")
+            .required(false)
+            .addValidator(StandardValidators.createRegexValidator(0, 1, false))
+            .defaultValue(null)
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -100,14 +100,14 @@ public class ScanAttribute extends AbstractProcessor {
     private volatile Set<String> dictionaryTerms = null;
     private volatile SynchronousFileWatcher fileWatcher = null;
 
-    public static final Relationship REL_MATCHED = new Relationship.Builder().
-            name("matched").
-            description("FlowFiles whose attributes are found in the 
dictionary will be routed to this relationship").
-            build();
-    public static final Relationship REL_UNMATCHED = new 
Relationship.Builder().
-            name("unmatched").
-            description("FlowFiles whose attributes are not found in the 
dictionary will be routed to this relationship").
-            build();
+    public static final Relationship REL_MATCHED = new Relationship.Builder()
+            .name("matched")
+            .description("FlowFiles whose attributes are found in the 
dictionary will be routed to this relationship")
+            .build();
+    public static final Relationship REL_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("FlowFiles whose attributes are not found in the 
dictionary will be routed to this relationship")
+            .build();
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
@@ -136,41 +136,32 @@ public class ScanAttribute extends AbstractProcessor {
 
     @OnScheduled
     public void onScheduled(final ProcessContext context) throws IOException {
-        final String filterRegex = context.getProperty(DICTIONARY_FILTER).
-                getValue();
-        this.dictionaryFilterPattern = (filterRegex == null) ? null : Pattern.
-                compile(filterRegex);
+        final String filterRegex = 
context.getProperty(DICTIONARY_FILTER).getValue();
+        this.dictionaryFilterPattern = (filterRegex == null) ? null : 
Pattern.compile(filterRegex);
 
-        final String attributeRegex = context.getProperty(ATTRIBUTE_PATTERN).
-                getValue();
-        this.attributePattern = (attributeRegex.equals(".*")) ? null : Pattern.
-                compile(attributeRegex);
+        final String attributeRegex = 
context.getProperty(ATTRIBUTE_PATTERN).getValue();
+        this.attributePattern = (attributeRegex.equals(".*")) ? null : 
Pattern.compile(attributeRegex);
 
         this.dictionaryTerms = createDictionary(context);
-        this.fileWatcher = new SynchronousFileWatcher(Paths.get(context.
-                getProperty(DICTIONARY_FILE).
-                getValue()), new LastModifiedMonitor(), 1000L);
+        this.fileWatcher = new 
SynchronousFileWatcher(Paths.get(context.getProperty(DICTIONARY_FILE).getValue()),
 new LastModifiedMonitor(), 1000L);
     }
 
     private Set<String> createDictionary(final ProcessContext context) throws 
IOException {
         final Set<String> terms = new HashSet<>();
 
-        final File file = new File(context.getProperty(DICTIONARY_FILE).
-                getValue());
+        final File file = new 
File(context.getProperty(DICTIONARY_FILE).getValue());
         try (final InputStream fis = new FileInputStream(file);
                 final BufferedReader reader = new BufferedReader(new 
InputStreamReader(fis))) {
 
             String line;
             while ((line = reader.readLine()) != null) {
-                if (line.trim().
-                        isEmpty()) {
+                if (line.trim().isEmpty()) {
                     continue;
                 }
 
                 String matchingTerm = line;
                 if (dictionaryFilterPattern != null) {
-                    final Matcher matcher = dictionaryFilterPattern.
-                            matcher(line);
+                    final Matcher matcher = 
dictionaryFilterPattern.matcher(line);
                     if (!matcher.matches()) {
                         continue;
                     }
@@ -207,27 +198,20 @@ public class ScanAttribute extends AbstractProcessor {
             logger.error("Unable to reload dictionary due to {}", e);
         }
 
-        final boolean matchAll = context.getProperty(MATCHING_CRITERIA).
-                getValue().
-                equals(MATCH_CRITERIA_ALL);
+        final boolean matchAll = 
context.getProperty(MATCHING_CRITERIA).getValue().equals(MATCH_CRITERIA_ALL);
 
         for (final FlowFile flowFile : flowFiles) {
             final boolean matched = matchAll ? allMatch(flowFile, 
attributePattern, dictionaryTerms) : anyMatch(flowFile, attributePattern, 
dictionaryTerms);
             final Relationship relationship = matched ? REL_MATCHED : 
REL_UNMATCHED;
-            session.getProvenanceReporter().
-                    route(flowFile, relationship);
+            session.getProvenanceReporter().route(flowFile, relationship);
             session.transfer(flowFile, relationship);
-            logger.
-                    info("Transferred {} to {}", new Object[]{flowFile, 
relationship});
+            logger.info("Transferred {} to {}", new Object[]{flowFile, 
relationship});
         }
     }
 
     private boolean allMatch(final FlowFile flowFile, final Pattern 
attributePattern, final Set<String> dictionary) {
-        for (final Map.Entry<String, String> entry : flowFile.getAttributes().
-                entrySet()) {
-            if (attributePattern == null || attributePattern.matcher(entry.
-                    getKey()).
-                    matches()) {
+        for (final Map.Entry<String, String> entry : 
flowFile.getAttributes().entrySet()) {
+            if (attributePattern == null || 
attributePattern.matcher(entry.getKey()).matches()) {
                 if (!dictionary.contains(entry.getValue())) {
                     return false;
                 }
@@ -238,11 +222,8 @@ public class ScanAttribute extends AbstractProcessor {
     }
 
     private boolean anyMatch(final FlowFile flowFile, final Pattern 
attributePattern, final Set<String> dictionary) {
-        for (final Map.Entry<String, String> entry : flowFile.getAttributes().
-                entrySet()) {
-            if (attributePattern == null || attributePattern.matcher(entry.
-                    getKey()).
-                    matches()) {
+        for (final Map.Entry<String, String> entry : 
flowFile.getAttributes().entrySet()) {
+            if (attributePattern == null || 
attributePattern.matcher(entry.getKey()).matches()) {
                 if (dictionary.contains(entry.getValue())) {
                     return true;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
index 28d48ad..ab5e8b5 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java
@@ -74,31 +74,31 @@ public class ScanContent extends AbstractProcessor {
     public static final String BINARY_ENCODING = "binary";
     public static final String MATCH_ATTRIBUTE_KEY = "matching.term";
 
-    public static final PropertyDescriptor DICTIONARY = new 
PropertyDescriptor.Builder().
-            name("Dictionary File").
-            description("The filename of the terms dictionary").
-            required(true).
-            addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).
-            build();
-    public static final PropertyDescriptor DICTIONARY_ENCODING = new 
PropertyDescriptor.Builder().
-            name("Dictionary Encoding").
-            description("Indicates how the dictionary is encoded. If 'text', 
dictionary terms are new-line delimited and UTF-8 encoded; "
-                    + "if 'binary', dictionary terms are denoted by a 4-byte 
integer indicating the term length followed by the term itself").
-            required(true).
-            allowableValues(TEXT_ENCODING, BINARY_ENCODING).
-            defaultValue(TEXT_ENCODING).
-            build();
-
-    public static final Relationship REL_MATCH = new Relationship.Builder().
-            name("matched").
-            description("FlowFiles that match at least one "
-                    + "term in the dictionary are routed to this 
relationship").
-            build();
-    public static final Relationship REL_NO_MATCH = new Relationship.Builder().
-            name("unmatched").
-            description("FlowFiles that do not match any "
-                    + "term in the dictionary are routed to this 
relationship").
-            build();
+    public static final PropertyDescriptor DICTIONARY = new 
PropertyDescriptor.Builder()
+            .name("Dictionary File")
+            .description("The filename of the terms dictionary")
+            .required(true)
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor DICTIONARY_ENCODING = new 
PropertyDescriptor.Builder()
+            .name("Dictionary Encoding")
+            .description("Indicates how the dictionary is encoded. If 'text', 
dictionary terms are new-line delimited and UTF-8 encoded; "
+                    + "if 'binary', dictionary terms are denoted by a 4-byte 
integer indicating the term length followed by the term itself")
+            .required(true)
+            .allowableValues(TEXT_ENCODING, BINARY_ENCODING)
+            .defaultValue(TEXT_ENCODING)
+            .build();
+
+    public static final Relationship REL_MATCH = new Relationship.Builder()
+            .name("matched")
+            .description("FlowFiles that match at least one "
+                    + "term in the dictionary are routed to this relationship")
+            .build();
+    public static final Relationship REL_NO_MATCH = new Relationship.Builder()
+            .name("unmatched")
+            .description("FlowFiles that do not match any "
+                    + "term in the dictionary are routed to this relationship")
+            .build();
 
     public static final Charset UTF8 = Charset.forName("UTF-8");
 
@@ -135,8 +135,7 @@ public class ScanContent extends AbstractProcessor {
     @Override
     public void onPropertyModified(final PropertyDescriptor descriptor, final 
String oldValue, final String newValue) {
         if (descriptor.equals(DICTIONARY)) {
-            fileWatcherRef.
-                    set(new SynchronousFileWatcher(Paths.get(newValue), new 
LastModifiedMonitor(), 60000L));
+            fileWatcherRef.set(new SynchronousFileWatcher(Paths.get(newValue), 
new LastModifiedMonitor(), 60000L));
         }
     }
 
@@ -154,14 +153,10 @@ public class ScanContent extends AbstractProcessor {
                 final Search<byte[]> search = new AhoCorasick<>();
                 final Set<SearchTerm<byte[]>> terms = new HashSet<>();
 
-                final InputStream inStream = Files.newInputStream(Paths.
-                        get(context.getProperty(DICTIONARY).
-                                getValue()), StandardOpenOption.READ);
+                final InputStream inStream = 
Files.newInputStream(Paths.get(context.getProperty(DICTIONARY).getValue()), 
StandardOpenOption.READ);
 
                 final TermLoader termLoader;
-                if (context.getProperty(DICTIONARY_ENCODING).
-                        getValue().
-                        equalsIgnoreCase(TEXT_ENCODING)) {
+                if 
(context.getProperty(DICTIONARY_ENCODING).getValue().equalsIgnoreCase(TEXT_ENCODING))
 {
                     termLoader = new TextualTermLoader(inStream);
                 } else {
                     termLoader = new BinaryTermLoader(inStream);
@@ -175,10 +170,7 @@ public class ScanContent extends AbstractProcessor {
 
                     search.initializeDictionary(terms);
                     searchRef.set(search);
-                    logger.
-                            info("Loaded search dictionary from {}", new 
Object[]{context.
-                                getProperty(DICTIONARY).
-                                getValue()});
+                    logger.info("Loaded search dictionary from {}", new 
Object[]{context.getProperty(DICTIONARY).getValue()});
                     return true;
                 } finally {
                     termLoader.close();
@@ -231,13 +223,9 @@ public class ScanContent extends AbstractProcessor {
             @Override
             public void process(final InputStream rawIn) throws IOException {
                 try (final InputStream in = new BufferedInputStream(rawIn)) {
-                    final SearchState<byte[]> searchResult = finalSearch.
-                            search(in, false);
+                    final SearchState<byte[]> searchResult = 
finalSearch.search(in, false);
                     if (searchResult.foundMatch()) {
-                        termRef.set(searchResult.getResults().
-                                keySet().
-                                iterator().
-                                next());
+                        
termRef.set(searchResult.getResults().keySet().iterator().next());
                     }
                 }
             }
@@ -246,17 +234,13 @@ public class ScanContent extends AbstractProcessor {
         final SearchTerm<byte[]> matchingTerm = termRef.get();
         if (matchingTerm == null) {
             logger.info("Routing {} to 'unmatched'", new Object[]{flowFile});
-            session.getProvenanceReporter().
-                    route(flowFile, REL_NO_MATCH);
+            session.getProvenanceReporter().route(flowFile, REL_NO_MATCH);
             session.transfer(flowFile, REL_NO_MATCH);
         } else {
             final String matchingTermString = matchingTerm.toString(UTF8);
-            logger.
-                    info("Routing {} to 'matched' because it matched term {}", 
new Object[]{flowFile, matchingTermString});
-            flowFile = session.
-                    putAttribute(flowFile, MATCH_ATTRIBUTE_KEY, 
matchingTermString);
-            session.getProvenanceReporter().
-                    route(flowFile, REL_MATCH);
+            logger.info("Routing {} to 'matched' because it matched term {}", 
new Object[]{flowFile, matchingTermString});
+            flowFile = session.putAttribute(flowFile, MATCH_ATTRIBUTE_KEY, 
matchingTermString);
+            session.getProvenanceReporter().route(flowFile, REL_MATCH);
             session.transfer(flowFile, REL_MATCH);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
index 071f6fb..e5e90ea 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
@@ -83,22 +83,22 @@ public class SegmentContent extends AbstractProcessor {
     public static final String FRAGMENT_INDEX = "fragment.index";
     public static final String FRAGMENT_COUNT = "fragment.count";
 
-    public static final PropertyDescriptor SIZE = new 
PropertyDescriptor.Builder().
-            name("Segment Size").
-            description("The maximum data size for each segment").
-            addValidator(StandardValidators.DATA_SIZE_VALIDATOR).
-            required(true).
-            build();
-
-    public static final Relationship REL_SEGMENTS = new Relationship.Builder().
-            name("segments").
-            description("All segments will be sent to this relationship. If 
the file was small enough that it was not segmented, "
-                    + "a copy of the original is sent to this relationship as 
well as original").
-            build();
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder().
-            name("original").
-            description("The original FlowFile will be sent to this 
relationship").
-            build();
+    public static final PropertyDescriptor SIZE = new 
PropertyDescriptor.Builder()
+            .name("Segment Size")
+            .description("The maximum data size for each segment")
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_SEGMENTS = new Relationship.Builder()
+            .name("segments")
+            .description("All segments will be sent to this relationship. If 
the file was small enough that it was not segmented, "
+                    + "a copy of the original is sent to this relationship as 
well as original")
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original FlowFile will be sent to this 
relationship")
+            .build();
 
     private Set<Relationship> relationships;
     private List<PropertyDescriptor> propertyDescriptors;
@@ -132,21 +132,16 @@ public class SegmentContent extends AbstractProcessor {
             return;
         }
 
-        final String segmentId = UUID.randomUUID().
-                toString();
-        final long segmentSize = context.getProperty(SIZE).
-                asDataSize(DataUnit.B).
-                longValue();
+        final String segmentId = UUID.randomUUID().toString();
+        final long segmentSize = 
context.getProperty(SIZE).asDataSize(DataUnit.B).longValue();
 
-        final String originalFileName = flowFile.
-                getAttribute(CoreAttributes.FILENAME.key());
+        final String originalFileName = 
flowFile.getAttribute(CoreAttributes.FILENAME.key());
 
         if (flowFile.getSize() <= segmentSize) {
             flowFile = session.putAttribute(flowFile, SEGMENT_ID, segmentId);
             flowFile = session.putAttribute(flowFile, SEGMENT_INDEX, "1");
             flowFile = session.putAttribute(flowFile, SEGMENT_COUNT, "1");
-            flowFile = session.
-                    putAttribute(flowFile, SEGMENT_ORIGINAL_FILENAME, 
originalFileName);
+            flowFile = session.putAttribute(flowFile, 
SEGMENT_ORIGINAL_FILENAME, originalFileName);
 
             flowFile = session.putAttribute(flowFile, FRAGMENT_ID, segmentId);
             flowFile = session.putAttribute(flowFile, FRAGMENT_INDEX, "1");
@@ -174,8 +169,7 @@ public class SegmentContent extends AbstractProcessor {
         final Set<FlowFile> segmentSet = new HashSet<>();
         for (int i = 1; i <= totalSegments; i++) {
             final long segmentOffset = segmentSize * (i - 1);
-            FlowFile segment = session.clone(flowFile, segmentOffset, Math.
-                    min(segmentSize, flowFile.getSize() - segmentOffset));
+            FlowFile segment = session.clone(flowFile, segmentOffset, 
Math.min(segmentSize, flowFile.getSize() - segmentOffset));
             segmentAttributes.put(SEGMENT_INDEX, String.valueOf(i));
             segmentAttributes.put(FRAGMENT_INDEX, String.valueOf(i));
             segment = session.putAllAttributes(segment, segmentAttributes);
@@ -186,11 +180,9 @@ public class SegmentContent extends AbstractProcessor {
         session.transfer(flowFile, REL_ORIGINAL);
 
         if (totalSegments <= 10) {
-            getLogger().
-                    info("Segmented {} into {} segments: {}", new 
Object[]{flowFile, totalSegments, segmentSet});
+            getLogger().info("Segmented {} into {} segments: {}", new 
Object[]{flowFile, totalSegments, segmentSet});
         } else {
-            getLogger().
-                    info("Segmented {} into {} segments", new 
Object[]{flowFile, totalSegments});
+            getLogger().info("Segmented {} into {} segments", new 
Object[]{flowFile, totalSegments});
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java
index 1c9a8c5..cfa0bda 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java
@@ -85,43 +85,43 @@ public class SplitContent extends AbstractProcessor {
     static final AllowableValue TRAILING_POSITION = new 
AllowableValue("Trailing", "Trailing", "Keep the Byte Sequence at the end of 
the first split if <Keep Byte Sequence> is true");
     static final AllowableValue LEADING_POSITION = new 
AllowableValue("Leading", "Leading", "Keep the Byte Sequence at the beginning 
of the second split if <Keep Byte Sequence> is true");
 
-    public static final PropertyDescriptor FORMAT = new 
PropertyDescriptor.Builder().
-            name("Byte Sequence Format").
-            description("Specifies how the <Byte Sequence> property should be 
interpreted").
-            required(true).
-            allowableValues(HEX_FORMAT, UTF8_FORMAT).
-            defaultValue(HEX_FORMAT.getValue()).
-            build();
-    public static final PropertyDescriptor BYTE_SEQUENCE = new 
PropertyDescriptor.Builder().
-            name("Byte Sequence").
-            description("A representation of bytes to look for and upon which 
to split the source file into separate files").
-            addValidator(StandardValidators.NON_EMPTY_VALIDATOR).
-            required(true).
-            build();
-    public static final PropertyDescriptor KEEP_SEQUENCE = new 
PropertyDescriptor.Builder().
-            name("Keep Byte Sequence").
-            description("Determines whether or not the Byte Sequence should be 
included with each Split").
-            required(true).
-            allowableValues("true", "false").
-            defaultValue("false").
-            build();
-    public static final PropertyDescriptor BYTE_SEQUENCE_LOCATION = new 
PropertyDescriptor.Builder().
-            name("Byte Sequence Location").
-            description("If <Keep Byte Sequence> is set to true, specifies 
whether the byte sequence should be added to the end of the first "
-                    + "split or the beginning of the second; if <Keep Byte 
Sequence> is false, this property is ignored.").
-            required(true).
-            allowableValues(TRAILING_POSITION, LEADING_POSITION).
-            defaultValue(TRAILING_POSITION.getValue()).
-            build();
+    public static final PropertyDescriptor FORMAT = new 
PropertyDescriptor.Builder()
+            .name("Byte Sequence Format")
+            .description("Specifies how the <Byte Sequence> property should be 
interpreted")
+            .required(true)
+            .allowableValues(HEX_FORMAT, UTF8_FORMAT)
+            .defaultValue(HEX_FORMAT.getValue())
+            .build();
+    public static final PropertyDescriptor BYTE_SEQUENCE = new 
PropertyDescriptor.Builder()
+            .name("Byte Sequence")
+            .description("A representation of bytes to look for and upon which 
to split the source file into separate files")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor KEEP_SEQUENCE = new 
PropertyDescriptor.Builder()
+            .name("Keep Byte Sequence")
+            .description("Determines whether or not the Byte Sequence should 
be included with each Split")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+    public static final PropertyDescriptor BYTE_SEQUENCE_LOCATION = new 
PropertyDescriptor.Builder()
+            .name("Byte Sequence Location")
+            .description("If <Keep Byte Sequence> is set to true, specifies 
whether the byte sequence should be added to the end of the first "
+                    + "split or the beginning of the second; if <Keep Byte 
Sequence> is false, this property is ignored.")
+            .required(true)
+            .allowableValues(TRAILING_POSITION, LEADING_POSITION)
+            .defaultValue(TRAILING_POSITION.getValue())
+            .build();
 
     public static final Relationship REL_SPLITS = new Relationship.Builder()
-            .name("splits").
-            description("All Splits will be routed to the splits 
relationship").
-            build();
+            .name("splits")
+            .description("All Splits will be routed to the splits 
relationship")
+            .build();
     public static final Relationship REL_ORIGINAL = new Relationship.Builder()
-            .name("original").
-            description("The original file").
-            build();
+            .name("original")
+            .description("The original file")
+            .build();
 
     private Set<Relationship> relationships;
     private List<PropertyDescriptor> properties;
@@ -156,15 +156,10 @@ public class SplitContent extends AbstractProcessor {
     @Override
     protected Collection<ValidationResult> customValidate(final 
ValidationContext validationContext) {
         final List<ValidationResult> results = new ArrayList<>(1);
-        final String format = validationContext.getProperty(FORMAT).
-                getValue();
-        if (HEX_FORMAT.getValue().
-                equals(format)) {
-            final String byteSequence = validationContext.
-                    getProperty(BYTE_SEQUENCE).
-                    getValue();
-            final ValidationResult result = new HexStringPropertyValidator().
-                    validate(BYTE_SEQUENCE.getName(), byteSequence, 
validationContext);
+        final String format = validationContext.getProperty(FORMAT).getValue();
+        if (HEX_FORMAT.getValue().equals(format)) {
+            final String byteSequence = 
validationContext.getProperty(BYTE_SEQUENCE).getValue();
+            final ValidationResult result = new 
HexStringPropertyValidator().validate(BYTE_SEQUENCE.getName(), byteSequence, 
validationContext);
             results.add(result);
         }
         return results;
@@ -172,13 +167,10 @@ public class SplitContent extends AbstractProcessor {
 
     @OnScheduled
     public void initializeByteSequence(final ProcessContext context) throws 
DecoderException {
-        final String bytePattern = context.getProperty(BYTE_SEQUENCE).
-                getValue();
+        final String bytePattern = 
context.getProperty(BYTE_SEQUENCE).getValue();
 
-        final String format = context.getProperty(FORMAT).
-                getValue();
-        if (HEX_FORMAT.getValue().
-                equals(format)) {
+        final String format = context.getProperty(FORMAT).getValue();
+        if (HEX_FORMAT.getValue().equals(format)) {
             this.byteSequence.set(Hex.decodeHex(bytePattern.toCharArray()));
         } else {
             
this.byteSequence.set(bytePattern.getBytes(StandardCharsets.UTF_8));
@@ -193,14 +185,11 @@ public class SplitContent extends AbstractProcessor {
         }
 
         final ProcessorLog logger = getLogger();
-        final boolean keepSequence = context.getProperty(KEEP_SEQUENCE).
-                asBoolean();
+        final boolean keepSequence = 
context.getProperty(KEEP_SEQUENCE).asBoolean();
         final boolean keepTrailingSequence;
         final boolean keepLeadingSequence;
         if (keepSequence) {
-            if (context.getProperty(BYTE_SEQUENCE_LOCATION).
-                    getValue().
-                    equals(TRAILING_POSITION.getValue())) {
+            if 
(context.getProperty(BYTE_SEQUENCE_LOCATION).getValue().equals(TRAILING_POSITION.getValue()))
 {
                 keepTrailingSequence = true;
                 keepLeadingSequence = false;
             } else {
@@ -214,8 +203,7 @@ public class SplitContent extends AbstractProcessor {
 
         final byte[] byteSequence = this.byteSequence.get();
         if (byteSequence == null) {   // should never happen. But just in 
case...
-            logger.
-                    error("{} Unable to obtain Byte Sequence", new 
Object[]{this});
+            logger.error("{} Unable to obtain Byte Sequence", new 
Object[]{this});
             session.rollback();
             return;
         }
@@ -292,8 +280,7 @@ public class SplitContent extends AbstractProcessor {
             finalSplitOffset += byteSequence.length;
         }
         if (finalSplitOffset > -1L && finalSplitOffset < flowFile.getSize()) {
-            FlowFile finalSplit = session.
-                    clone(flowFile, finalSplitOffset, flowFile.getSize() - 
finalSplitOffset);
+            FlowFile finalSplit = session.clone(flowFile, finalSplitOffset, 
flowFile.getSize() - finalSplitOffset);
             splitList.add(finalSplit);
         }
 
@@ -302,13 +289,9 @@ public class SplitContent extends AbstractProcessor {
         session.transfer(flowFile, REL_ORIGINAL);
 
         if (splitList.size() > 10) {
-            logger.
-                    info("Split {} into {} files", new Object[]{flowFile, 
splitList.
-                        size()});
+            logger.info("Split {} into {} files", new Object[]{flowFile, 
splitList.size()});
         } else {
-            logger.
-                    info("Split {} into {} files: {}", new Object[]{flowFile, 
splitList.
-                        size(), splitList});
+            logger.info("Split {} into {} files: {}", new Object[]{flowFile, 
splitList.size(), splitList});
         }
     }
 
@@ -323,8 +306,7 @@ public class SplitContent extends AbstractProcessor {
         final String originalFilename = source.
                 getAttribute(CoreAttributes.FILENAME.key());
 
-        final String fragmentId = UUID.randomUUID().
-                toString();
+        final String fragmentId = UUID.randomUUID().toString();
         final ArrayList<FlowFile> newList = new ArrayList<>(splits);
         splits.clear();
         for (int i = 1; i <= newList.size(); i++) {
@@ -345,16 +327,9 @@ public class SplitContent extends AbstractProcessor {
         public ValidationResult validate(final String subject, final String 
input, final ValidationContext validationContext) {
             try {
                 Hex.decodeHex(input.toCharArray());
-                return new ValidationResult.Builder().valid(true).
-                        input(input).
-                        subject(subject).
-                        build();
+                return new 
ValidationResult.Builder().valid(true).input(input).subject(subject).build();
             } catch (final Exception e) {
-                return new ValidationResult.Builder().valid(false).
-                        explanation("Not a valid Hex String").
-                        input(input).
-                        subject(subject).
-                        build();
+                return new 
ValidationResult.Builder().valid(false).explanation("Not a valid Hex 
String").input(input).subject(subject).build();
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
index 2ffebd5..ef7a86a 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitJson.java
@@ -59,28 +59,27 @@ import java.util.concurrent.atomic.AtomicReference;
         + "does not evaluate to an array element, the original file is routed 
to 'failure' and no files are generated.")
 public class SplitJson extends AbstractJsonPathProcessor {
 
-    public static final PropertyDescriptor ARRAY_JSON_PATH_EXPRESSION = new 
PropertyDescriptor.Builder().
-            name("JsonPath Expression").
-            description("A JsonPath expression that indicates the array 
element to split into JSON/scalar fragments.").
-            addValidator(StandardValidators.NON_EMPTY_VALIDATOR) // Full 
validation/caching occurs in #customValidate
-            .
-            required(true).
-            build();
-
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder().
-            name("original").
-            description("The original FlowFile that was split into segments. 
If the FlowFile fails processing, nothing will be sent to "
-                    + "this relationship").
-            build();
-    public static final Relationship REL_SPLIT = new Relationship.Builder().
-            name("split").
-            description("All segments of the original FlowFile will be routed 
to this relationship").
-            build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().
-            name("failure").
-            description("If a FlowFile fails processing for any reason (for 
example, the FlowFile is not valid JSON or the specified "
-                    + "path does not exist), it will be routed to this 
relationship").
-            build();
+    public static final PropertyDescriptor ARRAY_JSON_PATH_EXPRESSION = new 
PropertyDescriptor.Builder()
+            .name("JsonPath Expression")
+            .description("A JsonPath expression that indicates the array 
element to split into JSON/scalar fragments.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) // Full 
validation/caching occurs in #customValidate
+            .required(true)
+            .build();
+
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original FlowFile that was split into segments. 
If the FlowFile fails processing, nothing will be sent to "
+                    + "this relationship")
+            .build();
+    public static final Relationship REL_SPLIT = new Relationship.Builder()
+            .name("split")
+            .description("All segments of the original FlowFile will be routed 
to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile fails processing for any reason (for 
example, the FlowFile is not valid JSON or the specified "
+                    + "path does not exist), it will be routed to this 
relationship")
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -137,10 +136,8 @@ public class SplitJson extends AbstractJsonPathProcessor {
             }
         };
 
-        String value = 
validationContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).
-                getValue();
-        return Collections.singleton(validator.
-                validate(ARRAY_JSON_PATH_EXPRESSION.getName(), value, 
validationContext));
+        String value = 
validationContext.getProperty(ARRAY_JSON_PATH_EXPRESSION).getValue();
+        return 
Collections.singleton(validator.validate(ARRAY_JSON_PATH_EXPRESSION.getName(), 
value, validationContext));
     }
 
     @Override
@@ -156,18 +153,14 @@ public class SplitJson extends AbstractJsonPathProcessor {
         try {
             documentContext = validateAndEstablishJsonContext(processSession, 
original);
         } catch (InvalidJsonException e) {
-            logger.
-                    error("FlowFile {} did not have valid JSON content.", new 
Object[]{original});
+            logger.error("FlowFile {} did not have valid JSON content.", new 
Object[]{original});
             processSession.transfer(original, REL_FAILURE);
             return;
         }
 
         final JsonPath jsonPath = JSON_PATH_REF.get();
-        String representationOption = processContext.
-                getProperty(NULL_VALUE_DEFAULT_REPRESENTATION).
-                getValue();
-        final String nullDefaultValue = NULL_REPRESENTATION_MAP.
-                get(representationOption);
+        String representationOption = 
processContext.getProperty(NULL_VALUE_DEFAULT_REPRESENTATION).getValue();
+        final String nullDefaultValue = 
NULL_REPRESENTATION_MAP.get(representationOption);
 
         final List<FlowFile> segments = new ArrayList<>();
 
@@ -175,17 +168,13 @@ public class SplitJson extends AbstractJsonPathProcessor {
         try {
             jsonPathResult = documentContext.read(jsonPath);
         } catch (PathNotFoundException e) {
-            logger.
-                    warn("JsonPath {} could not be found for FlowFile {}", new 
Object[]{jsonPath.
-                        getPath(), original});
+            logger.warn("JsonPath {} could not be found for FlowFile {}", new 
Object[]{jsonPath.getPath(), original});
             processSession.transfer(original, REL_FAILURE);
             return;
         }
 
         if (!(jsonPathResult instanceof List)) {
-            logger.
-                    error("The evaluated value {} of {} was not a JSON Array 
compatible type and cannot be split.",
-                            new Object[]{jsonPathResult, jsonPath.getPath()});
+            logger.error("The evaluated value {} of {} was not a JSON Array 
compatible type and cannot be split.", new Object[]{jsonPathResult, 
jsonPath.getPath()});
             processSession.transfer(original, REL_FAILURE);
             return;
         }
@@ -198,20 +187,16 @@ public class SplitJson extends AbstractJsonPathProcessor {
                 @Override
                 public void process(OutputStream out) throws IOException {
                     String resultSegmentContent = 
getResultRepresentation(resultSegment, nullDefaultValue);
-                    out.write(resultSegmentContent.
-                            getBytes(StandardCharsets.UTF_8));
+                    
out.write(resultSegmentContent.getBytes(StandardCharsets.UTF_8));
                 }
             });
             segments.add(split);
         }
 
-        processSession.getProvenanceReporter().
-                fork(original, segments);
+        processSession.getProvenanceReporter().fork(original, segments);
 
         processSession.transfer(segments, REL_SPLIT);
         processSession.transfer(original, REL_ORIGINAL);
-        logger.
-                info("Split {} into {} FlowFiles", new Object[]{original, 
segments.
-                    size()});
+        logger.info("Split {} into {} FlowFiles", new Object[]{original, 
segments.size()});
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
index f68ef4e..d641274 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java
@@ -77,41 +77,40 @@ public class SplitText extends AbstractProcessor {
     public static final String FRAGMENT_COUNT = "fragment.count";
     public static final String SEGMENT_ORIGINAL_FILENAME = 
"segment.original.filename";
 
-    public static final PropertyDescriptor LINE_SPLIT_COUNT = new 
PropertyDescriptor.Builder().
-            name("Line Split Count").
-            description("The number of lines that will be added to each split 
file").
-            required(true).
-            addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).
-            build();
-    public static final PropertyDescriptor HEADER_LINE_COUNT = new 
PropertyDescriptor.Builder().
-            name("Header Line Count").
-            description("The number of lines that should be considered part of 
the header; the header lines will be duplicated to all split files").
-            required(true).
-            addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR).
-            defaultValue("0").
-            build();
-    public static final PropertyDescriptor REMOVE_TRAILING_NEWLINES = new 
PropertyDescriptor.Builder().
-            name("Remove Trailing Newlines").
-            description(
-                    "Whether to remove newlines at the end of each split file. 
This should be false if you intend to merge the split files later").
-            required(true).
-            addValidator(StandardValidators.BOOLEAN_VALIDATOR).
-            allowableValues("true", "false").
-            defaultValue("true").
-            build();
-
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder().
-            name("original").
-            description("The original input file will be routed to this 
destination when it has been successfully split into 1 or more files").
-            build();
-    public static final Relationship REL_SPLITS = new Relationship.Builder().
-            name("splits").
-            description("The split files will be routed to this destination 
when an input file is successfully split into 1 or more split files").
-            build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().
-            name("failure").
-            description("If a file cannot be split for some reason, the 
original file will be routed to this destination and nothing will be routed 
elsewhere").
-            build();
+    public static final PropertyDescriptor LINE_SPLIT_COUNT = new 
PropertyDescriptor.Builder()
+            .name("Line Split Count")
+            .description("The number of lines that will be added to each split 
file")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor HEADER_LINE_COUNT = new 
PropertyDescriptor.Builder()
+            .name("Header Line Count")
+            .description("The number of lines that should be considered part 
of the header; the header lines will be duplicated to all split files")
+            .required(true)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .defaultValue("0")
+            .build();
+    public static final PropertyDescriptor REMOVE_TRAILING_NEWLINES = new 
PropertyDescriptor.Builder()
+            .name("Remove Trailing Newlines")
+            .description("Whether to remove newlines at the end of each split 
file. This should be false if you intend to merge the split files later")
+            .required(true)
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original input file will be routed to this 
destination when it has been successfully split into 1 or more files")
+            .build();
+    public static final Relationship REL_SPLITS = new Relationship.Builder()
+            .name("splits")
+            .description("The split files will be routed to this destination 
when an input file is successfully split into 1 or more split files")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a file cannot be split for some reason, the 
original file will be routed to this destination and nothing will be routed 
elsewhere")
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -235,13 +234,9 @@ public class SplitText extends AbstractProcessor {
         }
 
         final ProcessorLog logger = getLogger();
-        final int headerCount = context.getProperty(HEADER_LINE_COUNT).
-                asInteger();
-        final int splitCount = context.getProperty(LINE_SPLIT_COUNT).
-                asInteger();
-        final boolean removeTrailingNewlines = context.
-                getProperty(REMOVE_TRAILING_NEWLINES).
-                asBoolean();
+        final int headerCount = 
context.getProperty(HEADER_LINE_COUNT).asInteger();
+        final int splitCount = 
context.getProperty(LINE_SPLIT_COUNT).asInteger();
+        final boolean removeTrailingNewlines = 
context.getProperty(REMOVE_TRAILING_NEWLINES).asBoolean();
 
         final ObjectHolder<String> errorMessage = new ObjectHolder<>(null);
         final ArrayList<SplitInfo> splitInfos = new ArrayList<>();
@@ -258,8 +253,7 @@ public class SplitText extends AbstractProcessor {
                     final ByteArrayOutputStream headerStream = new 
ByteArrayOutputStream();
                     final int headerLinesCopied = readLines(in, headerCount, 
headerStream, true);
                     if (headerLinesCopied < headerCount) {
-                        errorMessage.
-                                set("Header Line Count is set to " + 
headerCount + " but file had only " + headerLinesCopied + " lines");
+                        errorMessage.set("Header Line Count is set to " + 
headerCount + " but file had only " + headerLinesCopied + " lines");
                         return;
                     }
 
@@ -270,23 +264,17 @@ public class SplitText extends AbstractProcessor {
                             final IntegerHolder linesCopied = new 
IntegerHolder(0);
                             FlowFile splitFile = session.create(flowFile);
                             try {
-                                splitFile = session.
-                                        write(splitFile, new 
OutputStreamCallback() {
-                                            @Override
-                                            public void process(final 
OutputStream rawOut) throws IOException {
-                                                try (final 
BufferedOutputStream out = new BufferedOutputStream(rawOut)) {
-                                                    headerStream.writeTo(out);
-                                                    linesCopied.
-                                                    set(readLines(in, 
splitCount, out, !removeTrailingNewlines));
-                                                }
-                                            }
-                                        });
-                                splitFile = session.
-                                        putAttribute(splitFile, 
SPLIT_LINE_COUNT, String.
-                                                valueOf(linesCopied.get()));
-                                logger.
-                                        debug("Created Split File {} with {} 
lines", new Object[]{splitFile, linesCopied.
-                                            get()});
+                                splitFile = session.write(splitFile, new 
OutputStreamCallback() {
+                                    @Override
+                                    public void process(final OutputStream 
rawOut) throws IOException {
+                                        try (final BufferedOutputStream out = 
new BufferedOutputStream(rawOut)) {
+                                            headerStream.writeTo(out);
+                                            linesCopied.set(readLines(in, 
splitCount, out, !removeTrailingNewlines));
+                                        }
+                                    }
+                                });
+                                splitFile = session.putAttribute(splitFile, 
SPLIT_LINE_COUNT, String.valueOf(linesCopied.get()));
+                                logger.debug("Created Split File {} with {} 
lines", new Object[]{splitFile, linesCopied.get()});
                             } finally {
                                 if (linesCopied.get() > 0) {
                                     splits.add(splitFile);
@@ -313,11 +301,10 @@ public class SplitText extends AbstractProcessor {
                                 info.offsetBytes = beforeReadingLines;
                                 splitInfos.add(info);
                                 final long procNanos = System.nanoTime() - 
startNanos;
-                                final long procMillis = TimeUnit.MILLISECONDS.
-                                        convert(procNanos, 
TimeUnit.NANOSECONDS);
-                                logger.
-                                        debug("Detected start of Split File in 
{} at byte offset {} with a length of {} bytes; total splits = {}; total 
processing time = {} ms", new Object[]{flowFile, beforeReadingLines, 
info.lengthBytes, splitInfos.
-                                            size(), procMillis});
+                                final long procMillis = 
TimeUnit.MILLISECONDS.convert(procNanos, TimeUnit.NANOSECONDS);
+                                logger.debug("Detected start of Split File in 
{} at byte offset {} with a length of {} bytes; "
+                                        + "total splits = {}; total processing 
time = {} ms",
+                                        new Object[]{flowFile, 
beforeReadingLines, info.lengthBytes, splitInfos.size(), procMillis});
                             }
                         }
                     }
@@ -326,9 +313,7 @@ public class SplitText extends AbstractProcessor {
         });
 
         if (errorMessage.get() != null) {
-            logger.
-                    error("Unable to split {} due to {}; routing to failure", 
new Object[]{flowFile, errorMessage.
-                        get()});
+            logger.error("Unable to split {} due to {}; routing to failure", 
new Object[]{flowFile, errorMessage.get()});
             session.transfer(flowFile, REL_FAILURE);
             if (splits != null && !splits.isEmpty()) {
                 session.remove(splits);
@@ -339,22 +324,17 @@ public class SplitText extends AbstractProcessor {
         if (!splitInfos.isEmpty()) {
             // Create the splits
             for (final SplitInfo info : splitInfos) {
-                FlowFile split = session.
-                        clone(flowFile, info.offsetBytes, info.lengthBytes);
-                split = session.putAttribute(split, SPLIT_LINE_COUNT, String.
-                        valueOf(info.lengthLines));
+                FlowFile split = session.clone(flowFile, info.offsetBytes, 
info.lengthBytes);
+                split = session.putAttribute(split, SPLIT_LINE_COUNT, 
String.valueOf(info.lengthLines));
                 splits.add(split);
             }
         }
         finishFragmentAttributes(session, flowFile, splits);
 
         if (splits.size() > 10) {
-            logger.info("Split {} into {} files", new Object[]{flowFile, 
splits.
-                size()});
+            logger.info("Split {} into {} files", new Object[]{flowFile, 
splits.size()});
         } else {
-            logger.
-                    info("Split {} into {} files: {}", new Object[]{flowFile, 
splits.
-                        size(), splits});
+            logger.info("Split {} into {} files: {}", new Object[]{flowFile, 
splits.size(), splits});
         }
 
         session.transfer(flowFile, REL_ORIGINAL);
@@ -369,11 +349,9 @@ public class SplitText extends AbstractProcessor {
      * @param unpacked
      */
     private void finishFragmentAttributes(final ProcessSession session, final 
FlowFile source, final List<FlowFile> splits) {
-        final String originalFilename = source.
-                getAttribute(CoreAttributes.FILENAME.key());
+        final String originalFilename = 
source.getAttribute(CoreAttributes.FILENAME.key());
 
-        final String fragmentId = UUID.randomUUID().
-                toString();
+        final String fragmentId = UUID.randomUUID().toString();
         final ArrayList<FlowFile> newList = new ArrayList<>(splits);
         splits.clear();
         for (int i = 1; i <= newList.size(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
index 8e80e91..adbfff2 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitXml.java
@@ -66,26 +66,26 @@ import org.xml.sax.XMLReader;
 @CapabilityDescription("Splits an XML File into multiple separate FlowFiles, 
each comprising a child or descendant of the original root element")
 public class SplitXml extends AbstractProcessor {
 
-    public static final PropertyDescriptor SPLIT_DEPTH = new 
PropertyDescriptor.Builder().
-            name("Split Depth").
-            description("Indicates the XML-nesting depth to start splitting 
XML fragments. A depth of 1 means split the root's children, whereas a depth of 
2 means split the root's children's children and so forth.").
-            required(true).
-            addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).
-            defaultValue("1").
-            build();
-
-    public static final Relationship REL_ORIGINAL = new Relationship.Builder().
-            name("original").
-            description("The original FlowFile that was split into segments. 
If the FlowFile fails processing, nothing will be sent to this relationship").
-            build();
-    public static final Relationship REL_SPLIT = new Relationship.Builder().
-            name("split").
-            description("All segments of the original FlowFile will be routed 
to this relationship").
-            build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().
-            name("failure").
-            description("If a FlowFile fails processing for any reason (for 
example, the FlowFile is not valid XML), it will be routed to this 
relationship").
-            build();
+    public static final PropertyDescriptor SPLIT_DEPTH = new 
PropertyDescriptor.Builder()
+            .name("Split Depth")
+            .description("Indicates the XML-nesting depth to start splitting 
XML fragments. A depth of 1 means split the root's children, whereas a depth of 
2 means split the root's children's children and so forth.")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .defaultValue("1")
+            .build();
+
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("The original FlowFile that was split into segments. 
If the FlowFile fails processing, nothing will be sent to this relationship")
+            .build();
+    public static final Relationship REL_SPLIT = new Relationship.Builder()
+            .name("split")
+            .description("All segments of the original FlowFile will be routed 
to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile fails processing for any reason (for 
example, the FlowFile is not valid XML), it will be routed to this 
relationship")
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -93,8 +93,7 @@ public class SplitXml extends AbstractProcessor {
     private static final String FEATURE_PREFIX = 
"http://xml.org/sax/features/";;
     public static final String ENABLE_NAMESPACES_FEATURE = FEATURE_PREFIX + 
"namespaces";
     public static final String ENABLE_NAMESPACE_PREFIXES_FEATURE = 
FEATURE_PREFIX + "namespace-prefixes";
-    private static final SAXParserFactory saxParserFactory = SAXParserFactory.
-            newInstance();
+    private static final SAXParserFactory saxParserFactory = 
SAXParserFactory.newInstance();
 
     static {
         saxParserFactory.setNamespaceAware(true);
@@ -103,8 +102,7 @@ public class SplitXml extends AbstractProcessor {
             saxParserFactory.setFeature(ENABLE_NAMESPACE_PREFIXES_FEATURE, 
true);
         } catch (Exception e) {
             final Logger staticLogger = 
LoggerFactory.getLogger(SplitXml.class);
-            staticLogger.
-                    warn("Unable to configure SAX Parser to make namespaces 
available", e);
+            staticLogger.warn("Unable to configure SAX Parser to make 
namespaces available", e);
         }
     }
 
@@ -138,8 +136,7 @@ public class SplitXml extends AbstractProcessor {
             return;
         }
 
-        final int depth = context.getProperty(SPLIT_DEPTH).
-                asInteger();
+        final int depth = context.getProperty(SPLIT_DEPTH).asInteger();
         final ProcessorLog logger = getLogger();
 
         final List<FlowFile> splits = new ArrayList<>();
@@ -169,8 +166,7 @@ public class SplitXml extends AbstractProcessor {
                         reader.setContentHandler(parser);
                         reader.parse(new InputSource(in));
                     } catch (final ParserConfigurationException | SAXException 
e) {
-                        logger.
-                                error("Unable to parse {} due to {}", new 
Object[]{original, e});
+                        logger.error("Unable to parse {} due to {}", new 
Object[]{original, e});
                         failed.set(true);
                     }
                 }
@@ -183,9 +179,7 @@ public class SplitXml extends AbstractProcessor {
         } else {
             session.transfer(splits, REL_SPLIT);
             session.transfer(original, REL_ORIGINAL);
-            logger.
-                    info("Split {} into {} FlowFiles", new Object[]{original, 
splits.
-                        size()});
+            logger.info("Split {} into {} FlowFiles", new Object[]{original, 
splits.size()});
         }
     }
 
@@ -247,9 +241,7 @@ public class SplitXml extends AbstractProcessor {
             // if we're at a level where we care about capturing text, then 
add the closing element
             if (newDepth >= splitDepth) {
                 // Add the element end tag.
-                sb.append("</").
-                        append(qName).
-                        append(">");
+                sb.append("</").append(qName).append(">");
             }
 
             // If we have now returned to level 1, we have finished processing
@@ -301,14 +293,8 @@ public class SplitXml extends AbstractProcessor {
                 int attCount = atts.getLength();
                 for (int i = 0; i < attCount; i++) {
                     String attName = atts.getQName(i);
-                    String attValue = StringEscapeUtils.escapeXml10(atts.
-                            getValue(i));
-                    sb.append(" ").
-                            append(attName).
-                            append("=").
-                            append("\"").
-                            append(attValue).
-                            append("\"");
+                    String attValue = 
StringEscapeUtils.escapeXml10(atts.getValue(i));
+                    sb.append(" 
").append(attName).append("=").append("\"").append(attValue).append("\"");
                 }
 
                 sb.append(">");

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d29a2d68/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
index 3451516..2abf4a1 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
@@ -69,21 +69,21 @@ import org.apache.nifi.util.Tuple;
         description = "These XSLT parameters are passed to the transformer")
 public class TransformXml extends AbstractProcessor {
 
-    public static final PropertyDescriptor XSLT_FILE_NAME = new 
PropertyDescriptor.Builder().
-            name("XSLT file name").
-            description("Provides the name (including full path) of the XSLT 
file to apply to the flowfile XML content.").
-            required(true).
-            addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).
-            build();
-
-    public static final Relationship REL_SUCCESS = new Relationship.Builder().
-            name("success").
-            description("The FlowFile with transformed content will be routed 
to this relationship").
-            build();
-    public static final Relationship REL_FAILURE = new Relationship.Builder().
-            name("failure").
-            description("If a FlowFile fails processing for any reason (for 
example, the FlowFile is not valid XML), it will be routed to this 
relationship").
-            build();
+    public static final PropertyDescriptor XSLT_FILE_NAME = new 
PropertyDescriptor.Builder()
+            .name("XSLT file name")
+            .description("Provides the name (including full path) of the XSLT 
file to apply to the flowfile XML content.")
+            .required(true)
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("The FlowFile with transformed content will be routed 
to this relationship")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile fails processing for any reason (for 
example, the FlowFile is not valid XML), it will be routed to this 
relationship")
+            .build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -113,13 +113,12 @@ public class TransformXml extends AbstractProcessor {
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final 
String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
-                .name(propertyDescriptorName).
-                expressionLanguageSupported(true).
-                addValidator(StandardValidators.
-                        
createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING,
 true)).
-                required(false).
-                dynamic(true).
-                build();
+                .name(propertyDescriptorName)
+                .expressionLanguageSupported(true)
+                
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING,
 true))
+                .required(false)
+                .dynamic(true)
+                .build();
     }
 
     @Override
@@ -139,26 +138,17 @@ public class TransformXml extends AbstractProcessor {
                         public void process(final InputStream rawIn, final 
OutputStream out) throws IOException {
                             try (final InputStream in = new 
BufferedInputStream(rawIn)) {
 
-                                File stylesheet = new File(context.
-                                        getProperty(XSLT_FILE_NAME).
-                                        getValue());
+                                File stylesheet = new 
File(context.getProperty(XSLT_FILE_NAME).getValue());
                                 StreamSource styleSource = new 
StreamSource(stylesheet);
                                 TransformerFactory tfactory = new 
net.sf.saxon.TransformerFactoryImpl();
-                                Transformer transformer = tfactory.
-                                newTransformer(styleSource);
+                                Transformer transformer = 
tfactory.newTransformer(styleSource);
 
                                 // pass all dynamic properties to the 
transformer
-                                for (final Map.Entry<PropertyDescriptor, 
String> entry : context.
-                                getProperties().
+                                for (final Map.Entry<PropertyDescriptor, 
String> entry : context.getProperties().
                                 entrySet()) {
-                                    if (entry.getKey().
-                                    isDynamic()) {
-                                        String value = context.
-                                        newPropertyValue(entry.getValue()).
-                                        evaluateAttributeExpressions(original).
-                                        getValue();
-                                        
transformer.setParameter(entry.getKey().
-                                                getName(), value);
+                                    if (entry.getKey().isDynamic()) {
+                                        String value = 
context.newPropertyValue(entry.getValue()).evaluateAttributeExpressions(original).getValue();
+                                        
transformer.setParameter(entry.getKey().getName(), value);
                                     }
                                 }
 
@@ -172,13 +162,10 @@ public class TransformXml extends AbstractProcessor {
                         }
                     });
             session.transfer(transformed, REL_SUCCESS);
-            session.getProvenanceReporter().
-                    modifyContent(transformed, stopWatch.
-                            getElapsed(TimeUnit.MILLISECONDS));
+            session.getProvenanceReporter().modifyContent(transformed, 
stopWatch.getElapsed(TimeUnit.MILLISECONDS));
             logger.info("Transformed {}", new Object[]{original});
         } catch (ProcessException e) {
-            logger.
-                    error("Unable to transform {} due to {}", new 
Object[]{original, e});
+            logger.error("Unable to transform {} due to {}", new 
Object[]{original, e});
             session.transfer(original, REL_FAILURE);
         }
     }
@@ -191,8 +178,7 @@ public class TransformXml extends AbstractProcessor {
         @Override
         public ValidationResult validate(final String subject, final String 
input, final ValidationContext validationContext) {
             final Tuple<String, ValidationResult> lastResult = 
this.cachedResult;
-            if (lastResult != null && lastResult.getKey().
-                    equals(input)) {
+            if (lastResult != null && lastResult.getKey().equals(input)) {
                 return lastResult.getValue();
             } else {
                 String error = null;
@@ -206,13 +192,12 @@ public class TransformXml extends AbstractProcessor {
                     error = e.toString();
                 }
 
-                this.cachedResult = new Tuple<>(input,
-                        new ValidationResult.Builder()
-                        .input(input).
-                        subject(subject).
-                        valid(error == null).
-                        explanation(error).
-                        build());
+                this.cachedResult = new Tuple<>(input, new 
ValidationResult.Builder()
+                        .input(input)
+                        .subject(subject)
+                        .valid(error == null)
+                        .explanation(error)
+                        .build());
                 return this.cachedResult.getValue();
             }
         }

Reply via email to