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.client;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.List;
23  import java.util.Map;
24  
25  import org.apache.hadoop.hbase.CellScannable;
26  import org.apache.hadoop.hbase.DoNotRetryIOException;
27  import org.apache.hadoop.hbase.HConstants;
28  import org.apache.hadoop.hbase.HRegionLocation;
29  import org.apache.hadoop.hbase.TableName;
30  import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
31  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
32  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
33  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
34  import org.apache.hadoop.hbase.protobuf.RequestConverter;
35  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
36  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
37  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
38  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
39  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
40  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
41  
42  import com.google.protobuf.ServiceException;
43  
44  /**
45   * Callable that handles the <code>multi</code> method call going against a single
46   * regionserver; i.e. A {@link RegionServerCallable} for the multi call (It is not a
47   * {@link RegionServerCallable} that goes against multiple regions.
48   * @param <R>
49   */
50  class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
51    private final MultiAction<R> multiAction;
52    private final boolean cellBlock;
53    private RpcControllerFactory rpcFactory;
54  
55    MultiServerCallable(final HConnection connection, final TableName tableName,
56        final HRegionLocation location, final RpcControllerFactory rpcFactory,
57        final MultiAction<R> multi) {
58      super(connection, tableName, null);
59      this.multiAction = multi;
60      this.rpcFactory = rpcFactory;
61      setLocation(location);
62      this.cellBlock = isCellBlock();
63    }
64  
65    MultiAction<R> getMulti() {
66      return this.multiAction;
67    }
68  
69    @Override
70    public MultiResponse call() throws IOException {
71      int countOfActions = this.multiAction.size();
72      if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions");
73      MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();
74      RegionAction.Builder regionActionBuilder = RegionAction.newBuilder();
75      ClientProtos.Action.Builder actionBuilder = ClientProtos.Action.newBuilder();
76      MutationProto.Builder mutationBuilder = MutationProto.newBuilder();
77      List<CellScannable> cells = null;
78      // The multi object is a list of Actions by region.  Iterate by region.
79      long nonceGroup = multiAction.getNonceGroup();
80      if (nonceGroup != HConstants.NO_NONCE) {
81        multiRequestBuilder.setNonceGroup(nonceGroup);
82      }
83      for (Map.Entry<byte[], List<Action<R>>> e: this.multiAction.actions.entrySet()) {
84        final byte [] regionName = e.getKey();
85        final List<Action<R>> actions = e.getValue();
86        regionActionBuilder.clear();
87        regionActionBuilder.setRegion(RequestConverter.buildRegionSpecifier(
88          HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName) );
89  
90  
91        if (this.cellBlock) {
92          // Presize.  Presume at least a KV per Action.  There are likely more.
93          if (cells == null) cells = new ArrayList<CellScannable>(countOfActions);
94          // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
95          // They have already been handled above. Guess at count of cells
96          regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells,
97            regionActionBuilder, actionBuilder, mutationBuilder);
98        } else {
99          regionActionBuilder = RequestConverter.buildRegionAction(regionName, actions,
100           regionActionBuilder, actionBuilder, mutationBuilder);
101       }
102       multiRequestBuilder.addRegionAction(regionActionBuilder.build());
103     }
104 
105     // Controller optionally carries cell data over the proxy/service boundary and also
106     // optionally ferries cell response data back out again.
107     PayloadCarryingRpcController controller = rpcFactory.newController(cells);
108     controller.setPriority(getTableName());
109     ClientProtos.MultiResponse responseProto;
110     ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
111     try {
112       responseProto = getStub().multi(controller, requestProto);
113     } catch (ServiceException e) {
114       throw ProtobufUtil.getRemoteException(e);
115     }
116     return ResponseConverter.getResults(requestProto, responseProto, controller.cellScanner());
117   }
118 
119 
120 
121   /**
122    * @return True if we should send data in cellblocks.  This is an expensive call.  Cache the
123    * result if you can rather than call each time.
124    */
125   private boolean isCellBlock() {
126     // This is not exact -- the configuration could have changed on us after connection was set up
127     // but it will do for now.
128     HConnection connection = getConnection();
129     // Default is to do cellblocks.
130     if (!(connection instanceof HConnectionImplementation)) return true;
131     return ((HConnectionImplementation) connection).hasCellBlockSupport();
132   }
133 
134   @Override
135   public void prepare(boolean reload) throws IOException {
136     // Use the location we were given in the constructor rather than go look it up.
137     setStub(getConnection().getClient(getLocation().getServerName()));
138   }
139 }