Repository: nifi Updated Branches: refs/heads/master fd92999da -> 091359b45
NIFI-3630 This closes #1632. Use a BufferedOutputStream when checkpointing FlowFile Repository Signed-off-by: joewitt <[email protected]> Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/091359b4 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/091359b4 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/091359b4 Branch: refs/heads/master Commit: 091359b450a7d0fb6bb04e2238c9171728cd2720 Parents: fd92999 Author: Mark Payne <[email protected]> Authored: Tue Mar 28 16:11:25 2017 -0400 Committer: joewitt <[email protected]> Committed: Thu Mar 30 16:46:53 2017 -0400 ---------------------------------------------------------------------- .../main/java/org/wali/MinimalLockingWriteAheadLog.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/nifi/blob/091359b4/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java ---------------------------------------------------------------------- diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java index 1a9e219..5334acb 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java @@ -569,7 +569,7 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor // perform checkpoint, writing to .partial file fileOut = new FileOutputStream(partialPath.toFile()); - dataOut = new DataOutputStream(fileOut); + dataOut = new DataOutputStream(new BufferedOutputStream(fileOut)); dataOut.writeUTF(MinimalLockingWriteAheadLog.class.getName()); dataOut.writeInt(getVersion()); dataOut.writeUTF(serde.getClass().getName()); @@ -590,9 +590,12 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor } finally { if (dataOut != null) { try { - dataOut.flush(); - fileOut.getFD().sync(); - dataOut.close(); + try { + dataOut.flush(); + fileOut.getFD().sync(); + } finally { + dataOut.close(); + } } catch (final IOException e) { logger.warn("Failed to close Data Stream due to {}", e.toString(), e); }
