Skip to content

Conversation

@ankitsol
Copy link

@ankitsol ankitsol commented Jan 12, 2026

ReplicationSourceShipper currently persists replication offsets immediately after shipping WAL batches. This is unsafe for buffering ReplicationEndpoints (e.g. ContinuousBackupReplicationEndpoint) that stage WAL entries before durably flushing them.

This change adds staged flush support by:

  • Tracking staged WAL size and last flush time in ReplicationSourceShipper
  • Adding optional buffer size and flush interval hooks to ReplicationEndpoint

Introducing beforePersistingReplicationOffset() to allow endpoints to flush staged WAL data before offsets are persisted

Default implementations preserve existing behaviour for non-buffering endpoints, which continue to persist offsets immediately.

Added TestReplicationSourceShipperBufferedFlush

  • Verifies beforePersistingReplicationOffset() is not called for empty batches

@Apache-HBase

This comment has been minimized.

@Apache-HBase

This comment has been minimized.

@ankitsol
Copy link
Author

@Apache9 @anmolnar @vinayakphegde Please review this PR as followup to preview PR #7591

@Apache-HBase
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 28s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 hbaseanti 0m 0s Patch does not have any anti-patterns.
_ master Compile Tests _
+1 💚 mvninstall 3m 39s master passed
+1 💚 compile 4m 9s master passed
+1 💚 checkstyle 1m 10s master passed
+1 💚 spotbugs 2m 5s master passed
+1 💚 spotless 1m 4s branch has no errors when running spotless:check.
_ Patch Compile Tests _
+1 💚 mvninstall 3m 45s the patch passed
+1 💚 compile 4m 8s the patch passed
+1 💚 javac 4m 8s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 1m 10s the patch passed
+1 💚 spotbugs 2m 28s the patch passed
+1 💚 hadoopcheck 13m 50s Patch does not cause any errors with Hadoop 3.3.6 3.4.1.
+1 💚 spotless 0m 56s patch has no errors when running spotless:check.
_ Other Tests _
+1 💚 asflicense 0m 13s The patch does not generate ASF License warnings.
48m 15s
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7617/2/artifact/yetus-general-check/output/Dockerfile
GITHUB PR #7617
Optional Tests dupname asflicense javac spotbugs checkstyle codespell detsecrets compile hadoopcheck hbaseanti spotless
uname Linux 4f1801e5c24c 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/hbase-personality.sh
git revision master / 95000e0
Default Java Eclipse Adoptium-17.0.11+9
Max. process+thread count 83 (vs. ulimit of 30000)
modules C: hbase-server U: hbase-server
Console output https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7617/2/console
versions git=2.34.1 maven=3.9.8 spotbugs=4.7.3
Powered by Apache Yetus 0.15.0 https://yetus.apache.org

This message was automatically generated.

@Apache-HBase
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 14s Docker mode activated.
-0 ⚠️ yetus 0m 3s Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --author-ignore-list --blanks-eol-ignore-file --blanks-tabs-ignore-file --quick-hadoopcheck
_ Prechecks _
_ master Compile Tests _
+1 💚 mvninstall 3m 19s master passed
+1 💚 compile 0m 59s master passed
+1 💚 javadoc 0m 30s master passed
+1 💚 shadedjars 6m 20s branch has no errors when building our shaded downstream artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 2m 55s the patch passed
+1 💚 compile 1m 0s the patch passed
+1 💚 javac 1m 0s the patch passed
+1 💚 javadoc 0m 27s the patch passed
+1 💚 shadedjars 6m 20s patch has no errors when building our shaded downstream artifacts.
_ Other Tests _
-1 ❌ unit 251m 26s /patch-unit-hbase-server.txt hbase-server in the patch failed.
277m 59s
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7617/2/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile
GITHUB PR #7617
Optional Tests javac javadoc unit compile shadedjars
uname Linux 82b2e4b8e507 6.14.0-1018-aws #18~24.04.1-Ubuntu SMP Mon Nov 24 19:46:27 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/hbase-personality.sh
git revision master / 95000e0
Default Java Eclipse Adoptium-17.0.11+9
Test Results https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7617/2/testReport/
Max. process+thread count 4718 (vs. ulimit of 30000)
modules C: hbase-server U: hbase-server
Console output https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-7617/2/console
versions git=2.34.1 maven=3.9.8
Powered by Apache Yetus 0.15.0 https://yetus.apache.org

This message was automatically generated.

Copy link

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

This PR adds staged flush support to ReplicationSourceShipper to enable safe offset persistence for buffering ReplicationEndpoints (like ContinuousBackupReplicationEndpoint). The shipper now tracks staged WAL size and flush timing, only persisting offsets after buffered data is durably flushed.

Changes:

  • Added buffer management hooks (getMaxBufferSize(), maxFlushInterval(), beforePersistingReplicationOffset()) to ReplicationEndpoint interface
  • Modified ReplicationSourceShipper to stage WAL entries and defer offset persistence based on buffer size and time thresholds
  • Added test to verify beforePersistingReplicationOffset() is not called for empty batches

Reviewed changes

Copilot reviewed 3 out of 3 changed files in this pull request and generated 7 comments.

File Description
ReplicationEndpoint.java Added three default methods to support buffered replication: getMaxBufferSize(), maxFlushInterval(), and beforePersistingReplicationOffset()
ReplicationSourceShipper.java Modified to track staged WAL entries and defer offset persistence until buffer/timeout thresholds are met
TestReplicationSourceShipperBufferedFlush.java Added test to verify beforePersistingReplicationOffset() behavior with empty batches

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

for (Entry entry : entriesForCleanUpHFileRefs) {
try {
cleanUpHFileRefs(entry.getEdit());
} catch (IOException e) {
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The error handling for IOException has been changed to catch and log the exception instead of propagating it. This silently suppresses IOException failures during cleanup, which could hide serious issues like file system problems. If cleanup failures should be non-fatal, this should be explicitly documented, or consider at least incrementing a failure metric to track these errors.

Suggested change
} catch (IOException e) {
} catch (IOException e) {
// Cleanup failures are intentionally treated as non-fatal: replication has already
// succeeded for these entries, so we log the failure and continue.

Copilot uses AI. Check for mistakes.
if (stagedWalSize == 0 || lastShippedBatch == null) {
return false;
}
return (stagedWalSize >= source.getReplicationEndpoint().getMaxBufferSize())
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The condition checks if stagedWalSize is greater than or equal to getMaxBufferSize(), but getMaxBufferSize() can return -1 for non-buffering endpoints (as documented in ReplicationEndpoint). This means the comparison 'stagedWalSize >= -1' would always be true when stagedWalSize > 0, causing immediate flushes for non-buffering endpoints. While this preserves existing behavior, the intent is unclear and could be confusing. Consider explicitly checking for -1 to make the logic more explicit.

Suggested change
return (stagedWalSize >= source.getReplicationEndpoint().getMaxBufferSize())
long maxBufferSize = source.getReplicationEndpoint().getMaxBufferSize();
// For non-buffering endpoints, getMaxBufferSize() returns a negative value (e.g., -1).
// In that case, we always trigger a flush based on size as soon as there is staged data.
boolean sizeBasedFlush =
(maxBufferSize < 0) || (stagedWalSize >= maxBufferSize);
return sizeBasedFlush

Copilot uses AI. Check for mistakes.
Comment on lines +123 to +126
// Allow loop to run
Waiter.waitFor(conf, 3000, () -> true);

shipper.interrupt();
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The test uses a fixed wait time of 3000ms with an empty lambda that always returns true (Waiter.waitFor(conf, 3000, () -> true)). This just sleeps unconditionally and doesn't actually verify any condition. The test should wait for a meaningful condition, such as verifying that the shipper has processed the batch or checking that the thread has reached a specific state. This makes the test timing-dependent and unreliable.

Suggested change
// Allow loop to run
Waiter.waitFor(conf, 3000, () -> true);
shipper.interrupt();
// Wait until the shipper thread has finished processing the batch
Waiter.waitFor(conf, 3000, () -> !shipper.isAlive());

Copilot uses AI. Check for mistakes.
batch.setLastWalPosition(1L);
// no entries, no heap size

Mockito.when(walReader.take()).thenReturn(batch).thenReturn(null);
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The mock is configured to stub 'walReader.take()' but the ReplicationSourceShipper actually calls 'entryReader.poll(getEntriesTimeout)' (line 118 in ReplicationSourceShipper.java). This means the mock configuration has no effect, and the test will call the unmocked poll() method which will return null, causing the shipper to skip processing any batches. The test should mock 'poll(anyInt())' instead of 'take()'.

Suggested change
Mockito.when(walReader.take()).thenReturn(batch).thenReturn(null);
Mockito.when(walReader.poll(Mockito.anyInt())).thenReturn(batch).thenReturn(null);

Copilot uses AI. Check for mistakes.
// ContinuousBackupReplicationEndpoint
// and -1 for other ReplicationEndpoint since they don't buffer.
// For other ReplicationEndpoint, everytime a WALEntryBatch is shipped, we update replication
// offset. Please check ReplicationSourceShipper#shouldFlushStagedWal()
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment references 'shouldFlushStagedWal()' but the actual method name in ReplicationSourceShipper is 'shouldPersistLogPosition()'. This inconsistency will confuse developers trying to understand the interaction between these components.

Copilot uses AI. Check for mistakes.
// ContinuousBackupReplicationEndpoint
// and Long.MAX_VALUE for other ReplicationEndpoint since they don't buffer.
// For other ReplicationEndpoint, everytime a WALEntryBatch is shipped, we update replication
// offset. Please check ReplicationSourceShipper#shouldFlushStagedWal()
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment references 'shouldFlushStagedWal()' but the actual method name in ReplicationSourceShipper is 'shouldPersistLogPosition()'. This inconsistency will confuse developers trying to understand the interaction between these components.

Copilot uses AI. Check for mistakes.
LOG.info("Running ReplicationSourceShipper Thread for wal group: {}", this.walGroupId);
// Loop until we close down
while (isActive()) {
// check if flush needed for WAL backup, this is need for timeout based flush
Copy link

Copilot AI Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment has a typo: 'need' should be 'needed'. Should read "this is needed for timeout based flush".

Suggested change
// check if flush needed for WAL backup, this is need for timeout based flush
// check if flush needed for WAL backup, this is needed for timeout based flush

Copilot uses AI. Check for mistakes.
Copy link
Contributor

@taklwu taklwu left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you also check if these two failures are related?

TestHRegionWithInMemoryFlush.testParallelIncrementWithMemStoreFlush
TestTags.testFlushAndCompactionwithCombinations

return -1;
}

// WAL entries are buffered in ContinuousBackupReplicationEndpoint before flushing to WAL backup
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: ContinuousBackupReplicationEndpoint is part of #7591 and it's yet committing to master, should we mention early in this change?

Comment on lines +265 to +268
} catch (IOException e) {
LOG.warn("{} threw unknown exception:",
source.getReplicationEndpoint().getClass().getName(), e);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: will this be a behavior change because previously when cleanUpHFileRefs failed, it's throwing thru the function but here we're logging it only.

@taklwu taklwu requested a review from Apache9 January 13, 2026 01:10
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants