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