1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.coprocessor;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.List;
23 import java.util.SortedSet;
24 import java.util.TreeSet;
25
26 import org.apache.hadoop.classification.InterfaceAudience;
27 import org.apache.hadoop.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.Coprocessor;
29 import org.apache.hadoop.hbase.CoprocessorEnvironment;
30 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
31 import org.apache.hadoop.hbase.HRegionInfo;
32 import org.apache.hadoop.hbase.client.Mutation;
33 import org.apache.hadoop.hbase.regionserver.HRegion;
34 import org.apache.hadoop.hbase.regionserver.WrongRegionException;
35 import org.apache.hadoop.hbase.util.Bytes;
36 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
37 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
38 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
39 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
40 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
41 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
42
43 import com.google.protobuf.RpcCallback;
44 import com.google.protobuf.RpcController;
45 import com.google.protobuf.Service;
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
78 @InterfaceStability.Evolving
79 public class MultiRowMutationEndpoint extends MultiRowMutationService implements
80 CoprocessorService, Coprocessor {
81 private RegionCoprocessorEnvironment env;
82 @Override
83 public void mutateRows(RpcController controller, MutateRowsRequest request,
84 RpcCallback<MutateRowsResponse> done) {
85 MutateRowsResponse response = MutateRowsResponse.getDefaultInstance();
86 try {
87
88 SortedSet<byte[]> rowsToLock = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
89 List<MutationProto> mutateRequestList = request.getMutationRequestList();
90 List<Mutation> mutations = new ArrayList<Mutation>(mutateRequestList.size());
91 for (MutationProto m : mutateRequestList) {
92 mutations.add(ProtobufUtil.toMutation(m));
93 }
94
95 HRegionInfo regionInfo = env.getRegion().getRegionInfo();
96 for (Mutation m : mutations) {
97
98 if (!HRegion.rowIsInRange(regionInfo, m.getRow())) {
99 String msg = "Requested row out of range '"
100 + Bytes.toStringBinary(m.getRow()) + "'";
101 if (rowsToLock.isEmpty()) {
102
103
104 throw new WrongRegionException(msg);
105 } else {
106
107 throw new org.apache.hadoop.hbase.DoNotRetryIOException(msg);
108 }
109 }
110 rowsToLock.add(m.getRow());
111 }
112
113 env.getRegion().mutateRowsWithLocks(mutations, rowsToLock);
114 } catch (IOException e) {
115 ResponseConverter.setControllerException(controller, e);
116 }
117 done.run(response);
118 }
119
120
121 @Override
122 public Service getService() {
123 return this;
124 }
125
126
127
128
129
130
131
132
133
134
135
136 @Override
137 public void start(CoprocessorEnvironment env) throws IOException {
138 if (env instanceof RegionCoprocessorEnvironment) {
139 this.env = (RegionCoprocessorEnvironment)env;
140 } else {
141 throw new CoprocessorException("Must be loaded on a table region!");
142 }
143 }
144
145 @Override
146 public void stop(CoprocessorEnvironment env) throws IOException {
147
148 }
149 }