[
https://issues.apache.org/jira/browse/NIFI-2854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15671340#comment-15671340
]
ASF GitHub Bot commented on NIFI-2854:
--------------------------------------
Github user olegz commented on a diff in the pull request:
https://github.com/apache/nifi/pull/1202#discussion_r88309841
--- Diff:
nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
---
@@ -512,25 +525,38 @@ public synchronized int checkpoint() throws
IOException {
swapLocations = new HashSet<>(externalLocations);
for (final Partition<T> partition : partitions) {
try {
- partition.rollover();
+ partitionStreams.add(partition.rollover());
} catch (final Throwable t) {
partition.blackList();
numberBlackListedPartitions.getAndIncrement();
throw t;
}
}
-
- // notify global sync with the write lock held. We do this
because we don't want the repository to get updated
- // while the listener is performing its necessary tasks
- if (syncListener != null) {
- syncListener.onGlobalSync();
- }
} finally {
writeLock.unlock();
}
stopTheWorldNanos = System.nanoTime() - stopTheWorldStart;
+ // Close all of the Partitions' Output Streams. We do this
here, instead of in Partition.rollover()
+ // because we want to do this outside of the write lock.
Because calling close() on FileOutputStream can
+ // be very expensive, as it has to flush the data to disk, we
don't want to prevent other Process Sessions
+ // from getting committed. Since rollover() transitions the
partition to write to a new file already, there
+ // is no reason that we need to close this FileOutputStream
before releasing the write lock. Also, if any Exception
+ // does get thrown when calling close(), we don't need to
blacklist the partition, as the stream that was getting
+ // closed is not the stream being written to for the partition
anyway.
+ for (final OutputStream partitionStream : partitionStreams) {
+ partitionStream.close();
--- End diff --
What if ```close()``` fails (i.e., IOException) on one of the streams in
the loop? Perhaps wrapping in try/catch?
> Enable repositories to support upgrades and rollback in well defined scenarios
> ------------------------------------------------------------------------------
>
> Key: NIFI-2854
> URL: https://issues.apache.org/jira/browse/NIFI-2854
> Project: Apache NiFi
> Issue Type: Improvement
> Components: Core Framework
> Reporter: Mark Payne
> Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> The flowfile, swapfile, provenance, and content repositories play a very
> important roll in NiFi's ability to be safely upgraded and rolled back. We
> need to have well documented behaviors, designs, and version adherence so
> that users can safely rely on these mechanisms.
> Once this is formalized and in place we should update our versioning guidance
> to reflect this as well.
> The following would be true from NiFi 1.2.0 onward
> * No changes to how the repositories are persisted to disk can be made which
> will break forward/backward compatibility and specifically this means that
> things like the way each is serialized to disk cannot change.
> * If changes are made which impact forward or backward compatibility they
> should be reserved for major releases only and should include a utility to
> help users with pre-existing data convert from some older format to the newer
> format. It may not be feasible to have rollback on major releases.
> * The content repository should not be changed within a major release cycle
> in any way that will harm forward or backward compatibility.
> * The flow file repository can change in that new fields can be added to
> existing write ahead log record types but no fields can be removed nor can
> any new types be added. Once a field is considered required it must remain
> required. Changes may only be made across minor version changes - not
> incremental.
> * Swap File storage should follow very similar rules to the flow file
> repository. Adding a schema to the swap file header may allow some variation
> there but the variation should only be hints to optimize how they're
> processed and not change their behavior otherwise. Changes are only permitted
> during minor version releases.
> * Provenance repository changes are only permitted during minor version
> releases. These changes may include adding or removing fields from existing
> event types. If a field is considered required it must always be considered
> required. If a field is removed then it must not be a required field and
> there must be a sensible default an older version could use if that value is
> not found in new data once rolled back. New event types may be added.
> Fields or event types not known to older version, if seen after a rollback,
> will simply be ignored.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)