HBASE-21221 Ineffective assertion in TestFromClientSide3#testMultiRowMutations

This commit is contained in:
tedyu 2018-09-25 09:17:26 -07:00
parent 8eaaa63114
commit b8134fe5b6
1 changed files with 12 additions and 9 deletions

View File

@ -35,6 +35,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -829,8 +830,8 @@ public class TestFromClientSide3 {
} }
}); });
ExecutorService cpService = Executors.newSingleThreadExecutor(); ExecutorService cpService = Executors.newSingleThreadExecutor();
AtomicBoolean exceptionDuringMutateRows = new AtomicBoolean();
cpService.execute(() -> { cpService.execute(() -> {
boolean threw;
Put put1 = new Put(row); Put put1 = new Put(row);
Put put2 = new Put(rowLocked); Put put2 = new Put(rowLocked);
put1.addColumn(FAMILY, QUALIFIER, value1); put1.addColumn(FAMILY, QUALIFIER, value1);
@ -850,20 +851,19 @@ public class TestFromClientSide3 {
CoprocessorRpcUtils.BlockingRpcCallback<MultiRowMutationProtos.MutateRowsResponse> CoprocessorRpcUtils.BlockingRpcCallback<MultiRowMutationProtos.MutateRowsResponse>
rpcCallback = new CoprocessorRpcUtils.BlockingRpcCallback<>(); rpcCallback = new CoprocessorRpcUtils.BlockingRpcCallback<>();
exe.mutateRows(controller, request, rpcCallback); exe.mutateRows(controller, request, rpcCallback);
if (controller.failedOnException()) {
exceptionDuringMutateRows.set(true);
}
return rpcCallback.get(); return rpcCallback.get();
}); });
threw = false;
} catch (Throwable ex) { } catch (Throwable ex) {
threw = true; LOG.error("encountered " + ex);
}
if (!threw) {
// Can't call fail() earlier because the catch would eat it.
fail("This cp should fail because the target lock is blocked by previous put");
} }
}); });
cpService.shutdown(); cpService.shutdown();
cpService.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS); cpService.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS);
WaitingForMultiMutationsObserver observer = find(tableName, WaitingForMultiMutationsObserver.class); WaitingForMultiMutationsObserver observer = find(tableName,
WaitingForMultiMutationsObserver.class);
observer.latch.countDown(); observer.latch.countDown();
putService.shutdown(); putService.shutdown();
putService.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS); putService.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS);
@ -877,6 +877,9 @@ public class TestFromClientSide3 {
assertTrue(Bytes.equals(r1.getValue(FAMILY, QUALIFIER), value0)); assertTrue(Bytes.equals(r1.getValue(FAMILY, QUALIFIER), value0));
} }
assertNoLocks(tableName); assertNoLocks(tableName);
if (!exceptionDuringMutateRows.get()) {
fail("This cp should fail because the target lock is blocked by previous put");
}
} }
} }