You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by markobean <gi...@git.apache.org> on 2016/05/16 00:27:37 UTC

[GitHub] nifi pull request: NIFI-1118 adding SplitText features of size lim...

GitHub user markobean opened a pull request:

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

    NIFI-1118 adding SplitText features of size limit and header marker c\u2026

    NIFI-1118 - Includes the addition of the new features of maximum split size limitation and header marker character(s) to identify header lines. Also, the recent bug fix to the Remove Trailing New Lines was incorporated.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/markobean/nifi NIFI-1118

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

    https://github.com/apache/nifi/pull/444.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #444
    
----
commit 5ba08ddc893561a630177a861e5e10b84f58f298
Author: Mark Bean <ma...@gmail.com>
Date:   2016-05-16T00:07:04Z

    NIFI-1118 adding SplitText features of size limit and header marker characters

----


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65467161
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -94,12 +118,16 @@
                 .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
                 .defaultValue("0")
                 .build();
    +    public static final PropertyDescriptor HEADER_MARKER = new PropertyDescriptor.Builder()
    +            .name("Header Line Marker Characters")
    +            .description("The first character(s) on the line of the datafile which signifies a header line. This value is ignored when Header Line Count is non-zero. " +
    +                    "The first line not containing the Header Line Marker Characters and all subsequent lines are considered non-header")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .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. If this is set to "
    -                + "'true' and a FlowFile is generated that contains only 'empty lines' (i.e., consists only of \r and \n characters), the FlowFile will not be emitted. "
    -                + "Note, however, that if the Header Line Count is greater than 0, the resultant FlowFile will never be empty as it will consist of the header lines, so "
    -                + "a FlowFile may be emitted that contians only the header lines.")
    +            .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.")
    --- End diff --
    
    @mosermw, agreed. Original description put back in place (with spelling correction)


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65425598
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
             long bytesRead = 0L;
    +        final ByteArrayOutputStream buffer;
    +        if (out != null) {
    +            buffer = new ByteArrayOutputStream();
    +        } else {
    +            buffer = null;
    +        }
    --- End diff --
    
    I stared at this if-else for longer than I care to admit.  A ternary operator for conditional assignment to a final variable would have helped me, at least, maybe others too.


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

[GitHub] nifi pull request: NIFI-1118 adding SplitText features of size limit and hea...

Posted by mosermw <gi...@git.apache.org>.
Github user mosermw commented on the pull request:

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


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65469870
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
             long bytesRead = 0L;
    +        final ByteArrayOutputStream buffer;
    +        if (out != null) {
    +            buffer = new ByteArrayOutputStream();
    +        } else {
    +            buffer = null;
    +        }
    --- End diff --
    
    No objection to leaving it as-is.


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65471941
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitText.java ---
    @@ -39,6 +39,303 @@
                 + "\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nLastLine\n";
     
         @Test
    +    public void testLastLineExceedsSizeLimit() {
    +        final TestRunner runner = TestRunners.newTestRunner(new SplitText());
    +        runner.setProperty(SplitText.HEADER_LINE_COUNT, "0");
    +        runner.setProperty(SplitText.LINE_SPLIT_COUNT, "2");
    +        runner.setProperty(SplitText.FRAGMENT_MAX_SIZE, "20 B");
    +
    +        runner.enqueue("Line #1\nLine #2\nLine #3\nLong line exceeding limit");
    +        runner.run();
    +
    +        runner.assertTransferCount(SplitText.REL_FAILURE, 0);
    +        runner.assertTransferCount(SplitText.REL_ORIGINAL, 1);
    +        runner.assertTransferCount(SplitText.REL_SPLITS, 3);
    +    }
    --- End diff --
    
    Talk about thorough! I don't know how you managed to catch this edge case. However, for completeness, the RTN functionality once again increased complexity and another buffer count was added. Now, if the size limit is exceeded (with RTN = true and no header lines), then the number of EOL bytes previously added to the info.lengthBytes are subtracted from the previous line effectively removing the final EOL characters of the split.


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65418261
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
    --- End diff --
    
    Looks like the lastByte local var is not used anymore.


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

[GitHub] nifi issue #444: NIFI-1118 adding SplitText features of size limit and heade...

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

    https://github.com/apache/nifi/pull/444
  
    +1 LGTM.  contrib-check on nifi-standard-bundle passes.  Old unit tests from NIFI-1649 continue to work and new unit tests make sense.  Templates from NIFI-1649 continue to work the same.  Will squash, merge and push tomorrow if nobody else reviews.  This is a long standing ticket on a heavily used processor, so I want to give others a chance to weigh in if they want to.


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65468881
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
             long bytesRead = 0L;
    +        final ByteArrayOutputStream buffer;
    +        if (out != null) {
    +            buffer = new ByteArrayOutputStream();
    +        } else {
    +            buffer = null;
    +        }
    --- End diff --
    
    It made more sense when this method had the possibility of having 'out' be null; this occurred by design in a previous version. Now, it is only called with a valid OutputStream. Still, I think this doesn't hurt and adds a layer of safety should the method be called differently in the future. However, if you still object,  the if/else can be removed and the final buffer be instantiated outright; all unit tests confirmed to pass in this configuration (although not staged to be committed at this time.)


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65467699
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
             long bytesRead = 0L;
    +        final ByteArrayOutputStream buffer;
    +        if (out != null) {
    +            buffer = new ByteArrayOutputStream();
    +        } else {
    +            buffer = null;
    +        }
             byte[] bytesToWriteFirst = leadingNewLineBytes;
     
    +        in.mark(Integer.MAX_VALUE);
             while (true) {
    -            in.mark(1);
                 final int nextByte = in.read();
     
    -            final boolean isNewLineChar = nextByte == '\r' || nextByte == '\n';
    -
    -            // if we hit end of stream or new line we're done
    +            // if we hit end of stream we're done
                 if (nextByte == -1) {
    -                if (lastByte == '\r') {
    -                    eolBuffer.addEndOfLine(true, false);
    +                if (buffer != null) {
    +                    buffer.writeTo(out);
    +                    buffer.close();
                     }
    -
    -                return new EndOfLineMarker(bytesRead, eolBuffer, true, bytesToWriteFirst);
    +                return new EndOfLineMarker(bytesRead, eolBuffer, true, bytesToWriteFirst);  // bytesToWriteFirst should be "null"?
                 }
     
    -            // If we get a character that's not an end-of-line char, then
    -            // we need to write out the EOL's that we have buffered (if out != null).
    -            // Then, we need to reset our EOL buffer because we no longer have consecutive EOL's
    -            if (!isNewLineChar) {
    +            // Verify leading bytes do not violate size limitation
    +            if (bytesToWriteFirst != null && (bytesToWriteFirst.length + bytesRead) > (maxSize - bytesReadSoFar) && includeLineDelimiter) {
    +                return new EndOfLineMarker(-1, eolBuffer, false, leadingNewLineBytes);
    +            }
    +            // Write leadingNewLines, if appropriate
    +            if ( buffer != null && includeLineDelimiter && bytesToWriteFirst != null) {
    +                bytesRead += bytesToWriteFirst.length;
    +                buffer.write(bytesToWriteFirst);
    +                bytesToWriteFirst = null;
    +            }
    +            // buffer the output
    +            bytesRead++;
    +            if (buffer != null && nextByte != '\n' && nextByte != '\r') {
                     if (bytesToWriteFirst != null) {
    -                    if (out != null) {
    -                        out.write(bytesToWriteFirst);
    -                    }
    -
    -                    bytesToWriteFirst = null;
    -                }
    -
    -                if (out != null) {
    -                    eolBuffer.drainTo(out);
    +                    buffer.write(bytesToWriteFirst);
                     }
    -
    +                bytesToWriteFirst = null;
    +                eolBuffer.drainTo(buffer);
                     eolBuffer.clear();
    +                buffer.write(nextByte);
                 }
     
    -            // if there's an OutputStream to copy the data to, copy it, if appropriate.
    -            // "if appropriate" means that it's not a line delimiter or that we want to copy line delimiters
    -            bytesRead++;
    -            if (out != null && (includeLineDelimiter || !isNewLineChar)) {
    -                if (bytesToWriteFirst != null) {
    -                    out.write(bytesToWriteFirst);
    -                    bytesToWriteFirst = null;
    +            // check the size limit
    +            if (bytesRead > (maxSize-bytesReadSoFar) && bytesReadSoFar > 0) {
    +                in.reset();
    +                if (buffer != null) {
    +                    buffer.close();
                     }
    -
    -                out.write(nextByte);
    +                return new EndOfLineMarker(-1, eolBuffer, false, leadingNewLineBytes);
                 }
     
                 // if we have a new line, then we're done
                 if (nextByte == '\n') {
    -                eolBuffer.addEndOfLine(lastByte == '\r', true);
    +                if (buffer != null) {
    +                    buffer.writeTo(out);
    +                    buffer.close();
    +                    eolBuffer.addEndOfLine(false, true);  //TODO: verify "false" is equivalent to "lastByte == '\r'"
    --- End diff --
    
    Leftover comment with no current significance. Removed.


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

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


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65440697
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitText.java ---
    @@ -39,6 +39,303 @@
                 + "\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nLastLine\n";
     
         @Test
    +    public void testLastLineExceedsSizeLimit() {
    +        final TestRunner runner = TestRunners.newTestRunner(new SplitText());
    +        runner.setProperty(SplitText.HEADER_LINE_COUNT, "0");
    +        runner.setProperty(SplitText.LINE_SPLIT_COUNT, "2");
    +        runner.setProperty(SplitText.FRAGMENT_MAX_SIZE, "20 B");
    +
    +        runner.enqueue("Line #1\nLine #2\nLine #3\nLong line exceeding limit");
    +        runner.run();
    +
    +        runner.assertTransferCount(SplitText.REL_FAILURE, 0);
    +        runner.assertTransferCount(SplitText.REL_ORIGINAL, 1);
    +        runner.assertTransferCount(SplitText.REL_SPLITS, 3);
    +    }
    --- End diff --
    
    I examined the content of each output FlowFile.  The first and third look fine, but the second FlowFile had a trailing newline even though the dreaded REMOVE_TRAILING_NEWLINES defaults to "true".


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65467195
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
    --- End diff --
    
    removed lastByte variable


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65425886
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -147,138 +193,200 @@ protected void init(final ProcessorInitializationContext context) {
             return properties;
         }
     
    -    private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines, final byte[] leadingNewLineBytes) throws IOException {
    +    private int readLines(final InputStream in, final int maxNumLines, final long maxByteCount, final OutputStream out,
    +                          final boolean includeLineDelimiter, final byte[] leadingNewLineBytes) throws IOException {
             final EndOfLineBuffer eolBuffer = new EndOfLineBuffer();
     
    -        int numLines = 0;
             byte[] leadingBytes = leadingNewLineBytes;
    +        int numLines = 0;
    +        long totalBytes = 0L;
             for (int i = 0; i < maxNumLines; i++) {
    -            final EndOfLineMarker eolMarker = locateEndOfLine(in, out, false, eolBuffer, leadingBytes);
    +            final EndOfLineMarker eolMarker = countBytesToSplitPoint(in, out, totalBytes, maxByteCount, includeLineDelimiter, eolBuffer, leadingBytes);
    +            final long bytes = eolMarker.getBytesConsumed();
                 leadingBytes = eolMarker.getLeadingNewLineBytes();
     
    -            if (keepAllNewLines && out != null) {
    +            if (includeLineDelimiter && out != null) {
                     if (leadingBytes != null) {
                         out.write(leadingBytes);
                         leadingBytes = null;
                     }
    -
                     eolBuffer.drainTo(out);
                 }
    -
    -            if (eolBuffer.length() > 0 || eolMarker.getBytesConsumed() > 0L) {
    -                numLines++;
    +            totalBytes += bytes;
    +            if (bytes <= 0) {
    +                return numLines;
                 }
    -
    -            if (eolMarker.isStreamEnded()) {
    +            numLines++;
    +            if (totalBytes >= maxByteCount) {
                     break;
                 }
             }
    -
             return numLines;
         }
     
    -    private EndOfLineMarker locateEndOfLine(final InputStream in, final OutputStream out, final boolean includeLineDelimiter,
    -        final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
    -
    +    private EndOfLineMarker countBytesToSplitPoint(final InputStream in, final OutputStream out, final long bytesReadSoFar, final long maxSize,
    +                                                   final boolean includeLineDelimiter, final EndOfLineBuffer eolBuffer, final byte[] leadingNewLineBytes) throws IOException {
             int lastByte = -1;
             long bytesRead = 0L;
    +        final ByteArrayOutputStream buffer;
    +        if (out != null) {
    +            buffer = new ByteArrayOutputStream();
    +        } else {
    +            buffer = null;
    +        }
             byte[] bytesToWriteFirst = leadingNewLineBytes;
     
    +        in.mark(Integer.MAX_VALUE);
             while (true) {
    -            in.mark(1);
                 final int nextByte = in.read();
     
    -            final boolean isNewLineChar = nextByte == '\r' || nextByte == '\n';
    -
    -            // if we hit end of stream or new line we're done
    +            // if we hit end of stream we're done
                 if (nextByte == -1) {
    -                if (lastByte == '\r') {
    -                    eolBuffer.addEndOfLine(true, false);
    +                if (buffer != null) {
    +                    buffer.writeTo(out);
    +                    buffer.close();
                     }
    -
    -                return new EndOfLineMarker(bytesRead, eolBuffer, true, bytesToWriteFirst);
    +                return new EndOfLineMarker(bytesRead, eolBuffer, true, bytesToWriteFirst);  // bytesToWriteFirst should be "null"?
                 }
     
    -            // If we get a character that's not an end-of-line char, then
    -            // we need to write out the EOL's that we have buffered (if out != null).
    -            // Then, we need to reset our EOL buffer because we no longer have consecutive EOL's
    -            if (!isNewLineChar) {
    +            // Verify leading bytes do not violate size limitation
    +            if (bytesToWriteFirst != null && (bytesToWriteFirst.length + bytesRead) > (maxSize - bytesReadSoFar) && includeLineDelimiter) {
    +                return new EndOfLineMarker(-1, eolBuffer, false, leadingNewLineBytes);
    +            }
    +            // Write leadingNewLines, if appropriate
    +            if ( buffer != null && includeLineDelimiter && bytesToWriteFirst != null) {
    +                bytesRead += bytesToWriteFirst.length;
    +                buffer.write(bytesToWriteFirst);
    +                bytesToWriteFirst = null;
    +            }
    +            // buffer the output
    +            bytesRead++;
    +            if (buffer != null && nextByte != '\n' && nextByte != '\r') {
                     if (bytesToWriteFirst != null) {
    -                    if (out != null) {
    -                        out.write(bytesToWriteFirst);
    -                    }
    -
    -                    bytesToWriteFirst = null;
    -                }
    -
    -                if (out != null) {
    -                    eolBuffer.drainTo(out);
    +                    buffer.write(bytesToWriteFirst);
                     }
    -
    +                bytesToWriteFirst = null;
    +                eolBuffer.drainTo(buffer);
                     eolBuffer.clear();
    +                buffer.write(nextByte);
                 }
     
    -            // if there's an OutputStream to copy the data to, copy it, if appropriate.
    -            // "if appropriate" means that it's not a line delimiter or that we want to copy line delimiters
    -            bytesRead++;
    -            if (out != null && (includeLineDelimiter || !isNewLineChar)) {
    -                if (bytesToWriteFirst != null) {
    -                    out.write(bytesToWriteFirst);
    -                    bytesToWriteFirst = null;
    +            // check the size limit
    +            if (bytesRead > (maxSize-bytesReadSoFar) && bytesReadSoFar > 0) {
    +                in.reset();
    +                if (buffer != null) {
    +                    buffer.close();
                     }
    -
    -                out.write(nextByte);
    +                return new EndOfLineMarker(-1, eolBuffer, false, leadingNewLineBytes);
                 }
     
                 // if we have a new line, then we're done
                 if (nextByte == '\n') {
    -                eolBuffer.addEndOfLine(lastByte == '\r', true);
    +                if (buffer != null) {
    +                    buffer.writeTo(out);
    +                    buffer.close();
    +                    eolBuffer.addEndOfLine(false, true);  //TODO: verify "false" is equivalent to "lastByte == '\r'"
    --- End diff --
    
    Is this TODO a reminder to future @markobean?  Does a current unit test verify it?


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

[GitHub] nifi pull request #444: NIFI-1118 adding SplitText features of size limit an...

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

    https://github.com/apache/nifi/pull/444#discussion_r65414058
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java ---
    @@ -94,12 +118,16 @@
                 .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
                 .defaultValue("0")
                 .build();
    +    public static final PropertyDescriptor HEADER_MARKER = new PropertyDescriptor.Builder()
    +            .name("Header Line Marker Characters")
    +            .description("The first character(s) on the line of the datafile which signifies a header line. This value is ignored when Header Line Count is non-zero. " +
    +                    "The first line not containing the Header Line Marker Characters and all subsequent lines are considered non-header")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .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. If this is set to "
    -                + "'true' and a FlowFile is generated that contains only 'empty lines' (i.e., consists only of \r and \n characters), the FlowFile will not be emitted. "
    -                + "Note, however, that if the Header Line Count is greater than 0, the resultant FlowFile will never be empty as it will consist of the header lines, so "
    -                + "a FlowFile may be emitted that contians only the header lines.")
    +            .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.")
    --- End diff --
    
    @markobean I think the longer description is still valid and should remain in place.


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