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