@@ -6454,6 +6454,90 @@ public Void call() throws Exception {
64546454 CONF .setInt ("hbase.rowlock.wait.duration" , prevLockTimeout );
64556455 }
64566456
6457+ @ Test
6458+ public void testBatchMutateWithZeroRowLockWait () throws Exception {
6459+ final byte [] a = Bytes .toBytes ("a" );
6460+ final byte [] b = Bytes .toBytes ("b" );
6461+ final byte [] c = Bytes .toBytes ("c" ); // exclusive
6462+
6463+ Configuration conf = new Configuration (CONF );
6464+ conf .setInt ("hbase.rowlock.wait.duration" , 0 );
6465+ final RegionInfo hri =
6466+ RegionInfoBuilder .newBuilder (tableName ).setStartKey (a ).setEndKey (c ).build ();
6467+ final TableDescriptor htd = TableDescriptorBuilder .newBuilder (tableName )
6468+ .setColumnFamily (ColumnFamilyDescriptorBuilder .of (fam1 )).build ();
6469+ region = HRegion .createHRegion (hri , TEST_UTIL .getDataTestDir (), conf , htd , TEST_UTIL .createWal (conf , TEST_UTIL .getDataTestDirOnTestFS (method + ".log" ), hri ));
6470+
6471+ Mutation [] mutations = new Mutation [] {
6472+ new Put (a )
6473+ .add (CellBuilderFactory .create (CellBuilderType .SHALLOW_COPY )
6474+ .setRow (a )
6475+ .setFamily (fam1 )
6476+ .setTimestamp (HConstants .LATEST_TIMESTAMP )
6477+ .setType (Cell .Type .Put )
6478+ .build ()),
6479+ new Put (b ).add (CellBuilderFactory .create (CellBuilderType .SHALLOW_COPY )
6480+ .setRow (b )
6481+ .setFamily (fam1 )
6482+ .setTimestamp (HConstants .LATEST_TIMESTAMP )
6483+ .setType (Cell .Type .Put )
6484+ .build ())
6485+ };
6486+
6487+ OperationStatus [] status = region .batchMutate (mutations );
6488+ assertEquals (OperationStatusCode .SUCCESS , status [0 ].getOperationStatusCode ());
6489+ assertEquals (OperationStatusCode .SUCCESS , status [1 ].getOperationStatusCode ());
6490+
6491+
6492+ // test with a row lock held for a long time
6493+ final CountDownLatch obtainedRowLock = new CountDownLatch (1 );
6494+ ExecutorService exec = Executors .newFixedThreadPool (2 );
6495+ Future <Void > f1 = exec .submit (new Callable <Void >() {
6496+ @ Override
6497+ public Void call () throws Exception {
6498+ LOG .info ("Acquiring row lock" );
6499+ RowLock rl = region .getRowLock (b );
6500+ obtainedRowLock .countDown ();
6501+ LOG .info ("Waiting for 5 seconds before releasing lock" );
6502+ Threads .sleep (5000 );
6503+ LOG .info ("Releasing row lock" );
6504+ rl .release ();
6505+ return null ;
6506+ }
6507+ });
6508+ obtainedRowLock .await (30 , TimeUnit .SECONDS );
6509+
6510+ Future <Void > f2 = exec .submit (new Callable <Void >() {
6511+ @ Override
6512+ public Void call () throws Exception {
6513+ Mutation [] mutations = new Mutation [] {
6514+ new Put (a ).add (CellBuilderFactory .create (CellBuilderType .SHALLOW_COPY )
6515+ .setRow (a )
6516+ .setFamily (fam1 )
6517+ .setTimestamp (HConstants .LATEST_TIMESTAMP )
6518+ .setType (Cell .Type .Put )
6519+ .build ()),
6520+ new Put (b ).add (CellBuilderFactory .create (CellBuilderType .SHALLOW_COPY )
6521+ .setRow (b )
6522+ .setFamily (fam1 )
6523+ .setTimestamp (HConstants .LATEST_TIMESTAMP )
6524+ .setType (Cell .Type .Put )
6525+ .build ()),
6526+ };
6527+ // when handling row b we are going to spin on the failure to get the row lock
6528+ // until the lock above is released, but we will still succeed so long as that
6529+ // takes less time then the test time out.
6530+ OperationStatus [] status = region .batchMutate (mutations );
6531+ assertEquals (OperationStatusCode .SUCCESS , status [0 ].getOperationStatusCode ());
6532+ assertEquals (OperationStatusCode .SUCCESS , status [1 ].getOperationStatusCode ());
6533+ return null ;
6534+ }
6535+ });
6536+
6537+ f1 .get ();
6538+ f2 .get ();
6539+ }
6540+
64576541 @ Test
64586542 public void testCheckAndRowMutateTimestampsAreMonotonic () throws IOException {
64596543 region = initHRegion (tableName , method , CONF , fam1 );
0 commit comments