HBASE-8877 Reentrant row locks (Dave Latham)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1504002 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
07337ac9c8
commit
00175ae834
|
@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.OperationStatus;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
@ -146,20 +145,19 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
|
|||
}
|
||||
}
|
||||
if (deleteRows.size() > 0) {
|
||||
Pair<Mutation, Integer>[] deleteWithLockArr = new Pair[deleteRows.size()];
|
||||
Mutation[] deleteArr = new Mutation[deleteRows.size()];
|
||||
int i = 0;
|
||||
for (List<KeyValue> deleteRow : deleteRows) {
|
||||
Delete delete = createDeleteMutation(deleteRow, deleteType, timestamp);
|
||||
deleteWithLockArr[i++] = new Pair<Mutation, Integer>(delete, null);
|
||||
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
|
||||
}
|
||||
OperationStatus[] opStatus = region.batchMutate(deleteWithLockArr);
|
||||
OperationStatus[] opStatus = region.batchMutate(deleteArr);
|
||||
for (i = 0; i < opStatus.length; i++) {
|
||||
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
|
||||
break;
|
||||
}
|
||||
totalRowsDeleted++;
|
||||
if (deleteType == DeleteType.VERSION) {
|
||||
byte[] versionsDeleted = deleteWithLockArr[i].getFirst().getAttribute(
|
||||
byte[] versionsDeleted = deleteArr[i].getAttribute(
|
||||
NO_OF_VERSIONS_TO_DELETE);
|
||||
if (versionsDeleted != null) {
|
||||
totalVersionsDeleted += Bytes.toInt(versionsDeleted);
|
||||
|
|
|
@ -253,12 +253,12 @@ public abstract class BaseRegionObserver implements RegionObserver {
|
|||
|
||||
@Override
|
||||
public void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -554,7 +554,7 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @throws IOException if an error occurred on the coprocessor
|
||||
*/
|
||||
void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException;
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException;
|
||||
|
||||
/**
|
||||
* This will be called after applying a batch of Mutations on a region. The Mutations are added to
|
||||
|
@ -564,7 +564,7 @@ public interface RegionObserver extends Coprocessor {
|
|||
* @throws IOException if an error occurred on the coprocessor
|
||||
*/
|
||||
void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException;
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException;
|
||||
|
||||
/**
|
||||
* Called before checkAndPut
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -3956,8 +3956,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
*/
|
||||
protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region,
|
||||
final List<MutationProto> mutations, final CellScanner cells, boolean isReplay) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()];
|
||||
Mutation[] mArray = new Mutation[mutations.size()];
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||
try {
|
||||
|
@ -3974,7 +3973,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
mutation = ProtobufUtil.toDelete(m, cells);
|
||||
batchContainsDelete = true;
|
||||
}
|
||||
mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, null);
|
||||
mArray[i++] = mutation;
|
||||
builder.addResult(result);
|
||||
}
|
||||
|
||||
|
@ -3983,7 +3982,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
|
||||
OperationStatus codes[] = region.batchMutate(mutationsWithLocks, isReplay);
|
||||
OperationStatus codes[] = region.batchMutate(mArray);
|
||||
for (i = 0; i < codes.length; i++) {
|
||||
switch (codes[i].getOperationStatusCode()) {
|
||||
case BAD_FAMILY:
|
||||
|
|
|
@ -993,7 +993,7 @@ public class RegionCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public boolean preBatchMutate(
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||
boolean bypass = false;
|
||||
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
|
||||
for (RegionEnvironment env : coprocessors) {
|
||||
|
@ -1018,7 +1018,7 @@ public class RegionCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public void postBatchMutate(
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
|
||||
for (RegionEnvironment env : coprocessors) {
|
||||
if (env.getInstance() instanceof RegionObserver) {
|
||||
|
|
|
@ -1061,7 +1061,7 @@ public class HBaseFsck extends Configured implements Tool {
|
|||
"You may need to restore the previously sidelined .META.");
|
||||
return false;
|
||||
}
|
||||
meta.put(puts.toArray(new Put[0]));
|
||||
meta.batchMutate(puts.toArray(new Put[0]));
|
||||
HRegion.closeHRegion(meta);
|
||||
LOG.info("Success! .META. table rebuilt.");
|
||||
LOG.info("Old .META. is moved into " + backupDir);
|
||||
|
|
|
@ -407,7 +407,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
|
|||
|
||||
@Override
|
||||
public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
||||
MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||
RegionCoprocessorEnvironment e = c.getEnvironment();
|
||||
assertNotNull(e);
|
||||
assertNotNull(e.getRegion());
|
||||
|
@ -417,7 +417,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
|
|||
|
||||
@Override
|
||||
public void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
||||
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||
RegionCoprocessorEnvironment e = c.getEnvironment();
|
||||
assertNotNull(e);
|
||||
assertNotNull(e.getRegion());
|
||||
|
|
|
@ -59,11 +59,8 @@ import org.apache.hadoop.hbase.io.HeapSize;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
|
||||
/**
|
||||
* Testing of HRegion.incrementColumnValue, HRegion.increment,
|
||||
|
@ -528,16 +525,12 @@ public class TestAtomicOperation extends HBaseTestCase {
|
|||
final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(
|
||||
Bytes.toBytes(tableName), tableName, conf, Bytes.toBytes(family));
|
||||
|
||||
List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
|
||||
Put[] puts = new Put[1];
|
||||
Put put = new Put(Bytes.toBytes("r1"));
|
||||
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
|
||||
puts[0] = put;
|
||||
Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
|
||||
|
||||
putsAndLocks.add(pair);
|
||||
|
||||
region.batchMutate(putsAndLocks.toArray(new Pair[0]));
|
||||
|
||||
region.batchMutate(puts);
|
||||
MultithreadedTestUtil.TestContext ctx =
|
||||
new MultithreadedTestUtil.TestContext(conf);
|
||||
ctx.addThread(new PutThread(ctx, region));
|
||||
|
@ -565,15 +558,12 @@ public class TestAtomicOperation extends HBaseTestCase {
|
|||
}
|
||||
|
||||
public void doWork() throws Exception {
|
||||
List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
|
||||
Put[] puts = new Put[1];
|
||||
Put put = new Put(Bytes.toBytes("r1"));
|
||||
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
|
||||
puts[0] = put;
|
||||
Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
|
||||
putsAndLocks.add(pair);
|
||||
testStep = TestStep.PUT_STARTED;
|
||||
region.batchMutate(putsAndLocks.toArray(new Pair[0]));
|
||||
region.batchMutate(puts);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -607,43 +597,50 @@ public class TestAtomicOperation extends HBaseTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void releaseRowLock(Integer lockId) {
|
||||
if (testStep == TestStep.INIT) {
|
||||
super.releaseRowLock(lockId);
|
||||
return;
|
||||
}
|
||||
|
||||
if (testStep == TestStep.PUT_STARTED) {
|
||||
try {
|
||||
testStep = TestStep.PUT_COMPLETED;
|
||||
super.releaseRowLock(lockId);
|
||||
// put has been written to the memstore and the row lock has been released, but the
|
||||
// MVCC has not been advanced. Prior to fixing HBASE-7051, the following order of
|
||||
// operations would cause the non-atomicity to show up:
|
||||
// 1) Put releases row lock (where we are now)
|
||||
// 2) CheckAndPut grabs row lock and reads the value prior to the put (10)
|
||||
// because the MVCC has not advanced
|
||||
// 3) Put advances MVCC
|
||||
// So, in order to recreate this order, we wait for the checkAndPut to grab the rowLock
|
||||
// (see below), and then wait some more to give the checkAndPut time to read the old
|
||||
// value.
|
||||
latch.await();
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
else if (testStep == TestStep.CHECKANDPUT_STARTED) {
|
||||
super.releaseRowLock(lockId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getLock(Integer lockid, byte[] row, boolean waitForLock) throws IOException {
|
||||
public RowLock getRowLock(final byte[] row, boolean waitForLock) throws IOException {
|
||||
if (testStep == TestStep.CHECKANDPUT_STARTED) {
|
||||
latch.countDown();
|
||||
}
|
||||
return super.getLock(lockid, row, waitForLock);
|
||||
return new WrappedRowLock(super.getRowLock(row, waitForLock));
|
||||
}
|
||||
|
||||
public class WrappedRowLock extends RowLock {
|
||||
|
||||
private WrappedRowLock(RowLock rowLock) {
|
||||
super(rowLock.context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void release() {
|
||||
if (testStep == TestStep.INIT) {
|
||||
super.release();
|
||||
return;
|
||||
}
|
||||
|
||||
if (testStep == TestStep.PUT_STARTED) {
|
||||
try {
|
||||
testStep = TestStep.PUT_COMPLETED;
|
||||
super.release();
|
||||
// put has been written to the memstore and the row lock has been released, but the
|
||||
// MVCC has not been advanced. Prior to fixing HBASE-7051, the following order of
|
||||
// operations would cause the non-atomicity to show up:
|
||||
// 1) Put releases row lock (where we are now)
|
||||
// 2) CheckAndPut grabs row lock and reads the value prior to the put (10)
|
||||
// because the MVCC has not advanced
|
||||
// 3) Put advances MVCC
|
||||
// So, in order to recreate this order, we wait for the checkAndPut to grab the rowLock
|
||||
// (see below), and then wait some more to give the checkAndPut time to read the old
|
||||
// value.
|
||||
latch.await();
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
else if (testStep == TestStep.CHECKANDPUT_STARTED) {
|
||||
super.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -72,7 +72,6 @@ import org.apache.hadoop.hbase.client.Durability;
|
|||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -96,6 +95,7 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
|||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
|
@ -107,7 +107,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Assert;
|
||||
|
@ -764,7 +763,6 @@ public class TestHRegion extends HBaseTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testBatchPut() throws Exception {
|
||||
byte[] b = Bytes.toBytes(getName());
|
||||
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
|
||||
|
@ -783,7 +781,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
puts[i].add(cf, qual, val);
|
||||
}
|
||||
|
||||
OperationStatus[] codes = this.region.put(puts);
|
||||
OperationStatus[] codes = this.region.batchMutate(puts);
|
||||
assertEquals(10, codes.length);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
assertEquals(OperationStatusCode.SUCCESS, codes[i]
|
||||
|
@ -794,7 +792,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
|
||||
LOG.info("Next a batch put with one invalid family");
|
||||
puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
|
||||
codes = this.region.put(puts);
|
||||
codes = this.region.batchMutate(puts);
|
||||
assertEquals(10, codes.length);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
|
||||
|
@ -804,7 +802,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 2, source);
|
||||
|
||||
LOG.info("Next a batch put that has to break into two batches to avoid a lock");
|
||||
Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
|
||||
RowLock rowLock = region.getRowLock(Bytes.toBytes("row_2"));
|
||||
|
||||
MultithreadedTestUtil.TestContext ctx =
|
||||
new MultithreadedTestUtil.TestContext(conf);
|
||||
|
@ -813,7 +811,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
TestThread putter = new TestThread(ctx) {
|
||||
@Override
|
||||
public void doWork() throws IOException {
|
||||
retFromThread.set(region.put(puts));
|
||||
retFromThread.set(region.batchMutate(puts));
|
||||
}
|
||||
};
|
||||
LOG.info("...starting put thread while holding lock");
|
||||
|
@ -829,7 +827,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
}
|
||||
}
|
||||
LOG.info("...releasing row lock, which should let put thread continue");
|
||||
region.releaseRowLock(lockedRow);
|
||||
rowLock.release();
|
||||
LOG.info("...joining on thread");
|
||||
ctx.stop();
|
||||
LOG.info("...checking that next batch was synced");
|
||||
|
@ -840,29 +838,6 @@ public class TestHRegion extends HBaseTestCase {
|
|||
OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
|
||||
}
|
||||
|
||||
LOG.info("Nexta, a batch put which uses an already-held lock");
|
||||
lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
|
||||
LOG.info("...obtained row lock");
|
||||
List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[i], null);
|
||||
if (i == 2) pair.setSecond(lockedRow);
|
||||
putsAndLocks.add(pair);
|
||||
}
|
||||
|
||||
codes = region.batchMutate(putsAndLocks.toArray(new Pair[0]));
|
||||
LOG.info("...performed put");
|
||||
for (int i = 0; i < 10; i++) {
|
||||
assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
|
||||
OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
|
||||
}
|
||||
// Make sure we didn't do an extra batch
|
||||
metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 5, source);
|
||||
|
||||
// Make sure we still hold lock
|
||||
assertTrue(region.isRowLocked(lockedRow));
|
||||
LOG.info("...releasing lock");
|
||||
region.releaseRowLock(lockedRow);
|
||||
} finally {
|
||||
HRegion.closeHRegion(this.region);
|
||||
this.region = null;
|
||||
|
@ -891,7 +866,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||
puts[i].add(cf, qual, val);
|
||||
}
|
||||
|
||||
OperationStatus[] codes = this.region.put(puts);
|
||||
OperationStatus[] codes = this.region.batchMutate(puts);
|
||||
assertEquals(10, codes.length);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i]
|
||||
|
|
|
@ -233,7 +233,7 @@ public class TestParallelPut extends HBaseTestCase {
|
|||
put.add(fam1, qual1, value);
|
||||
in[0] = put;
|
||||
try {
|
||||
OperationStatus[] ret = region.put(in);
|
||||
OperationStatus[] ret = region.batchMutate(in);
|
||||
assertEquals(1, ret.length);
|
||||
assertEquals(OperationStatusCode.SUCCESS, ret[0].getOperationStatusCode());
|
||||
assertGet(rowkey, fam1, qual1, value);
|
||||
|
|
Loading…
Reference in New Issue