HBASE-21221 Ineffective assertion in TestFromClientSide3#testMultiRowMutations
This commit is contained in:
parent
8eaaa63114
commit
b8134fe5b6
|
@ -35,6 +35,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -829,8 +830,8 @@ public class TestFromClientSide3 {
|
|||
}
|
||||
});
|
||||
ExecutorService cpService = Executors.newSingleThreadExecutor();
|
||||
AtomicBoolean exceptionDuringMutateRows = new AtomicBoolean();
|
||||
cpService.execute(() -> {
|
||||
boolean threw;
|
||||
Put put1 = new Put(row);
|
||||
Put put2 = new Put(rowLocked);
|
||||
put1.addColumn(FAMILY, QUALIFIER, value1);
|
||||
|
@ -844,26 +845,25 @@ public class TestFromClientSide3 {
|
|||
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.PUT, put2))
|
||||
.build();
|
||||
table.coprocessorService(MultiRowMutationProtos.MultiRowMutationService.class,
|
||||
ROW, ROW,
|
||||
ROW, ROW,
|
||||
(MultiRowMutationProtos.MultiRowMutationService exe) -> {
|
||||
ServerRpcController controller = new ServerRpcController();
|
||||
CoprocessorRpcUtils.BlockingRpcCallback<MultiRowMutationProtos.MutateRowsResponse>
|
||||
rpcCallback = new CoprocessorRpcUtils.BlockingRpcCallback<>();
|
||||
exe.mutateRows(controller, request, rpcCallback);
|
||||
if (controller.failedOnException()) {
|
||||
exceptionDuringMutateRows.set(true);
|
||||
}
|
||||
return rpcCallback.get();
|
||||
});
|
||||
threw = false;
|
||||
} catch (Throwable ex) {
|
||||
threw = true;
|
||||
}
|
||||
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");
|
||||
LOG.error("encountered " + ex);
|
||||
}
|
||||
});
|
||||
cpService.shutdown();
|
||||
cpService.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS);
|
||||
WaitingForMultiMutationsObserver observer = find(tableName, WaitingForMultiMutationsObserver.class);
|
||||
WaitingForMultiMutationsObserver observer = find(tableName,
|
||||
WaitingForMultiMutationsObserver.class);
|
||||
observer.latch.countDown();
|
||||
putService.shutdown();
|
||||
putService.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS);
|
||||
|
@ -877,6 +877,9 @@ public class TestFromClientSide3 {
|
|||
assertTrue(Bytes.equals(r1.getValue(FAMILY, QUALIFIER), value0));
|
||||
}
|
||||
assertNoLocks(tableName);
|
||||
if (!exceptionDuringMutateRows.get()) {
|
||||
fail("This cp should fail because the target lock is blocked by previous put");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue