ramkrish86 commented on a change in pull request #2021:
URL: https://github.com/apache/hbase/pull/2021#discussion_r459384595
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractWALRoller.java
##########
@@ -148,53 +147,49 @@ private void abort(String reason, Throwable cause) {
@Override
public void run() {
while (running) {
- boolean periodic = false;
long now = System.currentTimeMillis();
checkLowReplication(now);
- periodic = (now - this.lastRollTime) > this.rollPeriod;
- if (periodic) {
- // Time for periodic roll, fall through
- LOG.debug("WAL roll period {} ms elapsed", this.rollPeriod);
- } else {
- synchronized (this) {
- if (walNeedsRoll.values().stream().anyMatch(Boolean::booleanValue)) {
- // WAL roll requested, fall through
- LOG.debug("WAL roll requested");
- } else {
- try {
- wait(this.threadWakeFrequency);
- } catch (InterruptedException e) {
- // restore the interrupt state
- Thread.currentThread().interrupt();
- }
- // goto the beginning to check whether again whether we should
fall through to roll
- // several WALs, and also check whether we should quit.
- continue;
+ synchronized (this) {
+ if (wals.values().stream().noneMatch(rc -> rc.needsRoll(now))) {
+ try {
+ wait(this.threadWakeFrequency);
+ } catch (InterruptedException e) {
+ // restore the interrupt state
+ Thread.currentThread().interrupt();
}
+ // goto the beginning to check whether again whether we should fall
through to roll
+ // several WALs, and also check whether we should quit.
+ continue;
}
}
try {
- this.lastRollTime = System.currentTimeMillis();
- for (Iterator<Entry<WAL, Boolean>> iter =
walNeedsRoll.entrySet().iterator(); iter
- .hasNext();) {
- Entry<WAL, Boolean> entry = iter.next();
+ for (Iterator<Entry<WAL, RollController>> iter =
wals.entrySet().iterator();
+ iter.hasNext();) {
+ Entry<WAL, RollController> entry = iter.next();
WAL wal = entry.getKey();
- // reset the flag in front to avoid missing roll request before we
return from rollWriter.
- walNeedsRoll.put(wal, Boolean.FALSE);
- Map<byte[], List<byte[]>> regionsToFlush = null;
+ RollController controller = entry.getValue();
+ if (controller.isRollRequested()) {
+ // WAL roll requested, fall through
+ LOG.debug("WAL {} roll requested", wal);
+ } else if (controller.needsPeriodicRoll(now)){
+ // Time for periodic roll, fall through
+ LOG.debug("WAL {} roll period {} ms elapsed", wal,
this.rollPeriod);
+ } else {
+ continue;
Review comment:
This is they key. We ensure we only roll if one of the condition is met
(if either size reach caused a log roll ) or the time elapsed. That is also
tracked per wal.
----------------------------------------------------------------
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]