mirror of https://github.com/apache/nifi.git
NIFI-13862: Ensure that when FlowFile Repository is checkpointed, we always call the onGlobalSync() method to ensure that any transient claims that may have been provided (such as those from Stateless NiFi) get dealt with appropriately.
This closes #9375 Signed-off-by: Joseph Witt <joewitt@apache.org>
This commit is contained in:
parent
ade260266e
commit
c97510aa66
|
@ -17,6 +17,12 @@
|
|||
|
||||
package org.apache.nifi.wali;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.wali.SerDeFactory;
|
||||
import org.wali.SyncListener;
|
||||
import org.wali.WriteAheadRepository;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
@ -32,11 +38,6 @@ import java.util.concurrent.locks.Lock;
|
|||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.regex.Pattern;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.wali.SerDeFactory;
|
||||
import org.wali.SyncListener;
|
||||
import org.wali.WriteAheadRepository;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -265,6 +266,7 @@ public class SequentialAccessWriteAheadLog<T> implements WriteAheadRepository<T>
|
|||
final JournalSummary journalSummary = journal.getSummary();
|
||||
if (journalSummary.getTransactionCount() == 0 && journal.isHealthy()) {
|
||||
logger.debug("Will not checkpoint Write-Ahead Log because no updates have occurred since last checkpoint");
|
||||
syncListener.onGlobalSync();
|
||||
return snapshot.getRecordCount();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue