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