@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public class MultiRowMutationEndpoint extends org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService implements RegionCoprocessor
HRegion.mutateRowsWithLocks(Collection, Collection, long, long)
and Coprocessor
endpoints. We can also specify some conditions to perform conditional update. Defines a protocol
to perform multi row transactions. See MultiRowMutationEndpoint
for the implementation.
HRegion.mutateRowsWithLocks(Collection, Collection, long, long)
for details and
limitations.
Put p = new Put(row1);
Delete d = new Delete(row2);
Increment i = new Increment(row3);
Append a = new Append(row4);
...
Mutate m1 = ProtobufUtil.toMutate(MutateType.PUT, p);
Mutate m2 = ProtobufUtil.toMutate(MutateType.DELETE, d);
Mutate m3 = ProtobufUtil.toMutate(MutateType.INCREMENT, i);
Mutate m4 = ProtobufUtil.toMutate(MutateType.Append, a);
MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
mrmBuilder.addMutationRequest(m1);
mrmBuilder.addMutationRequest(m2);
mrmBuilder.addMutationRequest(m3);
mrmBuilder.addMutationRequest(m4);
// We can also specify conditions to preform conditional update
mrmBuilder.addCondition(ProtobufUtil.toCondition(row, FAMILY, QUALIFIER,
CompareOperator.EQUAL, value, TimeRange.allTime()));
CoprocessorRpcChannel channel = t.coprocessorService(ROW);
MultiRowMutationService.BlockingInterface service =
MultiRowMutationService.newBlockingStub(channel);
MutateRowsRequest mrm = mrmBuilder.build();
MutateRowsResponse response = service.mutateRows(null, mrm);
// We can get the result of the conditional update
boolean processed = response.getProcessed();
org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService.BlockingInterface, org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService.Interface, org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService.Stub
Coprocessor.State
Modifier and Type | Field and Description |
---|---|
private RegionCoprocessorEnvironment |
env |
private static org.slf4j.Logger |
LOGGER |
PRIORITY_HIGHEST, PRIORITY_LOWEST, PRIORITY_SYSTEM, PRIORITY_USER, VERSION
Constructor and Description |
---|
MultiRowMutationEndpoint() |
Modifier and Type | Method and Description |
---|---|
private void |
checkFamily(Region region,
byte[] family) |
Iterable<com.google.protobuf.Service> |
getServices()
Coprocessor endpoints providing protobuf services should override this method.
|
private boolean |
matches(CompareOperator op,
int compareResult) |
private boolean |
matches(Region region,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition condition) |
void |
mutateRows(com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse> done) |
void |
start(CoprocessorEnvironment env)
Stores a reference to the coprocessor environment provided by the
RegionCoprocessorHost from the region where this
coprocessor is loaded. |
void |
stop(CoprocessorEnvironment env)
Called by the
CoprocessorEnvironment during it's own shutdown to stop the coprocessor. |
callMethod, getDescriptor, getDescriptorForType, getRequestPrototype, getResponsePrototype, newBlockingStub, newReflectiveBlockingService, newReflectiveService, newStub
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
getBulkLoadObserver, getEndpointObserver, getRegionObserver
private static final org.slf4j.Logger LOGGER
private RegionCoprocessorEnvironment env
public MultiRowMutationEndpoint()
public void mutateRows(com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest request, com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse> done)
mutateRows
in class org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService
private boolean matches(Region region, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition condition) throws IOException
IOException
private void checkFamily(Region region, byte[] family) throws NoSuchColumnFamilyException
NoSuchColumnFamilyException
private boolean matches(CompareOperator op, int compareResult)
public Iterable<com.google.protobuf.Service> getServices()
Coprocessor
getServices
in interface Coprocessor
Service
s or empty collection. Implementations should never return
null.public void start(CoprocessorEnvironment env) throws IOException
RegionCoprocessorHost
from the region where this
coprocessor is loaded. Since this is a coprocessor endpoint, it always expects to be loaded on
a table region, so always expects this to be an instance of
RegionCoprocessorEnvironment
.start
in interface Coprocessor
env
- the environment provided by the coprocessor hostIOException
- if the provided environment is not an instance of
RegionCoprocessorEnvironment
public void stop(CoprocessorEnvironment env) throws IOException
Coprocessor
CoprocessorEnvironment
during it's own shutdown to stop the coprocessor.stop
in interface Coprocessor
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.