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