HBASE-26448 Make sure we do not flush a region too frequently (#3847)
Signed-off-by: Xiaolin Ha <haxiaolin@apache.org>
This commit is contained in:
parent
1b0d9ceaba
commit
a8d46db416
@ -0,0 +1,142 @@
|
|||||||
|
/**
|
||||||
|
* 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.regionreplication;
|
||||||
|
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
|
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
|
||||||
|
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A helper class for requesting flush on a given region.
|
||||||
|
* <p/>
|
||||||
|
* In general, we do not want to trigger flush too frequently for a region, so here we will add
|
||||||
|
* something like a rate control, i.e, the interval of the two flush request should not be too
|
||||||
|
* small.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class RegionReplicationFlushRequester {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The timer for executing delayed flush request task.
|
||||||
|
* <p/>
|
||||||
|
* It will be shared across all the instances {@link RegionReplicationFlushRequester}. Created on
|
||||||
|
* demand to save one extra thread as not every user uses region replication.
|
||||||
|
*/
|
||||||
|
private static volatile HashedWheelTimer TIMER;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The minimum interval between two flush requests
|
||||||
|
*/
|
||||||
|
public static final String MIN_INTERVAL_SECS =
|
||||||
|
"hbase.region.read-replica.sink.flush.min-interval.secs";
|
||||||
|
|
||||||
|
public static final int MIN_INTERVAL_SECS_DEFAULT = 30;
|
||||||
|
|
||||||
|
private final Runnable flushRequester;
|
||||||
|
|
||||||
|
private final long minIntervalSecs;
|
||||||
|
|
||||||
|
private long lastRequestNanos;
|
||||||
|
|
||||||
|
private long pendingFlushRequestSequenceId;
|
||||||
|
|
||||||
|
private long lastFlushedSequenceId;
|
||||||
|
|
||||||
|
private Timeout pendingFlushRequest;
|
||||||
|
|
||||||
|
RegionReplicationFlushRequester(Configuration conf, Runnable flushRequester) {
|
||||||
|
this.flushRequester = flushRequester;
|
||||||
|
this.minIntervalSecs = conf.getInt(MIN_INTERVAL_SECS, MIN_INTERVAL_SECS_DEFAULT);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static HashedWheelTimer getTimer() {
|
||||||
|
HashedWheelTimer timer = TIMER;
|
||||||
|
if (timer != null) {
|
||||||
|
return timer;
|
||||||
|
}
|
||||||
|
synchronized (RegionReplicationFlushRequester.class) {
|
||||||
|
timer = TIMER;
|
||||||
|
if (timer != null) {
|
||||||
|
return timer;
|
||||||
|
}
|
||||||
|
timer = new HashedWheelTimer(
|
||||||
|
new ThreadFactoryBuilder().setNameFormat("RegionReplicationFlushRequester-Timer-pool-%d")
|
||||||
|
.setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
|
||||||
|
500, TimeUnit.MILLISECONDS);
|
||||||
|
TIMER = timer;
|
||||||
|
}
|
||||||
|
return timer;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void request() {
|
||||||
|
flushRequester.run();
|
||||||
|
lastRequestNanos = System.nanoTime();
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized void flush(Timeout timeout) {
|
||||||
|
pendingFlushRequest = null;
|
||||||
|
if (pendingFlushRequestSequenceId >= lastFlushedSequenceId) {
|
||||||
|
request();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Request a flush for the given region.
|
||||||
|
* <p/>
|
||||||
|
* The sequence id of the edit which we fail to replicate. A flush must happen after this sequence
|
||||||
|
* id to recover the failure.
|
||||||
|
*/
|
||||||
|
synchronized void requestFlush(long sequenceId) {
|
||||||
|
// if there is already a flush task, just reuse it.
|
||||||
|
if (pendingFlushRequest != null) {
|
||||||
|
pendingFlushRequestSequenceId = Math.max(sequenceId, pendingFlushRequestSequenceId);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// check last flush time
|
||||||
|
long elapsedSecs = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - lastRequestNanos);
|
||||||
|
if (elapsedSecs >= minIntervalSecs) {
|
||||||
|
request();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// schedule a timer task
|
||||||
|
HashedWheelTimer timer = getTimer();
|
||||||
|
pendingFlushRequest =
|
||||||
|
timer.newTimeout(this::flush, minIntervalSecs - elapsedSecs, TimeUnit.SECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Record that we have already finished a flush with the given {@code sequenceId}.
|
||||||
|
* <p/>
|
||||||
|
* We can cancel the pending flush request if the failed sequence id is less than the given
|
||||||
|
* {@code sequenceId}.
|
||||||
|
*/
|
||||||
|
synchronized void recordFlush(long sequenceId) {
|
||||||
|
this.lastFlushedSequenceId = sequenceId;
|
||||||
|
// cancel the pending flush request if it is necessary, i.e, we have already finished a flush
|
||||||
|
// with higher sequence id.
|
||||||
|
if (sequenceId > pendingFlushRequestSequenceId && pendingFlushRequest != null) {
|
||||||
|
pendingFlushRequest.cancel();
|
||||||
|
pendingFlushRequest = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -118,7 +118,7 @@ public class RegionReplicationSink {
|
|||||||
|
|
||||||
private final RegionReplicationBufferManager manager;
|
private final RegionReplicationBufferManager manager;
|
||||||
|
|
||||||
private final Runnable flushRequester;
|
private final RegionReplicationFlushRequester flushRequester;
|
||||||
|
|
||||||
private final AsyncClusterConnection conn;
|
private final AsyncClusterConnection conn;
|
||||||
|
|
||||||
@ -136,7 +136,7 @@ public class RegionReplicationSink {
|
|||||||
|
|
||||||
private volatile long pendingSize;
|
private volatile long pendingSize;
|
||||||
|
|
||||||
private long lastFlushSequenceNumber;
|
private long lastFlushedSequenceId;
|
||||||
|
|
||||||
private boolean sending;
|
private boolean sending;
|
||||||
|
|
||||||
@ -154,7 +154,7 @@ public class RegionReplicationSink {
|
|||||||
this.primary = primary;
|
this.primary = primary;
|
||||||
this.tableDesc = td;
|
this.tableDesc = td;
|
||||||
this.manager = manager;
|
this.manager = manager;
|
||||||
this.flushRequester = flushRequester;
|
this.flushRequester = new RegionReplicationFlushRequester(conf, flushRequester);
|
||||||
this.conn = conn;
|
this.conn = conn;
|
||||||
this.retries = conf.getInt(RETRIES_NUMBER, RETRIES_NUMBER_DEFAULT);
|
this.retries = conf.getInt(RETRIES_NUMBER, RETRIES_NUMBER_DEFAULT);
|
||||||
this.rpcTimeoutNs =
|
this.rpcTimeoutNs =
|
||||||
@ -178,19 +178,19 @@ public class RegionReplicationSink {
|
|||||||
Integer replicaId = entry.getKey();
|
Integer replicaId = entry.getKey();
|
||||||
Throwable error = entry.getValue().getValue();
|
Throwable error = entry.getValue().getValue();
|
||||||
if (error != null) {
|
if (error != null) {
|
||||||
if (maxSequenceId > lastFlushSequenceNumber) {
|
if (maxSequenceId > lastFlushedSequenceId) {
|
||||||
LOG.warn(
|
LOG.warn(
|
||||||
"Failed to replicate to secondary replica {} for {}, since the max sequence" +
|
"Failed to replicate to secondary replica {} for {}, since the max sequence" +
|
||||||
" id of sunk entris is {}, which is greater than the last flush SN {}," +
|
" id of sunk entris is {}, which is greater than the last flush SN {}," +
|
||||||
" we will stop replicating for a while and trigger a flush",
|
" we will stop replicating for a while and trigger a flush",
|
||||||
replicaId, primary, maxSequenceId, lastFlushSequenceNumber, error);
|
replicaId, primary, maxSequenceId, lastFlushedSequenceId, error);
|
||||||
failed.add(replicaId);
|
failed.add(replicaId);
|
||||||
} else {
|
} else {
|
||||||
LOG.warn(
|
LOG.warn(
|
||||||
"Failed to replicate to secondary replica {} for {}, since the max sequence" +
|
"Failed to replicate to secondary replica {} for {}, since the max sequence" +
|
||||||
" id of sunk entris is {}, which is less than or equal to the last flush SN {}," +
|
" id of sunk entris is {}, which is less than or equal to the last flush SN {}," +
|
||||||
" we will not stop replicating",
|
" we will not stop replicating",
|
||||||
replicaId, primary, maxSequenceId, lastFlushSequenceNumber, error);
|
replicaId, primary, maxSequenceId, lastFlushedSequenceId, error);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -198,7 +198,7 @@ public class RegionReplicationSink {
|
|||||||
pendingSize -= toReleaseSize;
|
pendingSize -= toReleaseSize;
|
||||||
if (!failed.isEmpty()) {
|
if (!failed.isEmpty()) {
|
||||||
failedReplicas.addAll(failed);
|
failedReplicas.addAll(failed);
|
||||||
flushRequester.run();
|
flushRequester.requestFlush(maxSequenceId);
|
||||||
}
|
}
|
||||||
sending = false;
|
sending = false;
|
||||||
if (stopping) {
|
if (stopping) {
|
||||||
@ -296,6 +296,7 @@ public class RegionReplicationSink {
|
|||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (flushDesc != null && flushAllStores(flushDesc)) {
|
if (flushDesc != null && flushAllStores(flushDesc)) {
|
||||||
|
long flushedSequenceId = flushDesc.getFlushSequenceNumber();
|
||||||
int toClearCount = 0;
|
int toClearCount = 0;
|
||||||
long toClearSize = 0;
|
long toClearSize = 0;
|
||||||
for (;;) {
|
for (;;) {
|
||||||
@ -303,7 +304,7 @@ public class RegionReplicationSink {
|
|||||||
if (e == null) {
|
if (e == null) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
if (e.key.getSequenceId() < flushDesc.getFlushSequenceNumber()) {
|
if (e.key.getSequenceId() < flushedSequenceId) {
|
||||||
entries.poll();
|
entries.poll();
|
||||||
toClearCount++;
|
toClearCount++;
|
||||||
toClearSize += e.size;
|
toClearSize += e.size;
|
||||||
@ -311,13 +312,14 @@ public class RegionReplicationSink {
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
lastFlushSequenceNumber = flushDesc.getFlushSequenceNumber();
|
lastFlushedSequenceId = flushedSequenceId;
|
||||||
failedReplicas.clear();
|
failedReplicas.clear();
|
||||||
LOG.debug(
|
LOG.debug(
|
||||||
"Got a flush all request with sequence id {}, clear failed replicas {}" +
|
"Got a flush all request with sequence id {}, clear failed replicas {}" +
|
||||||
" and {} pending entries with size {}",
|
" and {} pending entries with size {}",
|
||||||
flushDesc.getFlushSequenceNumber(), failedReplicas, toClearCount,
|
flushedSequenceId, failedReplicas, toClearCount,
|
||||||
StringUtils.TraditionalBinaryPrefix.long2String(toClearSize, "", 1));
|
StringUtils.TraditionalBinaryPrefix.long2String(toClearSize, "", 1));
|
||||||
|
flushRequester.recordFlush(flushedSequenceId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -340,7 +342,7 @@ public class RegionReplicationSink {
|
|||||||
for (int replicaId = 1; replicaId < regionReplication; replicaId++) {
|
for (int replicaId = 1; replicaId < regionReplication; replicaId++) {
|
||||||
failedReplicas.add(replicaId);
|
failedReplicas.add(replicaId);
|
||||||
}
|
}
|
||||||
flushRequester.run();
|
flushRequester.requestFlush(entry.key.getSequenceId());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,84 @@
|
|||||||
|
/**
|
||||||
|
* 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.regionreplication;
|
||||||
|
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.times;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category({ RegionServerTests.class, MediumTests.class })
|
||||||
|
public class TestRegionReplicationFlushRequester {
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestRegionReplicationFlushRequester.class);
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
private Runnable requester;
|
||||||
|
|
||||||
|
private RegionReplicationFlushRequester flushRequester;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
conf = HBaseConfiguration.create();
|
||||||
|
conf.setInt(RegionReplicationFlushRequester.MIN_INTERVAL_SECS, 1);
|
||||||
|
requester = mock(Runnable.class);
|
||||||
|
flushRequester = new RegionReplicationFlushRequester(conf, requester);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRequest() throws InterruptedException {
|
||||||
|
// should call request directly
|
||||||
|
flushRequester.requestFlush(100L);
|
||||||
|
verify(requester, times(1)).run();
|
||||||
|
|
||||||
|
// should not call request directly, since the min interval is 1 second
|
||||||
|
flushRequester.requestFlush(200L);
|
||||||
|
verify(requester, times(1)).run();
|
||||||
|
Thread.sleep(2000);
|
||||||
|
verify(requester, times(2)).run();
|
||||||
|
|
||||||
|
// should call request directly because we have already elapsed more than 1 second
|
||||||
|
Thread.sleep(2000);
|
||||||
|
flushRequester.requestFlush(300L);
|
||||||
|
verify(requester, times(3)).run();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCancelFlushRequest() throws InterruptedException {
|
||||||
|
flushRequester.requestFlush(100L);
|
||||||
|
flushRequester.requestFlush(200L);
|
||||||
|
verify(requester, times(1)).run();
|
||||||
|
|
||||||
|
// the pending flush request should be canceled
|
||||||
|
flushRequester.recordFlush(300L);
|
||||||
|
Thread.sleep(2000);
|
||||||
|
verify(requester, times(1)).run();
|
||||||
|
}
|
||||||
|
}
|
Loading…
x
Reference in New Issue
Block a user