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:
larsh 2013-07-17 05:34:00 +00:00
parent 07337ac9c8
commit 00175ae834
11 changed files with 453 additions and 510 deletions

View File

@ -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);

View File

@ -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

View File

@ -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

View File

@ -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:

View File

@ -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) {

View File

@ -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);

View File

@ -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());

View File

@ -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();
}
}
} }
} }
} }

View File

@ -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]

View File

@ -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);