View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  
21  package org.apache.hadoop.hbase.client;
22  
23  
24  import com.google.protobuf.ServiceException;
25  
26  import java.io.IOException;
27  import java.io.InterruptedIOException;
28  import java.util.Collections;
29  import java.util.List;
30  import java.util.concurrent.CancellationException;
31  import java.util.concurrent.ExecutionException;
32  import java.util.concurrent.ExecutorService;
33  import java.util.concurrent.Future;
34  import java.util.concurrent.TimeUnit;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.hbase.DoNotRetryIOException;
40  import org.apache.hadoop.hbase.HBaseIOException;
41  import org.apache.hadoop.hbase.HConstants;
42  import org.apache.hadoop.hbase.HRegionLocation;
43  import org.apache.hadoop.hbase.RegionLocations;
44  import org.apache.hadoop.hbase.ServerName;
45  import org.apache.hadoop.hbase.TableName;
46  import org.apache.hadoop.hbase.classification.InterfaceAudience;
47  import org.apache.hadoop.hbase.ipc.HBaseRpcController;
48  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
49  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
50  import org.apache.hadoop.hbase.protobuf.RequestConverter;
51  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
52  import org.apache.hadoop.hbase.util.Bytes;
53  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
54  
55  /**
56   * Caller that goes to replica if the primary region does no answer within a configurable
57   * timeout. If the timeout is reached, it calls all the secondary replicas, and returns
58   * the first answer. If the answer comes from one of the secondary replica, it will
59   * be marked as stale.
60   */
61  @InterfaceAudience.Private
62  public class RpcRetryingCallerWithReadReplicas {
63    private static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class);
64  
65    protected final ExecutorService pool;
66    protected final ClusterConnection cConnection;
67    protected final Configuration conf;
68    protected final Get get;
69    protected final TableName tableName;
70    protected final int timeBeforeReplicas;
71    private final int callTimeout;
72    private final int retries;
73    private final RpcControllerFactory rpcControllerFactory;
74    private final RpcRetryingCallerFactory rpcRetryingCallerFactory;
75  
76    public RpcRetryingCallerWithReadReplicas(
77        RpcControllerFactory rpcControllerFactory, TableName tableName,
78        ClusterConnection cConnection, final Get get,
79        ExecutorService pool, int retries, int callTimeout,
80        int timeBeforeReplicas) {
81      this.rpcControllerFactory = rpcControllerFactory;
82      this.tableName = tableName;
83      this.cConnection = cConnection;
84      this.conf = cConnection.getConfiguration();
85      this.get = get;
86      this.pool = pool;
87      this.retries = retries;
88      this.callTimeout = callTimeout;
89      this.timeBeforeReplicas = timeBeforeReplicas;
90      this.rpcRetryingCallerFactory = new RpcRetryingCallerFactory(conf);
91    }
92  
93    /**
94     * A RegionServerCallable that takes into account the replicas, i.e.
95     * - the call can be on any replica
96     * - we need to stop retrying when the call is completed
97     * - we can be interrupted
98     */
99    class ReplicaRegionServerCallable extends RegionServerCallable<Result> implements Cancellable {
100     final int id;
101     private final HBaseRpcController controller;
102 
103     public ReplicaRegionServerCallable(int id, HRegionLocation location) {
104       super(RpcRetryingCallerWithReadReplicas.this.cConnection,
105           RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow(), HConstants.PRIORITY_UNSET);
106       this.id = id;
107       this.location = location;
108       this.controller = rpcControllerFactory.newController();
109       controller.setPriority(tableName);
110     }
111 
112     @Override
113     public void cancel() {
114       controller.startCancel();
115     }
116 
117     /**
118      * Two responsibilities
119      * - if the call is already completed (by another replica) stops the retries.
120      * - set the location to the right region, depending on the replica.
121      */
122     @Override
123     public void prepare(final boolean reload) throws IOException {
124       if (controller.isCanceled()) return;
125 
126       if (Thread.interrupted()) {
127         throw new InterruptedIOException();
128       }
129 
130       if (reload || location == null) {
131         RegionLocations rl = getRegionLocations(false, id, cConnection, tableName, get.getRow());
132         location = id < rl.size() ? rl.getRegionLocation(id) : null;
133       }
134 
135       if (location == null || location.getServerName() == null) {
136         // With this exception, there will be a retry. The location can be null for a replica
137         //  when the table is created or after a split.
138         throw new HBaseIOException("There is no location for replica id #" + id);
139       }
140 
141       ServerName dest = location.getServerName();
142 
143       setStub(cConnection.getClient(dest));
144     }
145 
146     @Override
147     public Result call(int callTimeout) throws Exception {
148       if (controller.isCanceled()) return null;
149 
150       if (Thread.interrupted()) {
151         throw new InterruptedIOException();
152       }
153 
154       byte[] reg = location.getRegionInfo().getRegionName();
155 
156       ClientProtos.GetRequest request =
157           RequestConverter.buildGetRequest(reg, get);
158       controller.reset();
159       controller.setCallTimeout(callTimeout);
160 
161       try {
162         ClientProtos.GetResponse response = getStub().get(controller, request);
163         if (response == null) {
164           return null;
165         }
166         return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
167       } catch (ServiceException se) {
168         throw ProtobufUtil.getRemoteException(se);
169       }
170     }
171 
172     @Override
173     public boolean isCancelled() {
174       return controller.isCanceled();
175     }
176   }
177 
178   /**
179    * <p>
180    * Algo:
181    * - we put the query into the execution pool.
182    * - after x ms, if we don't have a result, we add the queries for the secondary replicas
183    * - we take the first answer
184    * - when done, we cancel what's left. Cancelling means:
185    * - removing from the pool if the actual call was not started
186    * - interrupting the call if it has started
187    * Client side, we need to take into account
188    * - a call is not executed immediately after being put into the pool
189    * - a call is a thread. Let's not multiply the number of thread by the number of replicas.
190    * Server side, if we can cancel when it's still in the handler pool, it's much better, as a call
191    * can take some i/o.
192    * </p>
193    * Globally, the number of retries, timeout and so on still applies, but it's per replica,
194    * not global. We continue until all retries are done, or all timeouts are exceeded.
195    */
196   public Result call(int operationTimeout)
197       throws DoNotRetryIOException, InterruptedIOException, RetriesExhaustedException {
198     boolean isTargetReplicaSpecified = (get.getReplicaId() >= 0);
199 
200     RegionLocations rl = null;
201     boolean skipPrimary = false;
202     try {
203       rl = getRegionLocations(true,
204         (isTargetReplicaSpecified ? get.getReplicaId() : RegionReplicaUtil.DEFAULT_REPLICA_ID),
205         cConnection, tableName, get.getRow());
206     } catch (RetriesExhaustedException | DoNotRetryIOException e) {
207       // When there is no specific replica id specified. It just needs to load all replicas.
208       if (isTargetReplicaSpecified) {
209         throw e;
210       } else {
211         // We cannot get the primary replica location, it is possible that the region
212         // server hosting meta is down, it needs to proceed to try cached replicas.
213         if (cConnection instanceof ConnectionManager.HConnectionImplementation) {
214           rl = ((ConnectionManager.HConnectionImplementation)cConnection).getCachedLocation(
215               tableName, get.getRow());
216           if (rl == null) {
217             // No cached locations
218             throw e;
219           }
220 
221           // Primary replica location is not known, skip primary replica
222           skipPrimary = true;
223         } else {
224           // For completeness
225           throw e;
226         }
227       }
228     }
229 
230     final ResultBoundedCompletionService<Result> cs =
231         new ResultBoundedCompletionService<>(this.rpcRetryingCallerFactory, pool, rl.size());
232 
233     int startIndex = 0;
234     int endIndex = rl.size();
235 
236     if(isTargetReplicaSpecified) {
237       addCallsForReplica(cs, rl, get.getReplicaId(), get.getReplicaId());
238       endIndex = 1;
239     } else {
240       if (!skipPrimary) {
241         addCallsForReplica(cs, rl, 0, 0);
242         try {
243           // wait for the timeout to see whether the primary responds back
244           Future<Result> f = cs.poll(timeBeforeReplicas, TimeUnit.MICROSECONDS); // Yes, microseconds
245           if (f != null) {
246             return f.get(); //great we got a response
247           }
248           if (cConnection.getConnectionMetrics() != null) {
249             cConnection.getConnectionMetrics().incrHedgedReadOps();
250           }
251         } catch (ExecutionException e) {
252           // We ignore the ExecutionException and continue with the secondary replicas
253           if (LOG.isDebugEnabled()) {
254             LOG.debug("Primary replica returns " + e.getCause());
255           }
256 
257           // Skip the result from the primary as we know that there is something wrong
258           startIndex = 1;
259         } catch (CancellationException e) {
260           throw new InterruptedIOException();
261         } catch (InterruptedException e) {
262           throw new InterruptedIOException();
263         }
264       } else {
265         // Since primary replica is skipped, the endIndex needs to be adjusted accordingly
266         endIndex --;
267       }
268 
269       // submit call for the all of the secondaries at once
270       addCallsForReplica(cs, rl, 1, rl.size() - 1);
271     }
272 
273     try {
274       ResultBoundedCompletionService<Result>.QueueingFuture<Result> f =
275           cs.pollForFirstSuccessfullyCompletedTask(operationTimeout, TimeUnit.MILLISECONDS, startIndex, endIndex);
276       if (f == null) {
277         throw new RetriesExhaustedException("Timed out after " + operationTimeout +
278             "ms. Get is sent to replicas with startIndex: " + startIndex +
279             ", endIndex: " + endIndex + ", Locations: " + rl);
280       }
281       if (cConnection.getConnectionMetrics() != null && !isTargetReplicaSpecified &&
282           !skipPrimary && f.getReplicaId() != RegionReplicaUtil.DEFAULT_REPLICA_ID) {
283         cConnection.getConnectionMetrics().incrHedgedReadWin();
284       }
285       return f.get();
286     } catch (ExecutionException e) {
287       throwEnrichedException(e, retries);
288     } catch (CancellationException e) {
289       throw new InterruptedIOException();
290     } catch (InterruptedException e) {
291       throw new InterruptedIOException();
292     } finally {
293       // We get there because we were interrupted or because one or more of the
294       // calls succeeded or failed. In all case, we stop all our tasks.
295       cs.cancelAll();
296     }
297 
298     LOG.error("Imposible? Arrive at an unreachable line..."); // unreachable
299     return null; // unreachable
300   }
301 
302   /**
303    * Extract the real exception from the ExecutionException, and throws what makes more
304    * sense.
305    */
306   static void throwEnrichedException(ExecutionException e, int retries)
307       throws RetriesExhaustedException, DoNotRetryIOException {
308     Throwable t = e.getCause();
309     assert t != null; // That's what ExecutionException is about: holding an exception
310 
311     if (t instanceof RetriesExhaustedException) {
312       throw (RetriesExhaustedException) t;
313     }
314 
315     if (t instanceof DoNotRetryIOException) {
316       throw (DoNotRetryIOException) t;
317     }
318 
319     if (t instanceof NeedUnmanagedConnectionException) {
320       throw new DoNotRetryIOException(t);
321     }
322 
323     RetriesExhaustedException.ThrowableWithExtraContext qt =
324         new RetriesExhaustedException.ThrowableWithExtraContext(t,
325             EnvironmentEdgeManager.currentTime(), null);
326 
327     List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions =
328         Collections.singletonList(qt);
329 
330     throw new RetriesExhaustedException(retries, exceptions);
331   }
332 
333   /**
334    * Creates the calls and submit them
335    *
336    * @param cs  - the completion service to use for submitting
337    * @param rl  - the region locations
338    * @param min - the id of the first replica, inclusive
339    * @param max - the id of the last replica, inclusive.
340    */
341   private void addCallsForReplica(ResultBoundedCompletionService<Result> cs,
342                                  RegionLocations rl, int min, int max) {
343     for (int id = min; id <= max; id++) {
344       HRegionLocation hrl = rl.getRegionLocation(id);
345       ReplicaRegionServerCallable callOnReplica = new ReplicaRegionServerCallable(id, hrl);
346       cs.submit(callOnReplica, callTimeout, id);
347     }
348   }
349 
350   static RegionLocations getRegionLocations(boolean useCache, int replicaId,
351                  ClusterConnection cConnection, TableName tableName, byte[] row)
352       throws RetriesExhaustedException, DoNotRetryIOException, InterruptedIOException {
353 
354     RegionLocations rl;
355     try {
356       if (useCache) {
357         rl = cConnection.locateRegion(tableName, row, true, true, replicaId);
358       } else {
359         rl = cConnection.relocateRegion(tableName, row, replicaId);
360       }
361     } catch (DoNotRetryIOException e) {
362       throw e;
363     } catch (NeedUnmanagedConnectionException e) {
364       throw new DoNotRetryIOException(e);
365     } catch (RetriesExhaustedException e) {
366       throw e;
367     } catch (InterruptedIOException e) {
368       throw e;
369     } catch (IOException e) {
370       throw new RetriesExhaustedException("Cannot get the location for replica" + replicaId
371           + " of region for " + Bytes.toStringBinary(row) + " in " + tableName, e);
372     }
373     if (rl == null) {
374       throw new RetriesExhaustedException("Cannot get the location for replica" + replicaId
375           + " of region for " + Bytes.toStringBinary(row) + " in " + tableName);
376     }
377 
378     return rl;
379   }
380 }