View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * This class demonstrates how to implement atomic multi row transactions using
48   * {@link HRegion#mutateRowsWithLocks(java.util.Collection, java.util.Collection)}
49   * and Coprocessor endpoints.
50   *
51   * Defines a protocol to perform multi row transactions.
52   * See {@link MultiRowMutationEndpoint} for the implementation.
53   * </br>
54   * See
55   * {@link HRegion#mutateRowsWithLocks(java.util.Collection, java.util.Collection)}
56   * for details and limitations.
57   * </br>
58   * Example:
59   * <code><pre>
60   * List<Mutation> mutations = ...;
61   * Put p1 = new Put(row1);
62   * Put p2 = new Put(row2);
63   * ...
64   * Mutate m1 = ProtobufUtil.toMutate(MutateType.PUT, p1);
65   * Mutate m2 = ProtobufUtil.toMutate(MutateType.PUT, p2);
66   * MultiMutateRequest.Builder mrmBuilder = MultiMutateRequest.newBuilder();
67   * mrmBuilder.addMutationRequest(m1);
68   * mrmBuilder.addMutationRequest(m2);
69   * CoprocessorRpcChannel channel = t.coprocessorService(ROW);
70   * MultiRowMutationService.BlockingInterface service = 
71   *    MultiRowMutationService.newBlockingStub(channel);
72   * MultiMutateRequest mrm = mrmBuilder.build();
73   * service.mutateRows(null, mrm);
74   * </pre></code>
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        // set of rows to lock, sorted to avoid deadlocks
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          // check whether rows are in range for this region
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             // if this is the first row, region might have moved,
102             // allow client to retry
103             throw new WrongRegionException(msg);
104           } else {
105             // rows are split between regions, do not retry
106             throw new org.apache.hadoop.hbase.DoNotRetryIOException(msg);
107           }
108         }
109         rowsToLock.add(m.getRow());
110       }
111       // call utility method on region
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    * Stores a reference to the coprocessor environment provided by the
127    * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this
128    * coprocessor is loaded.  Since this is a coprocessor endpoint, it always expects to be loaded
129    * on a table region, so always expects this to be an instance of
130    * {@link RegionCoprocessorEnvironment}.
131    * @param env the environment provided by the coprocessor host
132    * @throws IOException if the provided environment is not an instance of
133    * {@code RegionCoprocessorEnvironment}
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     // nothing to do
147   }
148 }