This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new c97510aa66 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
c97510aa66 is described below

commit c97510aa66727509b74e7c3d2396ccb3d77f41ad
Author: Mark Payne <[email protected]>
AuthorDate: Thu Oct 10 13:39:22 2024 -0400

    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 <[email protected]>
---
 .../org/apache/nifi/wali/SequentialAccessWriteAheadLog.java  | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)

diff --git 
a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/SequentialAccessWriteAheadLog.java
 
b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/SequentialAccessWriteAheadLog.java
index b3fb267573..a71b05c489 100644
--- 
a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/SequentialAccessWriteAheadLog.java
+++ 
b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/SequentialAccessWriteAheadLog.java
@@ -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();
                 }
 

Reply via email to