-
Notifications
You must be signed in to change notification settings - Fork 3.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HBASE-24665 MultiWAL : Avoid rolling of ALL WALs when one of the WAL needs a roll #2021
Conversation
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
@@ -178,14 +177,17 @@ public void run() { | |||
for (Iterator<Entry<WAL, Boolean>> iter = walNeedsRoll.entrySet().iterator(); iter | |||
.hasNext();) { | |||
Entry<WAL, Boolean> entry = iter.next(); | |||
if (!periodic && !entry.getValue()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The actual fix is this alone right?
long now = System.currentTimeMillis(); | ||
checkLowReplication(now); | ||
periodic = (now - this.lastRollTime) > this.rollPeriod; | ||
boolean periodic = (now - this.lastRollTime) > this.rollPeriod; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not on this patch directly.
We have the periodic WAL roll.. Every WAL should get rolled as per this period. When one WAL gets rolled, we will change the lastRollTime and so it can happen that some WALs will ever get rolled!
This issue will be visible now as we will selectively roll WAL files after this patch. Actually we need to track the lastRollTime per WAL instance.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's like this, I ignored it. Thanks for reminding
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
this.lastRollTime = System.currentTimeMillis(); | ||
} | ||
|
||
public boolean isRequestRoll() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
isRollRequested() can be the better name?
@@ -66,6 +67,10 @@ protected void scheduleFlush(String encodedRegionName, List<byte[]> families) { | |||
|
|||
@VisibleForTesting | |||
Map<WAL, Boolean> getWalNeedsRoll() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is exposed for test cases only. This is a private class.. So its ok to change the return type. The new RollController gives clear idea whether a wal instance needs roll because of periodic roll or being explicitly asked for. So that is better.
A return type of Map<WAL, RollController>
Any way then we dont need synchronized block. Else, if we have to do as what is being done below in patch, we would need synchronized block
@@ -58,14 +57,13 @@ | |||
|
|||
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> walNeedsRoll = new ConcurrentHashMap<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We add all WAL instances into this once it is created. We can just call it wals?
return isRequestRoll; | ||
} | ||
|
||
boolean isPeriodRoll(long now) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A better name ? needsPeriodicRoll ?
this.isRequestRoll = true; | ||
} | ||
|
||
void finishRoll() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This name is bit confusing. This is not called once roll is finished. We can just call this resetStatus()? Give proper comment that this resets rollReq status as well as lastRollTime. We can pass the ts as param 'lastRollTime' so that this is clear.
iter.hasNext();) { | ||
Entry<WAL, RollController> entry = iter.next(); | ||
RollController controller = entry.getValue(); | ||
if (!controller.isRequestRoll && !controller.isPeriodRoll(now)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can have a single method in RollController which says this WAL needs roll? RollController#needsRoll().
long now = System.currentTimeMillis(); | ||
checkLowReplication(now); | ||
periodic = (now - this.lastRollTime) > this.rollPeriod; | ||
if (periodic) { | ||
if (walNeedsRoll.values().stream().anyMatch(rc -> rc.isPeriodRoll(now))) { | ||
// Time for periodic roll, fall through | ||
LOG.debug("WAL roll period {} ms elapsed", this.rollPeriod); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This log is not much value added now as we dont say for which log(s) need this periodic roll. We can clearly say WAL roll period {} elapsed for one of the WAL.
Below we can make sure we log which wal(s) are getting rolled for what purpose
// 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)) { | ||
if (walNeedsRoll.values().stream().anyMatch(rc -> rc.isRequestRoll)) { | ||
// WAL roll requested, fall through | ||
LOG.debug("WAL roll requested"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same as above comment
WAL wal = entry.getKey(); | ||
// reset the flag in front to avoid missing roll request before we return from rollWriter. | ||
walNeedsRoll.put(wal, Boolean.FALSE); | ||
controller.finishRoll(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ideally we need this call also within a synchronized block. This is an existing issue.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you very much for your tips
this.isRequestRoll = true; | ||
} | ||
|
||
public synchronized Map<byte[], List<byte[]>> rollWal(long lastRollTime) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is overall coming good now. Much more clean than what we had before. Thanks
One issue here though. We have rollWal() and requestRoll() under same lock now.. That is not good. The requestRoll should not wait for ongoing roll to be finished. The request call is in hot path.
WAL wal = entry.getKey(); | ||
boolean needRollAlready = entry.getValue(); | ||
boolean needRollAlready = entry.getValue().isRollRequested(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Previously, when we are going for a roll because of periodic roll, then also we used to keep the value in Map as TRUE right? Means here we checked that also. Because of any reason, the WAL is marked for roll, dont do this check. So here that is missing now.
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
} | ||
|
||
public void requestRoll() { | ||
this.rollRequestCounter.incrementAndGet(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Am not sure whether this way of count based thing also correct. Like say 2 times req roll happened before the actual rolling thread picks up this work. Effectively we need to do roll only once right. Even if 2 or more times being asked. But now as per this change it will try do roll again and again.
So a boolean based thing is enough
What we can do is make the requestRoll being synchronized and within rollWal have a synchronized block where you can reset it ad then go ahead with actual roll call.
Or we just need an AtomicBoolean only?
🎊 +1 overall
This message was automatically generated. |
public Map<byte[], List<byte[]>> rollWal(long now) throws IOException { | ||
this.lastRollTime = now; | ||
Map<byte[], List<byte[]>> regionsToFlush = wal.rollWriter(true); | ||
this.rollRequest.set(false); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hey this is where we need to reset it even before we do actual roll work.. We were doing that. We had a comment also.
// reset the flag in front to avoid missing roll request before we return from rollWriter.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thank you very much! I also noticed this, but in order to avoid repeated roll, so fix to that. if use boolean, roll at most once, the impact can be ignored. Thanks for review
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry did not get. How AtomicBoolean is different compared to boolean wrt repeated roll possibility. In the past we used to keep the Boolean state in a Map. Now in this object.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
after reset flag, before roll is complete, if have a roll request, that will also cause an extra roll.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes. That is happening now also. We intentionally reset the boolean when we start roll itself.
// reset the flag in front to avoid missing roll request before we return from rollWriter.
So there is no change for that right now.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I go it
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Last set of comments.
Overall looks good. +1
walNeedsRoll.put(wal, Boolean.FALSE); | ||
Map<byte[], List<byte[]>> regionsToFlush = null; | ||
RollController controller = entry.getValue(); | ||
boolean isRequestRoll; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Seems this boolean is not needed now. Can u pls delete
@@ -232,7 +230,7 @@ private boolean isWaiting() { | |||
* @return true if all WAL roll finished | |||
*/ | |||
public boolean walRollFinished() { | |||
return walNeedsRoll.values().stream().allMatch(needRoll -> !needRoll) && isWaiting(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Checking its usage, I think this API impl is already buggy. This just checks the status of the boolean. Once we start a roll on a WAL, we reset the boolean (Even before this patch). So it is not clearly telling anything abt the roll status. This can return true even while an active wal roll is going on. We can keep it as an another jira and fix (if required).. Just add some TODO comments here.
We might need another boolean in Controller which clearly tracks whether we are ongoing a roll. So this really need to check that status as well as a requested roll status.
@@ -232,7 +230,7 @@ private boolean isWaiting() { | |||
* @return true if all WAL roll finished | |||
*/ | |||
public boolean walRollFinished() { | |||
return walNeedsRoll.values().stream().allMatch(needRoll -> !needRoll) && isWaiting(); | |||
return wals.values().stream().noneMatch(RollController::isRollRequested) && isWaiting(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here rather than isRollRequested, we need needsRoll?
wal.init(); | ||
wals.put(wal, wal.getCurrentFileName()); | ||
ROLLER.addWAL(wal); | ||
Thread.sleep(3000); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We need a 3 sec sleep here? !
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We want 3 WALs will different periodic roll time period? Why we need? If so, can we limit the time to be 1 sec or lesser?
walEntry.setValue(walEntry.getKey().getCurrentFileName()); | ||
while (it.hasNext()) { | ||
walEntry = it.next(); | ||
assertEquals(walEntry.getValue(), walEntry.getKey().getCurrentFileName()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good...
🎊 +1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
@@ -58,31 +58,31 @@ | |||
|
|||
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<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
walRolls? instead of wals?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
rollWals? more suitable?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will have all wal entries once the wal instance is created. Not just when it needs a roll. So ya these were wals which needed roll at some point. So wals name also fine IMO.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok
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 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this talking about what this PR is trying to do?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, the todo is pre-existing.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ramkrish86 the purpose of this PR is to make each wal separate roll when using multiwal. thanks review.
// Time for periodic roll, fall through | ||
LOG.debug("WAL {} roll period {} ms elapsed", wal, this.rollPeriod); | ||
} else { | ||
continue; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1. Looks good to me.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Left one comment, overall seems good.
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)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have already done wals.putIfAbsent(wal, new RollController(wal))
above.
Hence, wals.computeIfAbsent()
is needed here? Should we not directly get the value with RollController controller = wals.get(wal)
and expect non-null object?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good Q. In fact I also thought when reviewed this. Ideally speaking we should get the addWAL call 1st which will add the instance to the Map. When we get call here the wal should be in the map already. But if u see the cur impl, there is no such contract enforcing. It just add the WAL with True value. So believe while making patch, @WenFeiYi went with similar lines.
We can consider this.. Need to see any chance we get a rollReq before adding.. While RS start, we do some rollReq on WALs.. This introduced some bug in the past. We need to see that closely.. If we can confirm that we can add that contract enforcing and so what u suggested. I would say add a TODO here and raise another issue. This went through multiple cycles of changes. :-) U ok Viraj?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, this is my idea, if there is a change, the impact will be greater, it will be far away from the purpose of the issue.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, this is critical code anyways, we can live with bit extra atomic calls on Concurrent Map. Sounds good, TODO
should make this better :)
Nice work @WenFeiYi .
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
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)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, this is critical code anyways, we can live with bit extra atomic calls on Concurrent Map. Sounds good, TODO
should make this better :)
Nice work @WenFeiYi .
No description provided.