You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2019/11/18 03:31:42 UTC

[GitHub] [hbase] JeongDaeKim commented on a change in pull request #749: HBASE-23205 Correctly update the position of WALs currently being replicated

JeongDaeKim 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_r347190247
 
 

 ##########
 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 {
 
 Review comment:
   > any sub-sequent non-filterable one would need to wait for a log roll? That could take too long for some use cases.
   
   If `some use cases` means "no mutations come for a long time, but a batch has entries", this case is the one of the 1) case i mentioned `a batch has entries when eof reached`. reader would reach the eof, and log position would be updated.
   
   In addition, even while testing this issue with heavy writes, I observed the reader frequently reached EOF.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services