HBASE-24794 hbase.rowlock.wait.duration should not be <= 0 (#2174)

if hbase.rowlock.wait.duration is <=0 then log a message and treat it as a value of 1ms.

amended for branches-1

Signed-off-by: Viraj Jasani <vjasani@apache.org>
(cherry picked from commit 840a55761b)
This commit is contained in:
Sean Busbey 2020-07-30 12:26:12 -05:00
parent ac576d23e5
commit 51161b5f48
2 changed files with 70 additions and 2 deletions

View File

@ -761,8 +761,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed "
+ MAX_FLUSH_PER_CHANGES);
}
this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
DEFAULT_ROWLOCK_WAIT_DURATION);
int tmpRowLockDuration = conf.getInt("hbase.rowlock.wait.duration",
DEFAULT_ROWLOCK_WAIT_DURATION);
if (tmpRowLockDuration <= 0) {
LOG.info("Found hbase.rowlock.wait.duration set to " + tmpRowLockDuration + ". values <= 0 " +
"will cause all row locking to fail. Treating it as 1ms to avoid region failure.");
tmpRowLockDuration = 1;
}
this.rowLockWaitDuration = tmpRowLockDuration;
this.maxWaitForSeqId = conf.getInt(MAX_WAIT_FOR_SEQ_ID_KEY, DEFAULT_MAX_WAIT_FOR_SEQ_ID);
this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);

View File

@ -6355,6 +6355,68 @@ public class TestHRegion {
CONF.setInt("hbase.rowlock.wait.duration", prevLockTimeout);
}
@Test
public void testBatchMutateWithZeroRowLockWait() throws Exception {
final byte[] a = Bytes.toBytes("a");
final byte[] b = Bytes.toBytes("b");
final byte[] c = Bytes.toBytes("c"); // exclusive
Configuration conf = new Configuration(CONF);
conf.setInt("hbase.rowlock.wait.duration", 0);
final HRegionInfo hri = new HRegionInfo(TableName.valueOf(tableName), a, c);
final HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor(fam1));
region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), conf, htd, TEST_UTIL.createWal(conf, TEST_UTIL.getDataTestDir(), TEST_UTIL.getDataTestDirOnTestFS(method + ".log"), hri));
Mutation[] mutations = new Mutation[] {
new Put(a).addImmutable(fam1, null, null),
new Put(b).addImmutable(fam1, null, null),
};
OperationStatus[] status = region.batchMutate(mutations);
assertEquals(OperationStatusCode.SUCCESS, status[0].getOperationStatusCode());
assertEquals(OperationStatusCode.SUCCESS, status[1].getOperationStatusCode());
// test with a row lock held for a long time
final CountDownLatch obtainedRowLock = new CountDownLatch(1);
ExecutorService exec = Executors.newFixedThreadPool(2);
Future<Void> f1 = exec.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
LOG.info("Acquiring row lock");
RowLock rl = region.getRowLock(b);
obtainedRowLock.countDown();
LOG.info("Waiting for 5 seconds before releasing lock");
Threads.sleep(5000);
LOG.info("Releasing row lock");
rl.release();
return null;
}
});
obtainedRowLock.await(30, TimeUnit.SECONDS);
Future<Void> f2 = exec.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
Mutation[] mutations = new Mutation[] {
new Put(a).addImmutable(fam1, null, null),
new Put(b).addImmutable(fam1, null, null),
};
// when handling row b we are going to spin on the failure to get the row lock
// until the lock above is released, but we will still succeed so long as that
// takes less time then the test time out.
OperationStatus[] status = region.batchMutate(mutations);
assertEquals(OperationStatusCode.SUCCESS, status[0].getOperationStatusCode());
assertEquals(OperationStatusCode.SUCCESS, status[1].getOperationStatusCode());
return null;
}
});
f1.get();
f2.get();
}
@Test
public void testCheckAndRowMutateTimestampsAreMonotonic() throws IOException {
region = initHRegion(tableName, name.getMethodName(), CONF, fam1);