ankitsol commented on code in PR #7617:
URL: https://github.com/apache/hbase/pull/7617#discussion_r2827254931
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java:
##########
@@ -154,15 +196,16 @@ protected void postFinish() {
private void shipEdits(WALEntryBatch entryBatch) {
List<Entry> entries = entryBatch.getWalEntries();
int sleepMultiplier = 0;
- if (entries.isEmpty()) {
- updateLogPosition(entryBatch);
- return;
- }
int currentSize = (int) entryBatch.getHeapSize();
source.getSourceMetrics()
.setTimeStampNextToReplicate(entries.get(entries.size() -
1).getKey().getWriteTime());
while (isActive()) {
try {
+ if (entries.isEmpty()) {
Review Comment:
Currently in `shipEdits()`, the only place where an IOException can be
raised is from `persistLogPosition()` (via
`beforePersistingReplicationOffset()` or `cleanUpHFileRefs()`).
Based on your earlier comment about restarting from the last persisted
offset and resending WAL entries, I am thinking of splitting the existing catch
block into two parts:
```
catch (IOException ioe) {
// Persist failure → fatal → restart worker → WAL replay from last
persisted offset
throw new ReplicationRuntimeException(
"Failed to persist replication offset; restarting worker to replay
WAL", ioe);
} catch (Exception ex) {
// Replication/transient failure → retry current batch (existing behaviour)
source.getSourceMetrics().incrementFailedBatches();
LOG.warn("{} threw unknown exception:",
source.getReplicationEndpoint().getClass().getName(), ex);
if (sleepForRetries("ReplicationEndpoint threw exception",
sleepForRetries, sleepMultiplier, maxRetriesMultiplier)) {
sleepMultiplier++;
}
}
```
The intention is that propagating `ReplicationRuntimeException` will cause
the shipper worker to exit, and ReplicationSource will recreate it, so WAL
reading resumes from the last persisted offset and all batches since then are
resent.
Does this approach match what you had in mind? @Apache9
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]