wchevreuil commented on a change in pull request #749: HBASE-23205 Correctly
update the position of WALs currently being replicated
URL: https://github.com/apache/hbase/pull/749#discussion_r348092931
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -135,59 +127,46 @@ public void run() {
try (WALEntryStream entryStream =
new WALEntryStream(logQueue, fs, conf, currentPosition, metrics)) {
while (isReaderRunning()) { // loop here to keep reusing stream while
we can
- if (!checkQuota()) {
+ if (manager.isBufferQuotaReached()) {
+ Threads.sleep(sleepForRetries);
continue;
}
- WALEntryBatch batch = null;
- while (entryStream.hasNext()) {
- if (batch == null) {
- batch = new WALEntryBatch(replicationBatchCountCapacity,
entryStream.getCurrentPath());
- }
+ WALEntryBatch batch =
+ new WALEntryBatch(replicationBatchCountCapacity,
replicationBatchSizeCapacity);
+ boolean hasNext;
+ while ((hasNext = entryStream.hasNext()) == true) {
Entry entry = entryStream.next();
entry = filterEntry(entry);
if (entry != null) {
WALEdit edit = entry.getEdit();
if (edit != null && !edit.isEmpty()) {
- long entrySize = getEntrySizeIncludeBulkLoad(entry);
- long entrySizeExlucdeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
- batch.addEntry(entry);
- replicationSourceManager.setPendingShipment(true);
- updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
- boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExlucdeBulkLoad);
+ long entrySizeExcludeBulkLoad = batch.addEntry(entry);
+ boolean totalBufferTooLarge =
manager.acquireBufferQuota(entrySizeExcludeBulkLoad);
// Stop if too many entries or too big
- if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
- || batch.getNbEntries() >= replicationBatchCountCapacity) {
+ if (totalBufferTooLarge || batch.isLimitReached()) {
break;
}
}
- } else {
-
replicationSourceManager.logPositionAndCleanOldLogs(entryStream.getCurrentPath(),
- this.replicationQueueInfo.getPeerClusterZnode(),
- entryStream.getPosition(),
- this.replicationQueueInfo.isQueueRecovered(), false);
}
}
- if (batch != null && (!batch.getLastSeqIds().isEmpty() ||
batch.getNbEntries() > 0)) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(String.format("Read %s WAL entries eligible for
replication",
- batch.getNbEntries()));
- }
- entryBatchQueue.put(batch);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(String.format("Read %s WAL entries eligible for
replication",
+ batch.getNbEntries()));
+ }
+
+ updateBatch(entryStream, batch, hasNext);
+ if (isShippable(batch)) {
sleepMultiplier = 1;
- } else { // got no entries and didn't advance position in WAL
- LOG.trace("Didn't read any new entries from WAL");
- if (replicationQueueInfo.isQueueRecovered()) {
- // we're done with queue recovery, shut ourself down
+ entryBatchQueue.put(batch);
+ if (!batch.hasMoreEntries()) {
+ // we're done with queue recovery, shut ourselves down
setReaderRunning(false);
- // shuts down shipper thread immediately
- entryBatchQueue.put(batch != null ? batch
- : new WALEntryBatch(replicationBatchCountCapacity,
entryStream.getCurrentPath()));
- } else {
- Thread.sleep(sleepForRetries);
}
+ } else {
Review comment:
Ok, so _batch.hasMoreEntries()_ returns true if this is isn't a recovered
queue.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services