-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-29441 ReplicationSourceShipper should delegate the empty wal entries handling to ReplicationEndpoint #7145
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
d37a5bf
0ffeddc
9bcd1b2
be71bb0
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,34 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.hadoop.hbase.replication; | ||
|
||
import org.apache.yetus.audience.InterfaceAudience; | ||
|
||
/** | ||
* Policy that defines what a replication endpoint should do when the entry batch is empty. This is | ||
* used to determine whether the replication source should consider an empty batch as: - | ||
* {@code COMMIT}: Consider the position as fully committed, and update the WAL position. - | ||
* {@code SUBMIT}: Treat it as submitted but not committed, i.e., do not advance the WAL position. | ||
* Some endpoints may buffer entries (e.g., in open files on S3) and delay actual persistence. In | ||
* such cases, an empty batch should not result in WAL position commit. | ||
*/ | ||
@InterfaceAudience.Private | ||
public enum EmptyEntriesPolicy { | ||
COMMIT, | ||
SUBMIT | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,13 +20,15 @@ | |
import static org.apache.hadoop.hbase.replication.ReplicationUtils.getAdaptiveTimeout; | ||
import static org.apache.hadoop.hbase.replication.ReplicationUtils.sleepForRetries; | ||
|
||
import com.google.errorprone.annotations.RestrictedApi; | ||
import java.io.IOException; | ||
import java.util.List; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.hadoop.hbase.Cell; | ||
import org.apache.hadoop.hbase.CellUtil; | ||
import org.apache.hadoop.hbase.HConstants; | ||
import org.apache.hadoop.hbase.replication.EmptyEntriesPolicy; | ||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint; | ||
import org.apache.hadoop.hbase.replication.ReplicationResult; | ||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; | ||
|
@@ -150,13 +152,25 @@ protected void postFinish() { | |
} | ||
|
||
/** | ||
* Do the shipping logic | ||
* Do the shipping logic. | ||
*/ | ||
private void shipEdits(WALEntryBatch entryBatch) { | ||
@RestrictedApi( | ||
explanation = "Package-private for test visibility only. Do not use outside tests.", | ||
link = "", | ||
allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java)") | ||
void shipEdits(WALEntryBatch entryBatch) { | ||
List<Entry> entries = entryBatch.getWalEntries(); | ||
int sleepMultiplier = 0; | ||
if (entries.isEmpty()) { | ||
updateLogPosition(entryBatch, ReplicationResult.COMMITTED); | ||
/* | ||
* Delegate to the endpoint to decide how to treat empty entry batches. In most replication | ||
* flows, receiving an empty entry batch means that everything so far has been successfully | ||
* replicated and committed — so it's safe to mark the WAL position as committed (COMMIT). | ||
* However, some endpoints (e.g., asynchronous S3 backups) may buffer writes and delay actual | ||
* persistence. In such cases, we must avoid committing the WAL position prematurely. | ||
*/ | ||
final ReplicationResult result = getReplicationResult(); | ||
updateLogPosition(entryBatch, result); | ||
return; | ||
} | ||
int currentSize = (int) entryBatch.getHeapSize(); | ||
|
@@ -232,6 +246,13 @@ private void shipEdits(WALEntryBatch entryBatch) { | |
} | ||
} | ||
|
||
private ReplicationResult getReplicationResult() { | ||
EmptyEntriesPolicy policy = source.getReplicationEndpoint().getEmptyEntriesPolicy(); | ||
return (policy == EmptyEntriesPolicy.COMMIT) | ||
? ReplicationResult.COMMITTED | ||
: ReplicationResult.SUBMITTED; | ||
} | ||
|
||
private void cleanUpHFileRefs(WALEdit edit) throws IOException { | ||
String peerId = source.getPeerId(); | ||
if (peerId.contains("-")) { | ||
|
@@ -256,7 +277,11 @@ private void cleanUpHFileRefs(WALEdit edit) throws IOException { | |
} | ||
} | ||
|
||
private boolean updateLogPosition(WALEntryBatch batch, ReplicationResult replicated) { | ||
@RestrictedApi( | ||
explanation = "Package-private for test visibility only. Do not use outside tests.", | ||
link = "", | ||
allowedOnPath = "(.*/src/test/.*|.*/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java)") | ||
boolean updateLogPosition(WALEntryBatch batch, ReplicationResult replicated) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. VisibleForTesting? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it.
|
||
boolean updated = false; | ||
// if end of file is true, then the logPositionAndCleanOldLogs method will remove the file | ||
// record on zk, so let's call it. The last wal position maybe zero if end of file is true and | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
[nitpick] Consider annotating this package-private method with @VisibleForTesting (or a project-specific equivalent) to clearly signal its test-only visibility.
Copilot uses AI. Check for mistakes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: VisibleForTesting seems the right comment, and it may save few words for testing only,