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.OperationStatus;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
|
||||||
|
|
||||||
import com.google.protobuf.RpcCallback;
|
import com.google.protobuf.RpcCallback;
|
||||||
import com.google.protobuf.RpcController;
|
import com.google.protobuf.RpcController;
|
||||||
@ -146,20 +145,19 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (deleteRows.size() > 0) {
|
if (deleteRows.size() > 0) {
|
||||||
Pair<Mutation, Integer>[] deleteWithLockArr = new Pair[deleteRows.size()];
|
Mutation[] deleteArr = new Mutation[deleteRows.size()];
|
||||||
int i = 0;
|
int i = 0;
|
||||||
for (List<KeyValue> deleteRow : deleteRows) {
|
for (List<KeyValue> deleteRow : deleteRows) {
|
||||||
Delete delete = createDeleteMutation(deleteRow, deleteType, timestamp);
|
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
|
||||||
deleteWithLockArr[i++] = new Pair<Mutation, Integer>(delete, null);
|
|
||||||
}
|
}
|
||||||
OperationStatus[] opStatus = region.batchMutate(deleteWithLockArr);
|
OperationStatus[] opStatus = region.batchMutate(deleteArr);
|
||||||
for (i = 0; i < opStatus.length; i++) {
|
for (i = 0; i < opStatus.length; i++) {
|
||||||
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
|
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
totalRowsDeleted++;
|
totalRowsDeleted++;
|
||||||
if (deleteType == DeleteType.VERSION) {
|
if (deleteType == DeleteType.VERSION) {
|
||||||
byte[] versionsDeleted = deleteWithLockArr[i].getFirst().getAttribute(
|
byte[] versionsDeleted = deleteArr[i].getAttribute(
|
||||||
NO_OF_VERSIONS_TO_DELETE);
|
NO_OF_VERSIONS_TO_DELETE);
|
||||||
if (versionsDeleted != null) {
|
if (versionsDeleted != null) {
|
||||||
totalVersionsDeleted += Bytes.toInt(versionsDeleted);
|
totalVersionsDeleted += Bytes.toInt(versionsDeleted);
|
||||||
|
@ -253,12 +253,12 @@ public abstract class BaseRegionObserver implements RegionObserver {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
public void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
public void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -554,7 +554,7 @@ public interface RegionObserver extends Coprocessor {
|
|||||||
* @throws IOException if an error occurred on the coprocessor
|
* @throws IOException if an error occurred on the coprocessor
|
||||||
*/
|
*/
|
||||||
void preBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
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
|
* 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
|
* @throws IOException if an error occurred on the coprocessor
|
||||||
*/
|
*/
|
||||||
void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException;
|
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Called before checkAndPut
|
* 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,
|
protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region,
|
||||||
final List<MutationProto> mutations, final CellScanner cells, boolean isReplay) {
|
final List<MutationProto> mutations, final CellScanner cells, boolean isReplay) {
|
||||||
@SuppressWarnings("unchecked")
|
Mutation[] mArray = new Mutation[mutations.size()];
|
||||||
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()];
|
|
||||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||||
try {
|
try {
|
||||||
@ -3974,7 +3973,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||||||
mutation = ProtobufUtil.toDelete(m, cells);
|
mutation = ProtobufUtil.toDelete(m, cells);
|
||||||
batchContainsDelete = true;
|
batchContainsDelete = true;
|
||||||
}
|
}
|
||||||
mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, null);
|
mArray[i++] = mutation;
|
||||||
builder.addResult(result);
|
builder.addResult(result);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -3983,7 +3982,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||||||
cacheFlusher.reclaimMemStoreMemory();
|
cacheFlusher.reclaimMemStoreMemory();
|
||||||
}
|
}
|
||||||
|
|
||||||
OperationStatus codes[] = region.batchMutate(mutationsWithLocks, isReplay);
|
OperationStatus codes[] = region.batchMutate(mArray);
|
||||||
for (i = 0; i < codes.length; i++) {
|
for (i = 0; i < codes.length; i++) {
|
||||||
switch (codes[i].getOperationStatusCode()) {
|
switch (codes[i].getOperationStatusCode()) {
|
||||||
case BAD_FAMILY:
|
case BAD_FAMILY:
|
||||||
|
@ -993,7 +993,7 @@ public class RegionCoprocessorHost
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public boolean preBatchMutate(
|
public boolean preBatchMutate(
|
||||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||||
boolean bypass = false;
|
boolean bypass = false;
|
||||||
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
|
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
|
||||||
for (RegionEnvironment env : coprocessors) {
|
for (RegionEnvironment env : coprocessors) {
|
||||||
@ -1018,7 +1018,7 @@ public class RegionCoprocessorHost
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void postBatchMutate(
|
public void postBatchMutate(
|
||||||
final MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||||
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
|
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
|
||||||
for (RegionEnvironment env : coprocessors) {
|
for (RegionEnvironment env : coprocessors) {
|
||||||
if (env.getInstance() instanceof RegionObserver) {
|
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.");
|
"You may need to restore the previously sidelined .META.");
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
meta.put(puts.toArray(new Put[0]));
|
meta.batchMutate(puts.toArray(new Put[0]));
|
||||||
HRegion.closeHRegion(meta);
|
HRegion.closeHRegion(meta);
|
||||||
LOG.info("Success! .META. table rebuilt.");
|
LOG.info("Success! .META. table rebuilt.");
|
||||||
LOG.info("Old .META. is moved into " + backupDir);
|
LOG.info("Old .META. is moved into " + backupDir);
|
||||||
|
@ -407,7 +407,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
|
public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
|
||||||
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp) throws IOException {
|
MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
|
||||||
RegionCoprocessorEnvironment e = c.getEnvironment();
|
RegionCoprocessorEnvironment e = c.getEnvironment();
|
||||||
assertNotNull(e);
|
assertNotNull(e);
|
||||||
assertNotNull(e.getRegion());
|
assertNotNull(e.getRegion());
|
||||||
@ -417,7 +417,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void postBatchMutate(final ObserverContext<RegionCoprocessorEnvironment> c,
|
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();
|
RegionCoprocessorEnvironment e = c.getEnvironment();
|
||||||
assertNotNull(e);
|
assertNotNull(e);
|
||||||
assertNotNull(e.getRegion());
|
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.regionserver.wal.HLog;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Testing of HRegion.incrementColumnValue, HRegion.increment,
|
* Testing of HRegion.incrementColumnValue, HRegion.increment,
|
||||||
@ -528,16 +525,12 @@ public class TestAtomicOperation extends HBaseTestCase {
|
|||||||
final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(
|
final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(
|
||||||
Bytes.toBytes(tableName), tableName, conf, Bytes.toBytes(family));
|
Bytes.toBytes(tableName), tableName, conf, Bytes.toBytes(family));
|
||||||
|
|
||||||
List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
|
|
||||||
Put[] puts = new Put[1];
|
Put[] puts = new Put[1];
|
||||||
Put put = new Put(Bytes.toBytes("r1"));
|
Put put = new Put(Bytes.toBytes("r1"));
|
||||||
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
|
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
|
||||||
puts[0] = put;
|
puts[0] = put;
|
||||||
Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
|
|
||||||
|
region.batchMutate(puts);
|
||||||
putsAndLocks.add(pair);
|
|
||||||
|
|
||||||
region.batchMutate(putsAndLocks.toArray(new Pair[0]));
|
|
||||||
MultithreadedTestUtil.TestContext ctx =
|
MultithreadedTestUtil.TestContext ctx =
|
||||||
new MultithreadedTestUtil.TestContext(conf);
|
new MultithreadedTestUtil.TestContext(conf);
|
||||||
ctx.addThread(new PutThread(ctx, region));
|
ctx.addThread(new PutThread(ctx, region));
|
||||||
@ -565,15 +558,12 @@ public class TestAtomicOperation extends HBaseTestCase {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public void doWork() throws Exception {
|
public void doWork() throws Exception {
|
||||||
List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
|
|
||||||
Put[] puts = new Put[1];
|
Put[] puts = new Put[1];
|
||||||
Put put = new Put(Bytes.toBytes("r1"));
|
Put put = new Put(Bytes.toBytes("r1"));
|
||||||
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
|
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
|
||||||
puts[0] = put;
|
puts[0] = put;
|
||||||
Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
|
|
||||||
putsAndLocks.add(pair);
|
|
||||||
testStep = TestStep.PUT_STARTED;
|
testStep = TestStep.PUT_STARTED;
|
||||||
region.batchMutate(putsAndLocks.toArray(new Pair[0]));
|
region.batchMutate(puts);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -607,43 +597,50 @@ public class TestAtomicOperation extends HBaseTestCase {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void releaseRowLock(Integer lockId) {
|
public RowLock getRowLock(final byte[] row, boolean waitForLock) throws IOException {
|
||||||
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 {
|
|
||||||
if (testStep == TestStep.CHECKANDPUT_STARTED) {
|
if (testStep == TestStep.CHECKANDPUT_STARTED) {
|
||||||
latch.countDown();
|
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.Get;
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
import org.apache.hadoop.hbase.client.Increment;
|
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.Put;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
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.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
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.HLog;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
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.EnvironmentEdgeManagerTestHelper;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
|
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.PairOfSameType;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
@ -764,7 +763,6 @@ public class TestHRegion extends HBaseTestCase {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public void testBatchPut() throws Exception {
|
public void testBatchPut() throws Exception {
|
||||||
byte[] b = Bytes.toBytes(getName());
|
byte[] b = Bytes.toBytes(getName());
|
||||||
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
|
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
|
||||||
@ -783,7 +781,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||||||
puts[i].add(cf, qual, val);
|
puts[i].add(cf, qual, val);
|
||||||
}
|
}
|
||||||
|
|
||||||
OperationStatus[] codes = this.region.put(puts);
|
OperationStatus[] codes = this.region.batchMutate(puts);
|
||||||
assertEquals(10, codes.length);
|
assertEquals(10, codes.length);
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
assertEquals(OperationStatusCode.SUCCESS, codes[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");
|
LOG.info("Next a batch put with one invalid family");
|
||||||
puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
|
puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
|
||||||
codes = this.region.put(puts);
|
codes = this.region.batchMutate(puts);
|
||||||
assertEquals(10, codes.length);
|
assertEquals(10, codes.length);
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
|
assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
|
||||||
@ -804,7 +802,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||||||
metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 2, source);
|
metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 2, source);
|
||||||
|
|
||||||
LOG.info("Next a batch put that has to break into two batches to avoid a lock");
|
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 =
|
MultithreadedTestUtil.TestContext ctx =
|
||||||
new MultithreadedTestUtil.TestContext(conf);
|
new MultithreadedTestUtil.TestContext(conf);
|
||||||
@ -813,7 +811,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||||||
TestThread putter = new TestThread(ctx) {
|
TestThread putter = new TestThread(ctx) {
|
||||||
@Override
|
@Override
|
||||||
public void doWork() throws IOException {
|
public void doWork() throws IOException {
|
||||||
retFromThread.set(region.put(puts));
|
retFromThread.set(region.batchMutate(puts));
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
LOG.info("...starting put thread while holding lock");
|
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");
|
LOG.info("...releasing row lock, which should let put thread continue");
|
||||||
region.releaseRowLock(lockedRow);
|
rowLock.release();
|
||||||
LOG.info("...joining on thread");
|
LOG.info("...joining on thread");
|
||||||
ctx.stop();
|
ctx.stop();
|
||||||
LOG.info("...checking that next batch was synced");
|
LOG.info("...checking that next batch was synced");
|
||||||
@ -840,29 +838,6 @@ public class TestHRegion extends HBaseTestCase {
|
|||||||
OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
|
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 {
|
} finally {
|
||||||
HRegion.closeHRegion(this.region);
|
HRegion.closeHRegion(this.region);
|
||||||
this.region = null;
|
this.region = null;
|
||||||
@ -891,7 +866,7 @@ public class TestHRegion extends HBaseTestCase {
|
|||||||
puts[i].add(cf, qual, val);
|
puts[i].add(cf, qual, val);
|
||||||
}
|
}
|
||||||
|
|
||||||
OperationStatus[] codes = this.region.put(puts);
|
OperationStatus[] codes = this.region.batchMutate(puts);
|
||||||
assertEquals(10, codes.length);
|
assertEquals(10, codes.length);
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i]
|
assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i]
|
||||||
|
@ -233,7 +233,7 @@ public class TestParallelPut extends HBaseTestCase {
|
|||||||
put.add(fam1, qual1, value);
|
put.add(fam1, qual1, value);
|
||||||
in[0] = put;
|
in[0] = put;
|
||||||
try {
|
try {
|
||||||
OperationStatus[] ret = region.put(in);
|
OperationStatus[] ret = region.batchMutate(in);
|
||||||
assertEquals(1, ret.length);
|
assertEquals(1, ret.length);
|
||||||
assertEquals(OperationStatusCode.SUCCESS, ret[0].getOperationStatusCode());
|
assertEquals(OperationStatusCode.SUCCESS, ret[0].getOperationStatusCode());
|
||||||
assertGet(rowkey, fam1, qual1, value);
|
assertGet(rowkey, fam1, qual1, value);
|
||||||
|
Loading…
x
Reference in New Issue
Block a user