Skip to content

Commit

Permalink
HBASE-24665 all wal of RegionGroupingProvider together roll (#2021)
Browse files Browse the repository at this point in the history
Co-authored-by: wen_yi <liu.wenwen@immomo.com>
Signed-off-by: Anoop <anoopsamjohn@apache.org>
Signed-off-by: Ramkrishna <ramkrishna@apache.org>
Signed-off-by: Viraj Jasani <vjasani@apache.org>
  • Loading branch information
WenFeiYi and wen_yi committed Jul 24, 2020
1 parent 09e7ccd commit 975cdf7
Show file tree
Hide file tree
Showing 3 changed files with 180 additions and 87 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ protected void scheduleFlush(String encodedRegionName, List<byte[]> families) {
}

@VisibleForTesting
Map<WAL, Boolean> getWalNeedsRoll() {
return this.walNeedsRoll;
Map<WAL, RollController> getWalNeedsRoll() {
return this.wals;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,13 @@
import java.io.Closeable;
import java.io.IOException;
import java.net.ConnectException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
Expand Down Expand Up @@ -58,31 +58,31 @@ public abstract class AbstractWALRoller<T extends Abortable> extends Thread

protected static final String WAL_ROLL_PERIOD_KEY = "hbase.regionserver.logroll.period";

protected final ConcurrentMap<WAL, Boolean> walNeedsRoll = new ConcurrentHashMap<>();
protected final ConcurrentMap<WAL, RollController> wals = new ConcurrentHashMap<>();
protected final T abortable;
private volatile long lastRollTime = System.currentTimeMillis();
// Period to roll log.
private final long rollPeriod;
private final int threadWakeFrequency;
// The interval to check low replication on hlog's pipeline
private long checkLowReplicationInterval;
private final long checkLowReplicationInterval;

private volatile boolean running = true;

public void addWAL(WAL wal) {
// check without lock first
if (walNeedsRoll.containsKey(wal)) {
if (wals.containsKey(wal)) {
return;
}
// this is to avoid race between addWAL and requestRollAll.
synchronized (this) {
if (walNeedsRoll.putIfAbsent(wal, Boolean.FALSE) == null) {
if (wals.putIfAbsent(wal, new RollController(wal)) == null) {
wal.registerWALActionsListener(new WALActionsListener() {
@Override
public void logRollRequested(WALActionsListener.RollRequestReason reason) {
// TODO logs will contend with each other here, replace with e.g. DelayedQueue
synchronized (AbstractWALRoller.this) {
walNeedsRoll.put(wal, Boolean.TRUE);
RollController controller = wals.computeIfAbsent(wal, rc -> new RollController(wal));
controller.requestRoll();
AbstractWALRoller.this.notifyAll();
}
}
Expand All @@ -93,9 +93,8 @@ public void logRollRequested(WALActionsListener.RollRequestReason reason) {

public void requestRollAll() {
synchronized (this) {
List<WAL> wals = new ArrayList<WAL>(walNeedsRoll.keySet());
for (WAL wal : wals) {
walNeedsRoll.put(wal, Boolean.TRUE);
for (RollController controller : wals.values()) {
controller.requestRoll();
}
notifyAll();
}
Expand All @@ -115,9 +114,9 @@ protected AbstractWALRoller(String name, Configuration conf, T abortable) {
*/
private void checkLowReplication(long now) {
try {
for (Entry<WAL, Boolean> entry : walNeedsRoll.entrySet()) {
for (Entry<WAL, RollController> entry : wals.entrySet()) {
WAL wal = entry.getKey();
boolean needRollAlready = entry.getValue();
boolean needRollAlready = entry.getValue().needsRoll(now);
if (needRollAlready || !(wal instanceof AbstractFSWAL)) {
continue;
}
Expand All @@ -133,7 +132,7 @@ private void abort(String reason, Throwable cause) {
// This is because AsyncFSWAL replies on us for rolling a new writer to make progress, and if we
// failed, AsyncFSWAL may be stuck, so we need to close it to let the upper layer know that it
// is already broken.
for (WAL wal : walNeedsRoll.keySet()) {
for (WAL wal : wals.keySet()) {
// shutdown rather than close here since we are going to abort the RS and the wals need to be
// split when recovery
try {
Expand All @@ -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;
}
try {
// Force the roll if the logroll.period is elapsed or if a roll was requested.
// The returned value is an collection of actual region and family names.
regionsToFlush = wal.rollWriter(periodic || entry.getValue().booleanValue());
Map<byte[], List<byte[]>> regionsToFlush = controller.rollWal(now);
if (regionsToFlush != null) {
for (Map.Entry<byte[], List<byte[]>> r : regionsToFlush.entrySet()) {
scheduleFlush(Bytes.toString(r.getKey()), r.getValue());
}
}
} catch (WALClosedException e) {
LOG.warn("WAL has been closed. Skipping rolling of writer and just remove it", e);
iter.remove();
}
if (regionsToFlush != null) {
for (Map.Entry<byte[], List<byte[]>> r : regionsToFlush.entrySet()) {
scheduleFlush(Bytes.toString(r.getKey()), r.getValue());
}
}
afterRoll(wal);
}
} catch (FailedLogCloseException | ConnectException e) {
Expand Down Expand Up @@ -232,7 +227,9 @@ private boolean isWaiting() {
* @return true if all WAL roll finished
*/
public boolean walRollFinished() {
return walNeedsRoll.values().stream().allMatch(needRoll -> !needRoll) && isWaiting();
// TODO add a status field of roll in RollController
return wals.values().stream().noneMatch(rc -> rc.needsRoll(System.currentTimeMillis()))
&& isWaiting();
}

/**
Expand All @@ -249,4 +246,43 @@ public void close() {
running = false;
interrupt();
}

/**
* Independently control the roll of each wal. When use multiwal,
* can avoid all wal roll together. see HBASE-24665 for detail
*/
protected class RollController {
private final WAL wal;
private final AtomicBoolean rollRequest;
private long lastRollTime;

RollController(WAL wal) {
this.wal = wal;
this.rollRequest = new AtomicBoolean(false);
this.lastRollTime = System.currentTimeMillis();
}

public void requestRoll() {
this.rollRequest.set(true);
}

public Map<byte[], List<byte[]>> rollWal(long now) throws IOException {
this.lastRollTime = now;
// reset the flag in front to avoid missing roll request before we return from rollWriter.
this.rollRequest.set(false);
return wal.rollWriter(true);
}

public boolean isRollRequested() {
return rollRequest.get();
}

public boolean needsPeriodicRoll(long now) {
return (now - this.lastRollTime) > rollPeriod;
}

public boolean needsRoll(long now) {
return isRollRequested() || needsPeriodicRoll(now);
}
}
}

0 comments on commit 975cdf7

Please sign in to comment.