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