apurtell commented on a change in pull request #2975:
URL: https://github.com/apache/hbase/pull/2975#discussion_r580624967
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -133,71 +134,126 @@ public
ReplicationSourceWALReaderThread(ReplicationSourceManager manager,
@Override
public void run() {
int sleepMultiplier = 1;
- while (isReaderRunning()) { // we only loop back here if something fatal
happened to our stream
- try (WALEntryStream entryStream =
- new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics)) {
- while (isReaderRunning()) { // loop here to keep reusing stream while
we can
- if (!source.isPeerEnabled()) {
- Threads.sleep(sleepForRetries);
- continue;
- }
- if (!checkQuota()) {
- continue;
- }
- WALEntryBatch batch = new
WALEntryBatch(replicationBatchCountCapacity);
- 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 entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
- batch.addEntry(entry, entrySize);
- updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
- boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
- // Stop if too many entries or too big
- if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
+ WALEntryBatch batch = null;
+ WALEntryStream entryStream =
+ new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics);
+ try {
Review comment:
There is a new outer try block added here...
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
##########
@@ -237,6 +237,10 @@ public void enqueueLog(Path log) {
}
}
+ public Map<String, PriorityBlockingQueue<Path>> getQueues() {
Review comment:
If this is for tests only, you can make this package scope
##########
File path:
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
##########
@@ -726,8 +727,62 @@ public void testEOFExceptionForRecoveredQueue() throws
Exception {
queue, 0, fs, conf, getDummyFilter(),
new MetricsSource("1"), (ReplicationSource) source);
reader.run();
+ assertEquals(0, queue.size());
+ }
+
+ @Test
Review comment:
Nice test
##########
File path:
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
##########
@@ -34,6 +34,8 @@
import static org.mockito.Mockito.when;
import static org.mockito.internal.verification.VerificationModeFactory.times;
+import com.google.common.collect.Lists;
Review comment:
This should come from hbase-thirdparty e.g.
org.apache.hbase.thirdparty.com.google.common.collect.Lists
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -133,71 +134,126 @@ public
ReplicationSourceWALReaderThread(ReplicationSourceManager manager,
@Override
public void run() {
int sleepMultiplier = 1;
- while (isReaderRunning()) { // we only loop back here if something fatal
happened to our stream
- try (WALEntryStream entryStream =
- new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics)) {
- while (isReaderRunning()) { // loop here to keep reusing stream while
we can
- if (!source.isPeerEnabled()) {
- Threads.sleep(sleepForRetries);
- continue;
- }
- if (!checkQuota()) {
- continue;
- }
- WALEntryBatch batch = new
WALEntryBatch(replicationBatchCountCapacity);
- 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 entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
- batch.addEntry(entry, entrySize);
- updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
- boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
- // Stop if too many entries or too big
- if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
+ WALEntryBatch batch = null;
+ WALEntryStream entryStream =
+ new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics);
+ try {
+ while (isReaderRunning()) { // we only loop back here if something fatal
happens to stream
+ try {
+ entryStream = new WALEntryStream(logQueue, fs, conf,
lastReadPosition, metrics);
+ while (isReaderRunning()) { // loop here to keep reusing stream
while we can
+ if (!source.isPeerEnabled()) {
+ Threads.sleep(sleepForRetries);
+ continue;
+ }
+ if (!checkQuota()) {
+ continue;
+ }
+ batch = new WALEntryBatch(replicationBatchCountCapacity);
+ boolean hasNext = entryStream.hasNext();
+ while (hasNext) {
+ Entry entry = entryStream.next();
+ entry = filterEntry(entry);
+ if (entry != null) {
+ WALEdit edit = entry.getEdit();
+ if (edit != null && !edit.isEmpty()) {
+ long entrySize = getEntrySizeIncludeBulkLoad(entry);
+ long entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
+ batch.addEntry(entry, entrySize);
+ updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
+ boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
+ // Stop if too many entries or too big
+ if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
|| batch.getNbEntries() >= replicationBatchCountCapacity) {
- break;
+ break;
+ }
}
}
+ hasNext = entryStream.hasNext();
}
- }
- updateBatch(entryStream, batch, hasNext);
- if (isShippable(batch)) {
- sleepMultiplier = 1;
- entryBatchQueue.put(batch);
- if (!batch.hasMoreEntries()) {
- // we're done with queue recovery, shut ourselves down
- setReaderRunning(false);
+ // If the batch has data to max capacity or stream doesn't have
anything
+ // try to ship it
+ if (isBatchQueuedToBeShipped(entryStream, batch, hasNext, false)) {
+ sleepMultiplier = 1;
}
+ }
+ } catch (IOException | WALEntryStreamRuntimeException e) { // stream
related
+ if (handleEofException(e, entryStream, batch)) {
+ sleepMultiplier = 1;
} else {
- Thread.sleep(sleepForRetries);
+ if (sleepMultiplier < maxRetriesMultiplier) {
+ LOG.debug("Failed to read stream of replication entries: " + e);
+ sleepMultiplier++;
+ } else {
+ LOG.error("Failed to read stream of replication entries", e);
+ }
+ Threads.sleep(sleepForRetries * sleepMultiplier);
}
- resetStream(entryStream);
+ } catch (InterruptedException e) {
+ LOG.trace("Interrupted while sleeping between WAL reads");
+ Thread.currentThread().interrupt();
+ } finally {
+ entryStream.close();
}
- } catch (IOException | WALEntryStreamRuntimeException e) { // stream
related
- if (sleepMultiplier < maxRetriesMultiplier) {
- LOG.debug("Failed to read stream of replication entries: " + e);
- sleepMultiplier++;
- } else {
- LOG.error("Failed to read stream of replication entries", e);
- handleEofException(e);
- }
- Threads.sleep(sleepForRetries * sleepMultiplier);
- } catch (InterruptedException e) {
- LOG.trace("Interrupted while sleeping between WAL reads");
- Thread.currentThread().interrupt();
}
+ } catch (IOException e) {
Review comment:
... and the catch clauses from the inner try have been moved out to the
outer try. That's the idea, right? Or did I misread this
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -133,71 +134,126 @@ public
ReplicationSourceWALReaderThread(ReplicationSourceManager manager,
@Override
public void run() {
int sleepMultiplier = 1;
- while (isReaderRunning()) { // we only loop back here if something fatal
happened to our stream
- try (WALEntryStream entryStream =
- new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics)) {
- while (isReaderRunning()) { // loop here to keep reusing stream while
we can
- if (!source.isPeerEnabled()) {
- Threads.sleep(sleepForRetries);
- continue;
- }
- if (!checkQuota()) {
- continue;
- }
- WALEntryBatch batch = new
WALEntryBatch(replicationBatchCountCapacity);
- 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 entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
- batch.addEntry(entry, entrySize);
- updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
- boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
- // Stop if too many entries or too big
- if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
+ WALEntryBatch batch = null;
+ WALEntryStream entryStream =
+ new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics);
+ try {
+ while (isReaderRunning()) { // we only loop back here if something fatal
happens to stream
+ try {
+ entryStream = new WALEntryStream(logQueue, fs, conf,
lastReadPosition, metrics);
+ while (isReaderRunning()) { // loop here to keep reusing stream
while we can
+ if (!source.isPeerEnabled()) {
+ Threads.sleep(sleepForRetries);
+ continue;
+ }
+ if (!checkQuota()) {
+ continue;
+ }
+ batch = new WALEntryBatch(replicationBatchCountCapacity);
+ boolean hasNext = entryStream.hasNext();
+ while (hasNext) {
+ Entry entry = entryStream.next();
+ entry = filterEntry(entry);
+ if (entry != null) {
+ WALEdit edit = entry.getEdit();
+ if (edit != null && !edit.isEmpty()) {
+ long entrySize = getEntrySizeIncludeBulkLoad(entry);
+ long entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
+ batch.addEntry(entry, entrySize);
+ updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
+ boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
+ // Stop if too many entries or too big
+ if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
|| batch.getNbEntries() >= replicationBatchCountCapacity) {
- break;
+ break;
+ }
}
}
+ hasNext = entryStream.hasNext();
}
- }
- updateBatch(entryStream, batch, hasNext);
- if (isShippable(batch)) {
- sleepMultiplier = 1;
- entryBatchQueue.put(batch);
- if (!batch.hasMoreEntries()) {
- // we're done with queue recovery, shut ourselves down
- setReaderRunning(false);
+ // If the batch has data to max capacity or stream doesn't have
anything
+ // try to ship it
+ if (isBatchQueuedToBeShipped(entryStream, batch, hasNext, false)) {
+ sleepMultiplier = 1;
}
+ }
+ } catch (IOException | WALEntryStreamRuntimeException e) { // stream
related
+ if (handleEofException(e, entryStream, batch)) {
+ sleepMultiplier = 1;
} else {
- Thread.sleep(sleepForRetries);
+ if (sleepMultiplier < maxRetriesMultiplier) {
+ LOG.debug("Failed to read stream of replication entries: " + e);
+ sleepMultiplier++;
+ } else {
+ LOG.error("Failed to read stream of replication entries", e);
+ }
+ Threads.sleep(sleepForRetries * sleepMultiplier);
}
- resetStream(entryStream);
+ } catch (InterruptedException e) {
+ LOG.trace("Interrupted while sleeping between WAL reads");
+ Thread.currentThread().interrupt();
+ } finally {
Review comment:
... and the catch clause for the inner try now just ensures
'entryStream' is closed ...
##########
File path:
hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java
##########
@@ -542,6 +556,173 @@ public WALEntryFilter getWALEntryfilter() {
});
}
+ @Test
Review comment:
Nice test
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
##########
@@ -840,7 +844,7 @@ public void uncaughtException(final Thread t, final
Throwable e) {
// If this is a recovered queue, the queue is already full and the first
log
// normally has a position (unless the RS failed between 2 logs)
- private long getRecoveredQueueStartPos(long startPosition) {
+ public long getRecoveredQueueStartPos(long startPosition) {
Review comment:
Is this change needed? I don't see any changes to a caller of this
method in this patch
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -227,10 +283,18 @@ private void resetStream(WALEntryStream stream) throws
IOException {
stream.reset(); // reuse stream
}
- // if we get an EOF due to a zero-length log, and there are other logs in
queue
- // (highly likely we've closed the current log), we've hit the max retries,
and autorecovery is
- // enabled, then dump the log
- private void handleEofException(Exception e) {
+ /**
+ * This is to handle the EOFException from the WAL entry stream.
EOFException should
Review comment:
Good comment
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -133,71 +134,126 @@ public
ReplicationSourceWALReaderThread(ReplicationSourceManager manager,
@Override
public void run() {
int sleepMultiplier = 1;
- while (isReaderRunning()) { // we only loop back here if something fatal
happened to our stream
- try (WALEntryStream entryStream =
- new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics)) {
- while (isReaderRunning()) { // loop here to keep reusing stream while
we can
- if (!source.isPeerEnabled()) {
- Threads.sleep(sleepForRetries);
- continue;
- }
- if (!checkQuota()) {
- continue;
- }
- WALEntryBatch batch = new
WALEntryBatch(replicationBatchCountCapacity);
- 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 entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
- batch.addEntry(entry, entrySize);
- updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
- boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
- // Stop if too many entries or too big
- if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
+ WALEntryBatch batch = null;
+ WALEntryStream entryStream =
+ new WALEntryStream(logQueue, fs, conf, lastReadPosition, metrics);
+ try {
+ while (isReaderRunning()) { // we only loop back here if something fatal
happens to stream
+ try {
+ entryStream = new WALEntryStream(logQueue, fs, conf,
lastReadPosition, metrics);
+ while (isReaderRunning()) { // loop here to keep reusing stream
while we can
+ if (!source.isPeerEnabled()) {
+ Threads.sleep(sleepForRetries);
+ continue;
+ }
+ if (!checkQuota()) {
+ continue;
+ }
+ batch = new WALEntryBatch(replicationBatchCountCapacity);
+ boolean hasNext = entryStream.hasNext();
+ while (hasNext) {
+ Entry entry = entryStream.next();
+ entry = filterEntry(entry);
+ if (entry != null) {
+ WALEdit edit = entry.getEdit();
+ if (edit != null && !edit.isEmpty()) {
+ long entrySize = getEntrySizeIncludeBulkLoad(entry);
+ long entrySizeExcludeBulkLoad =
getEntrySizeExcludeBulkLoad(entry);
+ batch.addEntry(entry, entrySize);
+ updateBatchStats(batch, entry, entryStream.getPosition(),
entrySize);
+ boolean totalBufferTooLarge =
acquireBufferQuota(entrySizeExcludeBulkLoad);
+ // Stop if too many entries or too big
+ if (totalBufferTooLarge || batch.getHeapSize() >=
replicationBatchSizeCapacity
|| batch.getNbEntries() >= replicationBatchCountCapacity) {
- break;
+ break;
+ }
}
}
+ hasNext = entryStream.hasNext();
}
- }
- updateBatch(entryStream, batch, hasNext);
- if (isShippable(batch)) {
- sleepMultiplier = 1;
- entryBatchQueue.put(batch);
- if (!batch.hasMoreEntries()) {
- // we're done with queue recovery, shut ourselves down
- setReaderRunning(false);
+ // If the batch has data to max capacity or stream doesn't have
anything
+ // try to ship it
+ if (isBatchQueuedToBeShipped(entryStream, batch, hasNext, false)) {
+ sleepMultiplier = 1;
}
+ }
+ } catch (IOException | WALEntryStreamRuntimeException e) { // stream
related
+ if (handleEofException(e, entryStream, batch)) {
+ sleepMultiplier = 1;
} else {
- Thread.sleep(sleepForRetries);
+ if (sleepMultiplier < maxRetriesMultiplier) {
+ LOG.debug("Failed to read stream of replication entries: " + e);
+ sleepMultiplier++;
+ } else {
+ LOG.error("Failed to read stream of replication entries", e);
+ }
+ Threads.sleep(sleepForRetries * sleepMultiplier);
}
- resetStream(entryStream);
+ } catch (InterruptedException e) {
+ LOG.trace("Interrupted while sleeping between WAL reads");
+ Thread.currentThread().interrupt();
+ } finally {
+ entryStream.close();
}
- } catch (IOException | WALEntryStreamRuntimeException e) { // stream
related
- if (sleepMultiplier < maxRetriesMultiplier) {
- LOG.debug("Failed to read stream of replication entries: " + e);
- sleepMultiplier++;
- } else {
- LOG.error("Failed to read stream of replication entries", e);
- handleEofException(e);
- }
- Threads.sleep(sleepForRetries * sleepMultiplier);
- } catch (InterruptedException e) {
- LOG.trace("Interrupted while sleeping between WAL reads");
- Thread.currentThread().interrupt();
}
+ } catch (IOException e) {
+ if (sleepMultiplier < maxRetriesMultiplier) {
+ LOG.debug("Failed to read stream of replication entries: " + e);
+ sleepMultiplier++;
+ } else {
+ LOG.error("Failed to read stream of replication entries", e);
+ }
+ Threads.sleep(sleepForRetries * sleepMultiplier);
+ } catch (InterruptedException e) {
+ LOG.trace("Interrupted while sleeping between WAL reads");
+ Thread.currentThread().interrupt();
}
}
- private void updateBatch(WALEntryStream entryStream, WALEntryBatch batch,
boolean moreData) {
+ /**
+ * Update the batch try to ship and return true if shipped
+ * @param entryStream stream of the WALs
+ * @param batch Batch of entries to ship
+ * @param hasMoreData if the stream has more yet more data to read
+ * @param isEOFException if we have hit the EOF exception before this. For
EOF exception,
+ * we do not want to reset the stream since entry
stream doesn't
+ * have correct information.
+ * @return if batch is shipped successfully
+ * @throws InterruptedException throws interrupted exception
+ * @throws IOException throws io exception from stream
+ */
+ private boolean isBatchQueuedToBeShipped(WALEntryStream entryStream,
WALEntryBatch batch,
Review comment:
This method does more than just check for a condition and return a
boolean, it also takes action. Giving it a name isFoo is misleading. I guess
you could call it updateAndShip or updateAndShipBatch or whatever. This is only
called from one place, inside handleEofException, just put this code in there
inline?
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
##########
@@ -405,7 +481,7 @@ public long getLastReadPosition() {
* Holds a batch of WAL entries to replicate, along with some statistics
*
*/
- static class WALEntryBatch {
+ final static class WALEntryBatch {
Review comment:
Not directly related but this is a good change
----------------------------------------------------------------
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]