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.HRegionInfo;
31 import org.apache.hadoop.hbase.client.Mutation;
32 import org.apache.hadoop.hbase.regionserver.HRegion;
33 import org.apache.hadoop.hbase.regionserver.WrongRegionException;
34 import org.apache.hadoop.hbase.util.Bytes;
35 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
37 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
38 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest;
39 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateResponse;
40 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService;
41
42 import com.google.protobuf.RpcCallback;
43 import com.google.protobuf.RpcController;
44 import com.google.protobuf.Service;
45
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 @InterfaceAudience.Public
77 @InterfaceStability.Evolving
78 public class MultiRowMutationEndpoint extends MultiRowMutationService implements
79 CoprocessorService, Coprocessor {
80 private RegionCoprocessorEnvironment env;
81 @Override
82 public void mutateRows(RpcController controller, MultiMutateRequest request,
83 RpcCallback<MultiMutateResponse> done) {
84 MultiMutateResponse response = MultiMutateResponse.getDefaultInstance();
85 try {
86
87 SortedSet<byte[]> rowsToLock = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
88 List<MutationProto> mutateRequestList = request.getMutationRequestList();
89 List<Mutation> mutations = new ArrayList<Mutation>(mutateRequestList.size());
90 for (MutationProto m : mutateRequestList) {
91 mutations.add(ProtobufUtil.toMutation(m));
92 }
93
94 HRegionInfo regionInfo = env.getRegion().getRegionInfo();
95 for (Mutation m : mutations) {
96
97 if (!HRegion.rowIsInRange(regionInfo, m.getRow())) {
98 String msg = "Requested row out of range '"
99 + Bytes.toStringBinary(m.getRow()) + "'";
100 if (rowsToLock.isEmpty()) {
101
102
103 throw new WrongRegionException(msg);
104 } else {
105
106 throw new org.apache.hadoop.hbase.DoNotRetryIOException(msg);
107 }
108 }
109 rowsToLock.add(m.getRow());
110 }
111
112 env.getRegion().mutateRowsWithLocks(mutations, rowsToLock);
113 } catch (IOException e) {
114 ResponseConverter.setControllerException(controller, e);
115 }
116 done.run(response);
117 }
118
119
120 @Override
121 public Service getService() {
122 return this;
123 }
124
125
126
127
128
129
130
131
132
133
134
135 @Override
136 public void start(CoprocessorEnvironment env) throws IOException {
137 if (env instanceof RegionCoprocessorEnvironment) {
138 this.env = (RegionCoprocessorEnvironment)env;
139 } else {
140 throw new CoprocessorException("Must be loaded on a table region!");
141 }
142 }
143
144 @Override
145 public void stop(CoprocessorEnvironment env) throws IOException {
146
147 }
148 }