HBASE-26660 delayed FlushRegionEntry should be removed when we need a non-delayed one (#4042)
Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
a3e7d36f2e
commit
a2406f3764
|
@ -9227,6 +9227,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
flushesQueued.incrementAndGet();
|
flushesQueued.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void decrementFlushesQueuedCount() {
|
||||||
|
flushesQueued.decrementAndGet();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Do not change this sequence id.
|
* Do not change this sequence id.
|
||||||
* @return sequenceId
|
* @return sequenceId
|
||||||
|
|
|
@ -81,7 +81,7 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
// a corresponding entry in the other.
|
// a corresponding entry in the other.
|
||||||
private final BlockingQueue<FlushQueueEntry> flushQueue =
|
private final BlockingQueue<FlushQueueEntry> flushQueue =
|
||||||
new DelayQueue<FlushQueueEntry>();
|
new DelayQueue<FlushQueueEntry>();
|
||||||
private final Map<Region, FlushRegionEntry> regionsInQueue =
|
protected final Map<Region, FlushRegionEntry> regionsInQueue =
|
||||||
new HashMap<Region, FlushRegionEntry>();
|
new HashMap<Region, FlushRegionEntry>();
|
||||||
private AtomicBoolean wakeupPending = new AtomicBoolean();
|
private AtomicBoolean wakeupPending = new AtomicBoolean();
|
||||||
|
|
||||||
|
@ -363,16 +363,28 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
@Override
|
@Override
|
||||||
public boolean requestFlush(Region r, boolean forceFlushAllStores) {
|
public boolean requestFlush(Region r, boolean forceFlushAllStores) {
|
||||||
synchronized (regionsInQueue) {
|
synchronized (regionsInQueue) {
|
||||||
if (!regionsInQueue.containsKey(r)) {
|
FlushRegionEntry existFqe = regionsInQueue.get(r);
|
||||||
// This entry has no delay so it will be added at the top of the flush
|
if (existFqe != null) {
|
||||||
// queue. It'll come out near immediately.
|
// if a delayed one exists and not reach the time to execute, just remove it
|
||||||
FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
|
if (existFqe.isDelay() && existFqe.whenToExpire > EnvironmentEdgeManager.currentTime()) {
|
||||||
this.regionsInQueue.put(r, fqe);
|
LOG.info("Remove the existing delayed flush entry for " + r + ", "
|
||||||
this.flushQueue.add(fqe);
|
+ "because we need to flush it immediately");
|
||||||
((HRegion)r).incrementFlushesQueuedCount();
|
this.regionsInQueue.remove(r);
|
||||||
return true;
|
this.flushQueue.remove(existFqe);
|
||||||
|
((HRegion)r).decrementFlushesQueuedCount();
|
||||||
|
} else {
|
||||||
|
LOG.info("Flush already requested on " + r);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return false;
|
|
||||||
|
// This entry has no delay so it will be added at the top of the flush
|
||||||
|
// queue. It'll come out near immediately.
|
||||||
|
FlushRegionEntry fqe = new FlushRegionEntry(r, forceFlushAllStores);
|
||||||
|
this.regionsInQueue.put(r, fqe);
|
||||||
|
this.flushQueue.add(fqe);
|
||||||
|
((HRegion)r).incrementFlushesQueuedCount();
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -752,6 +764,13 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
return (EnvironmentEdgeManager.currentTime() - this.createTime) > maximumWait;
|
return (EnvironmentEdgeManager.currentTime() - this.createTime) > maximumWait;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return True if the entry is a delay flush task
|
||||||
|
*/
|
||||||
|
protected boolean isDelay() {
|
||||||
|
return this.whenToExpire > this.createTime;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Count of times {@link #requeue(long)} was called; i.e this is
|
* @return Count of times {@link #requeue(long)} was called; i.e this is
|
||||||
* number of times we've been requeued.
|
* number of times we've been requeued.
|
||||||
|
|
|
@ -0,0 +1,80 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.mockito.Mockito.doReturn;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category(SmallTests.class)
|
||||||
|
public class TestMemStoreFlusher {
|
||||||
|
private MemStoreFlusher msf;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.set("hbase.hstore.flusher.count", "0");
|
||||||
|
msf = new MemStoreFlusher(conf, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReplaceDelayedFlushEntry() {
|
||||||
|
HRegionInfo hri = new HRegionInfo(1, TableName.valueOf("TestTable"), 0);
|
||||||
|
HRegion r = mock(HRegion.class);
|
||||||
|
doReturn(hri).when(r).getRegionInfo();
|
||||||
|
|
||||||
|
// put a delayed task with 30s delay
|
||||||
|
msf.requestDelayedFlush(r, 30000, false);
|
||||||
|
assertEquals(1, msf.getFlushQueueSize());
|
||||||
|
assertTrue(msf.regionsInQueue.get(r).isDelay());
|
||||||
|
|
||||||
|
// put a non-delayed task, then the delayed one should be replaced
|
||||||
|
assertTrue(msf.requestFlush(r, false));
|
||||||
|
assertEquals(1, msf.getFlushQueueSize());
|
||||||
|
assertFalse(msf.regionsInQueue.get(r).isDelay());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNotReplaceDelayedFlushEntryWhichExpired() {
|
||||||
|
HRegionInfo hri = new HRegionInfo(1, TableName.valueOf("TestTable"), 0);
|
||||||
|
HRegion r = mock(HRegion.class);
|
||||||
|
doReturn(hri).when(r).getRegionInfo();
|
||||||
|
|
||||||
|
// put a delayed task with 100ms delay
|
||||||
|
msf.requestDelayedFlush(r, 100, false);
|
||||||
|
assertEquals(1, msf.getFlushQueueSize());
|
||||||
|
assertTrue(msf.regionsInQueue.get(r).isDelay());
|
||||||
|
|
||||||
|
Threads.sleep(200);
|
||||||
|
|
||||||
|
// put a non-delayed task, and the delayed one is expired, so it should not be replaced
|
||||||
|
assertFalse(msf.requestFlush(r, false));
|
||||||
|
assertEquals(1, msf.getFlushQueueSize());
|
||||||
|
assertTrue(msf.regionsInQueue.get(r).isDelay());
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue