Skip to content
Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ public abstract class AbstractWALRoller<T extends Abortable> extends Thread
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;

Expand Down Expand Up @@ -148,10 +148,9 @@ 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;
boolean periodic = (now - this.lastRollTime) > this.rollPeriod;
Copy link
Contributor

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.

Copy link
Contributor Author

@WenFeiYi WenFeiYi Jul 10, 2020

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

if (periodic) {
// Time for periodic roll, fall through
LOG.debug("WAL roll period {} ms elapsed", this.rollPeriod);
Copy link
Contributor

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

Expand All @@ -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()) {
Copy link
Contributor

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?

continue;
Copy link
Contributor

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.

}
WAL wal = entry.getKey();
// reset the flag in front to avoid missing roll request before we return from rollWriter.
walNeedsRoll.put(wal, Boolean.FALSE);
entry.setValue(Boolean.FALSE);
Map<byte[], List<byte[]>> regionsToFlush = null;
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());
regionsToFlush = wal.rollWriter(true);
} catch (WALClosedException e) {
LOG.warn("WAL has been closed. Skipping rolling of writer and just remove it", e);
iter.remove();
Expand Down Expand Up @@ -232,7 +234,7 @@ private boolean isWaiting() {
* @return true if all WAL roll finished
*/
public boolean walRollFinished() {
return walNeedsRoll.values().stream().allMatch(needRoll -> !needRoll) && isWaiting();
Copy link
Contributor

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.

return walNeedsRoll.values().stream().noneMatch(needRoll -> needRoll) && isWaiting();
}

/**
Expand Down