mirror of https://github.com/apache/nifi.git
NIFI-4439: This closes #2190. When a Provenance Event File is rolled over, we were failing to close the resource before attempting to compress it. Fixed that.
NIFI-4439: Addressed threading bug that can occur when rolling over provenance record writer Signed-off-by: joewitt <joewitt@apache.org>
This commit is contained in:
parent
ba8f17bac0
commit
b950eed1a5
|
@ -253,10 +253,18 @@ public class WriteAheadStorePartition implements EventStorePartition {
|
||||||
final long nextEventId = idGenerator.get();
|
final long nextEventId = idGenerator.get();
|
||||||
final File updatedEventFile = new File(partitionDirectory, nextEventId + ".prov");
|
final File updatedEventFile = new File(partitionDirectory, nextEventId + ".prov");
|
||||||
final RecordWriter updatedWriter = recordWriterFactory.createWriter(updatedEventFile, idGenerator, false, true);
|
final RecordWriter updatedWriter = recordWriterFactory.createWriter(updatedEventFile, idGenerator, false, true);
|
||||||
|
|
||||||
|
// Synchronize on the writer to ensure that no other thread is able to obtain the writer and start writing events to it until after it has
|
||||||
|
// been fully initialized (i.e., the header has been written, etc.)
|
||||||
|
synchronized (updatedWriter) {
|
||||||
final RecordWriterLease updatedLease = new RecordWriterLease(updatedWriter, config.getMaxEventFileCapacity(), config.getMaxEventFileCount());
|
final RecordWriterLease updatedLease = new RecordWriterLease(updatedWriter, config.getMaxEventFileCapacity(), config.getMaxEventFileCount());
|
||||||
final boolean updated = eventWriterLeaseRef.compareAndSet(lease, updatedLease);
|
final boolean updated = eventWriterLeaseRef.compareAndSet(lease, updatedLease);
|
||||||
|
|
||||||
if (updated) {
|
if (updated) {
|
||||||
|
if (lease != null) {
|
||||||
|
lease.close();
|
||||||
|
}
|
||||||
|
|
||||||
updatedWriter.writeHeader(nextEventId);
|
updatedWriter.writeHeader(nextEventId);
|
||||||
|
|
||||||
synchronized (minEventIdToPathMap) {
|
synchronized (minEventIdToPathMap) {
|
||||||
|
@ -287,6 +295,7 @@ public class WriteAheadStorePartition implements EventStorePartition {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private Map<ProvenanceEventRecord, StorageSummary> addEvents(final Iterable<ProvenanceEventRecord> events, final RecordWriter writer) throws IOException {
|
private Map<ProvenanceEventRecord, StorageSummary> addEvents(final Iterable<ProvenanceEventRecord> events, final RecordWriter writer) throws IOException {
|
||||||
final Map<ProvenanceEventRecord, StorageSummary> locationMap = new HashMap<>();
|
final Map<ProvenanceEventRecord, StorageSummary> locationMap = new HashMap<>();
|
||||||
|
|
Loading…
Reference in New Issue