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  
19  package org.apache.hadoop.hbase.client;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.util.HashSet;
24  import java.util.Set;
25  import java.util.concurrent.Callable;
26  import java.util.concurrent.CancellationException;
27  import java.util.concurrent.ExecutionException;
28  import java.util.concurrent.ExecutorService;
29  import java.util.concurrent.Future;
30  import java.util.concurrent.TimeUnit;
31  import java.util.concurrent.atomic.AtomicBoolean;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.DoNotRetryIOException;
37  import org.apache.hadoop.hbase.HRegionInfo;
38  import org.apache.hadoop.hbase.RegionLocations;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.classification.InterfaceAudience;
41  import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
42  import org.apache.hadoop.hbase.util.Pair;
43  
44  /**
45   * This class has the logic for handling scanners for regions with and without replicas.
46   * 1. A scan is attempted on the default (primary) region
47   * 2. The scanner sends all the RPCs to the default region until it is done, or, there
48   * is a timeout on the default (a timeout of zero is disallowed).
49   * 3. If there is a timeout in (2) above, scanner(s) is opened on the non-default replica(s)
50   * 4. The results from the first successful scanner are taken, and it is stored which server
51   * returned the results.
52   * 5. The next RPCs are done on the above stored server until it is done or there is a timeout,
53   * in which case, the other replicas are queried (as in (3) above).
54   *
55   */
56  @InterfaceAudience.Private
57  class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
58    private static final Log LOG = LogFactory.getLog(ScannerCallableWithReplicas.class);
59    volatile ScannerCallable currentScannerCallable;
60    AtomicBoolean replicaSwitched = new AtomicBoolean(false);
61    final ClusterConnection cConnection;
62    protected final ExecutorService pool;
63    protected final int timeBeforeReplicas;
64    private final Scan scan;
65    private final int retries;
66    private Result lastResult;
67    private final RpcRetryingCaller<Result[]> caller;
68    private final TableName tableName;
69    private Configuration conf;
70    private int scannerTimeout;
71    private Set<ScannerCallable> outstandingCallables = new HashSet<ScannerCallable>();
72    private boolean someRPCcancelled = false; //required for testing purposes only
73    private int regionReplication = 0;
74  
75    public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConnection,
76        ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan,
77        int retries, int scannerTimeout, int caching, Configuration conf,
78        RpcRetryingCaller<Result []> caller) {
79      this.currentScannerCallable = baseCallable;
80      this.cConnection = cConnection;
81      this.pool = pool;
82      if (timeBeforeReplicas < 0) {
83        throw new IllegalArgumentException("Invalid value of operation timeout on the primary");
84      }
85      this.timeBeforeReplicas = timeBeforeReplicas;
86      this.scan = scan;
87      this.retries = retries;
88      this.tableName = tableName;
89      this.conf = conf;
90      this.scannerTimeout = scannerTimeout;
91      this.caller = caller;
92    }
93  
94    public void setClose() {
95      if(currentScannerCallable != null) {
96        currentScannerCallable.setClose();
97      } else {
98        LOG.warn("Calling close on ScannerCallable reference that is already null, "
99          + "which shouldn't happen.");
100     }
101   }
102 
103   public void setRenew(boolean val) {
104     currentScannerCallable.setRenew(val);
105   }
106 
107   public void setCaching(int caching) {
108     currentScannerCallable.setCaching(caching);
109   }
110 
111   public int getCaching() {
112     return currentScannerCallable.getCaching();
113   }
114 
115   public HRegionInfo getHRegionInfo() {
116     return currentScannerCallable.getHRegionInfo();
117   }
118 
119   public MoreResults moreResultsInRegion() {
120     return currentScannerCallable.moreResultsInRegion();
121   }
122 
123   public MoreResults moreResultsForScan() {
124     return currentScannerCallable.moreResultsForScan();
125   }
126 
127   @Override
128   public Result [] call(int timeout) throws IOException {
129     // If the active replica callable was closed somewhere, invoke the RPC to
130     // really close it. In the case of regular scanners, this applies. We make couple
131     // of RPCs to a RegionServer, and when that region is exhausted, we set
132     // the closed flag. Then an RPC is required to actually close the scanner.
133     if (currentScannerCallable != null && currentScannerCallable.closed) {
134       // For closing we target that exact scanner (and not do replica fallback like in
135       // the case of normal reads)
136       if (LOG.isTraceEnabled()) {
137         LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId);
138       }
139       Result[] r = currentScannerCallable.call(timeout);
140       currentScannerCallable = null;
141       return r;
142     } else if(currentScannerCallable == null) {
143       LOG.warn("Another call received, but our ScannerCallable is already null. "
144         + "This shouldn't happen, but there's not much to do, so logging and returning null.");
145       return null;
146     }
147     // We need to do the following:
148     //1. When a scan goes out to a certain replica (default or not), we need to
149     //   continue to hit that until there is a failure. So store the last successfully invoked
150     //   replica
151     //2. We should close the "losing" scanners (scanners other than the ones we hear back
152     //   from first)
153     //
154     // Since RegionReplication is a table attribute, it wont change as long as table is enabled,
155     // it just needs to be set once.
156 
157     if (regionReplication <= 0) {
158       RegionLocations rl = null;
159       try {
160         rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(true,
161             RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName,
162             currentScannerCallable.getRow());
163       } catch (RetriesExhaustedException | DoNotRetryIOException e) {
164         // We cannot get the primary replica region location, it is possible that the region server
165         // hosting meta table is down, it needs to proceed to try cached replicas directly.
166         if (cConnection instanceof ConnectionManager.HConnectionImplementation) {
167           rl = ((ConnectionManager.HConnectionImplementation) cConnection)
168               .getCachedLocation(tableName, currentScannerCallable.getRow());
169           if (rl == null) {
170             throw e;
171           }
172         } else {
173           // For completeness
174           throw e;
175         }
176       }
177       regionReplication = rl.size();
178     }
179     // allocate a boundedcompletion pool of some multiple of number of replicas.
180     // We want to accomodate some RPCs for redundant replica scans (but are still in progress)
181     ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs =
182         new ResultBoundedCompletionService<Pair<Result[], ScannerCallable>>(
183             RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf), pool,
184             regionReplication * 5);
185 
186     AtomicBoolean done = new AtomicBoolean(false);
187     replicaSwitched.set(false);
188     // submit call for the primary replica.
189     addCallsForCurrentReplica(cs);
190     int startIndex = 0;
191 
192     try {
193       // wait for the timeout to see whether the primary responds back
194       Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeBeforeReplicas,
195           TimeUnit.MICROSECONDS); // Yes, microseconds
196       if (f != null) {
197         // After poll, if f is not null, there must be a completed task
198         Pair<Result[], ScannerCallable> r = f.get();
199         if (r != null && r.getSecond() != null) {
200           updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
201         }
202         return r == null ? null : r.getFirst(); //great we got a response
203       }
204     } catch (ExecutionException e) {
205       // We ignore the ExecutionException and continue with the replicas
206       if (LOG.isDebugEnabled()) {
207         LOG.debug("Scan with primary region returns " + e.getCause());
208       }
209 
210       // If rl's size is 1 or scan's consitency is strong, it needs to throw
211       // out the exception from the primary replica
212       if ((regionReplication == 1) || (scan.getConsistency() == Consistency.STRONG)) {
213         // Rethrow the first exception
214         RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries);
215       }
216 
217       startIndex = 1;
218     } catch (CancellationException e) {
219       throw new InterruptedIOException(e.getMessage());
220     } catch (InterruptedException e) {
221       throw new InterruptedIOException(e.getMessage());
222     }
223 
224     // submit call for the all of the secondaries at once
225     int endIndex = regionReplication;
226     if (scan.getConsistency() == Consistency.STRONG) {
227       // When scan's consistency is strong, do not send to the secondaries
228       endIndex = 1;
229     } else {
230       // TODO: this may be an overkill for large region replication
231       addCallsForOtherReplicas(cs, 0, regionReplication - 1);
232     }
233 
234     try {
235       Future<Pair<Result[], ScannerCallable>> f = cs.pollForFirstSuccessfullyCompletedTask(timeout,
236           TimeUnit.MILLISECONDS, startIndex, endIndex);
237 
238       if (f == null) {
239         throw new IOException("Failed to get result within timeout, timeout=" + timeout + "ms");
240       }
241       Pair<Result[], ScannerCallable> r = f.get();
242 
243       if (r != null && r.getSecond() != null) {
244         updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
245       }
246       return r == null ? null : r.getFirst(); // great we got an answer
247 
248     } catch (ExecutionException e) {
249       RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries);
250     } catch (CancellationException e) {
251       throw new InterruptedIOException(e.getMessage());
252     } catch (InterruptedException e) {
253       throw new InterruptedIOException(e.getMessage());
254     } finally {
255       // We get there because we were interrupted or because one or more of the
256       // calls succeeded or failed. In all case, we stop all our tasks.
257       cs.cancelAll();
258     }
259     LOG.error("Imposible? Arrive at an unreachable line..."); // unreachable
260     throw new IOException("Imposible? Arrive at an unreachable line...");
261   }
262 
263   private void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result,
264       AtomicBoolean done, ExecutorService pool) {
265     if (done.compareAndSet(false, true)) {
266       if (currentScannerCallable != scanner) replicaSwitched.set(true);
267       currentScannerCallable = scanner;
268       // store where to start the replica scanner from if we need to.
269       if (result != null && result.length != 0) this.lastResult = result[result.length - 1];
270       if (LOG.isTraceEnabled()) {
271         LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId +
272             " associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId());
273       }
274       // close all outstanding replica scanners but the one we heard back from
275       outstandingCallables.remove(scanner);
276       for (ScannerCallable s : outstandingCallables) {
277         if (LOG.isTraceEnabled()) {
278           LOG.trace("Closing scanner id=" + s.scannerId +
279             ", replica=" + s.getHRegionInfo().getRegionId() +
280             " because slow and replica=" +
281             this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded");
282         }
283         // Submit the "close" to the pool since this might take time, and we don't
284         // want to wait for the "close" to happen yet. The "wait" will happen when
285         // the table is closed (when the awaitTermination of the underlying pool is called)
286         s.setClose();
287         final RetryingRPC r = new RetryingRPC(s);
288         pool.submit(new Callable<Void>(){
289           @Override
290           public Void call() throws Exception {
291             r.call(scannerTimeout);
292             return null;
293           }
294         });
295       }
296       // now clear outstandingCallables since we scheduled a close for all the contained scanners
297       outstandingCallables.clear();
298     }
299   }
300 
301   /**
302    * When a scanner switches in the middle of scanning (the 'next' call fails
303    * for example), the upper layer {@link ClientScanner} needs to know
304    * @return
305    */
306   public boolean switchedToADifferentReplica() {
307     return replicaSwitched.get();
308   }
309 
310   /**
311    * @return true when the most recent RPC response indicated that the response was a heartbeat
312    *         message. Heartbeat messages are sent back from the server when the processing of the
313    *         scan request exceeds a certain time threshold. Heartbeats allow the server to avoid
314    *         timeouts during long running scan operations.
315    */
316   public boolean isHeartbeatMessage() {
317     return currentScannerCallable != null && currentScannerCallable.isHeartbeatMessage();
318   }
319 
320   public Cursor getCursor() {
321     return currentScannerCallable != null ? currentScannerCallable.getCursor() : null;
322   }
323 
324   private void addCallsForCurrentReplica(
325       ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs) {
326     RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable);
327     outstandingCallables.add(currentScannerCallable);
328     cs.submit(retryingOnReplica, scannerTimeout, currentScannerCallable.id);
329   }
330 
331   private void addCallsForOtherReplicas(
332       ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs, int min, int max) {
333 
334     for (int id = min; id <= max; id++) {
335       if (currentScannerCallable.id == id) {
336         continue; //this was already scheduled earlier
337       }
338       ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
339       setStartRowForReplicaCallable(s);
340       outstandingCallables.add(s);
341       RetryingRPC retryingOnReplica = new RetryingRPC(s);
342       cs.submit(retryingOnReplica, scannerTimeout, id);
343     }
344   }
345 
346   /**
347    * Set the start row for the replica callable based on the state of the last result received.
348    * @param callable The callable to set the start row on
349    */
350   private void setStartRowForReplicaCallable(ScannerCallable callable) {
351     if (this.lastResult == null || callable == null) {
352       return;
353     }
354     // 1. The last result was a partial result which means we have not received all of the cells
355     // for this row. Thus, use the last result's row as the start row. If a replica switch
356     // occurs, the scanner will ensure that any accumulated partial results are cleared,
357     // and the scan can resume from this row.
358     // 2. The last result was not a partial result which means it contained all of the cells for
359     // that row (we no longer need any information from it). Set the start row to the next
360     // closest row that could be seen.
361     callable.getScan().withStartRow(this.lastResult.getRow(), this.lastResult.mayHaveMoreCellsInRow());
362   }
363 
364   boolean isAnyRPCcancelled() {
365     return someRPCcancelled;
366   }
367 
368   class RetryingRPC implements RetryingCallable<Pair<Result[], ScannerCallable>>, Cancellable {
369     final ScannerCallable callable;
370     RpcRetryingCaller<Result[]> caller;
371     private volatile boolean cancelled = false;
372 
373     RetryingRPC(ScannerCallable callable) {
374       this.callable = callable;
375       // For the Consistency.STRONG (default case), we reuse the caller
376       // to keep compatibility with what is done in the past
377       // For the Consistency.TIMELINE case, we can't reuse the caller
378       // since we could be making parallel RPCs (caller.callWithRetries is synchronized
379       // and we can't invoke it multiple times at the same time)
380       this.caller = ScannerCallableWithReplicas.this.caller;
381       if (scan.getConsistency() == Consistency.TIMELINE) {
382         this.caller = RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf)
383             .<Result[]>newCaller();
384       }
385     }
386 
387     @Override
388     public Pair<Result[], ScannerCallable> call(int callTimeout) throws IOException {
389       // since the retries is done within the ResultBoundedCompletionService,
390       // we don't invoke callWithRetries here
391       if (cancelled) {
392         return null;
393       }
394       Result[] res = this.caller.callWithoutRetries(this.callable, callTimeout);
395       return new Pair<Result[], ScannerCallable>(res, this.callable);
396     }
397 
398     @Override
399     public void prepare(boolean reload) throws IOException {
400       if (cancelled) return;
401 
402       if (Thread.interrupted()) {
403         throw new InterruptedIOException();
404       }
405 
406       callable.prepare(reload);
407     }
408 
409     @Override
410     public void throwable(Throwable t, boolean retrying) {
411       callable.throwable(t, retrying);
412     }
413 
414     @Override
415     public String getExceptionMessageAdditionalDetail() {
416       return callable.getExceptionMessageAdditionalDetail();
417     }
418 
419     @Override
420     public long sleep(long pause, int tries) {
421       return callable.sleep(pause, tries);
422     }
423 
424     @Override
425     public void cancel() {
426       cancelled = true;
427       caller.cancel();
428       if (callable.getController() != null) {
429         callable.getController().startCancel();
430       }
431       someRPCcancelled = true;
432     }
433 
434     @Override
435     public boolean isCancelled() {
436       return cancelled;
437     }
438   }
439 
440   @Override
441   public void prepare(boolean reload) throws IOException {
442   }
443 
444   @Override
445   public void throwable(Throwable t, boolean retrying) {
446     currentScannerCallable.throwable(t, retrying);
447   }
448 
449   @Override
450   public String getExceptionMessageAdditionalDetail() {
451     return currentScannerCallable.getExceptionMessageAdditionalDetail();
452   }
453 
454   @Override
455   public long sleep(long pause, int tries) {
456     return currentScannerCallable.sleep(pause, tries);
457   }
458 }