private void updateCurrentlyServingReplica( ScannerCallable scanner, Result[] result, AtomicBoolean done, ExecutorService pool) { if (done.compareAndSet(false, true)) { if (currentScannerCallable != scanner) replicaSwitched.set(true); currentScannerCallable = scanner; // store where to start the replica scanner from if we need to. if (result != null && result.length != 0) this.lastResult = result[result.length - 1]; if (LOG.isTraceEnabled()) { LOG.trace( "Setting current scanner as " + currentScannerCallable.scannerId + " associated with " + currentScannerCallable.getHRegionInfo().getReplicaId()); } // close all outstanding replica scanners but the one we heard back from outstandingCallables.remove(scanner); for (ScannerCallable s : outstandingCallables) { if (LOG.isDebugEnabled()) { LOG.debug( "Closing scanner " + s.scannerId + " because this was slow and another replica succeeded"); } // Submit the "close" to the pool since this might take time, and we don't // want to wait for the "close" to happen yet. The "wait" will happen when // the table is closed (when the awaitTermination of the underlying pool is called) s.setClose(); RetryingRPC r = new RetryingRPC(s); pool.submit(r); } // now clear outstandingCallables since we scheduled a close for all the contained scanners outstandingCallables.clear(); } }
@Override public void cancel() { cancelled = true; caller.cancel(); if (callable.getController() != null) { callable.getController().startCancel(); } someRPCcancelled = true; }
@Override public void prepare(boolean reload) throws IOException { if (cancelled) return; if (Thread.interrupted()) { throw new InterruptedIOException(); } callable.prepare(reload); }
/** * Set the start row for the replica callable based on the state of the last result received. * * @param callable The callable to set the start row on */ private void setStartRowForReplicaCallable(ScannerCallable callable) { if (this.lastResult == null || callable == null) return; if (this.lastResult.isPartial()) { // The last result was a partial result which means we have not received all of the cells // for this row. Thus, use the last result's row as the start row. If a replica switch // occurs, the scanner will ensure that any accumulated partial results are cleared, // and the scan can resume from this row. callable.getScan().setStartRow(this.lastResult.getRow()); } else { // The last result was not a partial result which means it contained all of the cells for // that row (we no longer need any information from it). Set the start row to the next // closest row that could be seen. if (callable.getScan().isReversed()) { callable.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow())); } else { callable.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1])); } } }
@InterfaceAudience.Private protected ScannerCallableWithReplicas getScannerCallable(byte[] localStartKey, int nbRows) { scan.setStartRow(localStartKey); ScannerCallable s = new ScannerCallable( getConnection(), getTable(), scan, this.scanMetrics, this.rpcControllerFactory); s.setCaching(nbRows); ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas( tableName, getConnection(), s, pool, primaryOperationTimeout, scan, retries, scannerTimeout, caching, conf, caller); return sr; }
private int addCallsForOtherReplicas( BoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl, int min, int max) { if (scan.getConsistency() == Consistency.STRONG) { return 0; // not scheduling on other replicas for strong consistency } for (int id = min; id <= max; id++) { if (currentScannerCallable.getHRegionInfo().getReplicaId() == id) { continue; // this was already scheduled earlier } ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id); if (this.lastResult != null) { s.getScan().setStartRow(this.lastResult.getRow()); } outstandingCallables.add(s); RetryingRPC retryingOnReplica = new RetryingRPC(s); cs.submit(retryingOnReplica); } return max - min + 1; }
private void addCallsForOtherReplicas( ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl, int min, int max) { if (scan.getConsistency() == Consistency.STRONG) { return; // not scheduling on other replicas for strong consistency } for (int id = min; id <= max; id++) { if (currentScannerCallable.id == id) { continue; // this was already scheduled earlier } ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id); setStartRowForReplicaCallable(s); outstandingCallables.add(s); RetryingRPC retryingOnReplica = new RetryingRPC(s); cs.submit(retryingOnReplica, scannerTimeout, id); } }
public void setClose() { currentScannerCallable.setClose(); }
@Override public long sleep(long pause, int tries) { return currentScannerCallable.sleep(pause, tries); }
@Override public String getExceptionMessageAdditionalDetail() { return currentScannerCallable.getExceptionMessageAdditionalDetail(); }
@Override public void throwable(Throwable t, boolean retrying) { currentScannerCallable.throwable(t, retrying); }
public boolean hasMoreResultsContext() { return currentScannerCallable.hasMoreResultsContext(); }
@Override public Result[] call(int timeout) throws IOException { // If the active replica callable was closed somewhere, invoke the RPC to // really close it. In the case of regular scanners, this applies. We make couple // of RPCs to a RegionServer, and when that region is exhausted, we set // the closed flag. Then an RPC is required to actually close the scanner. if (currentScannerCallable != null && currentScannerCallable.closed) { // For closing we target that exact scanner (and not do replica fallback like in // the case of normal reads) if (LOG.isDebugEnabled()) { LOG.debug("Closing scanner " + currentScannerCallable.scannerId); } Result[] r = currentScannerCallable.call(timeout); currentScannerCallable = null; return r; } // We need to do the following: // 1. When a scan goes out to a certain replica (default or not), we need to // continue to hit that until there is a failure. So store the last successfully invoked // replica // 2. We should close the "losing" scanners (scanners other than the ones we hear back // from first) // RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations( true, RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName, currentScannerCallable.getRow()); // allocate a boundedcompletion pool of some multiple of number of replicas. // We want to accomodate some RPCs for redundant replica scans (but are still in progress) BoundedCompletionService<Pair<Result[], ScannerCallable>> cs = new BoundedCompletionService<Pair<Result[], ScannerCallable>>(pool, rl.size() * 5); List<ExecutionException> exceptions = null; int submitted = 0, completed = 0; AtomicBoolean done = new AtomicBoolean(false); replicaSwitched.set(false); // submit call for the primary replica. submitted += addCallsForCurrentReplica(cs, rl); try { // wait for the timeout to see whether the primary responds back Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeBeforeReplicas, TimeUnit.MICROSECONDS); // Yes, microseconds if (f != null) { Pair<Result[], ScannerCallable> r = f.get(); if (r != null && r.getSecond() != null) { updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); } return r == null ? null : r.getFirst(); // great we got a response } } catch (ExecutionException e) { // the primary call failed with RetriesExhaustedException or DoNotRetryIOException // but the secondaries might still succeed. Continue on the replica RPCs. exceptions = new ArrayList<ExecutionException>(rl.size()); exceptions.add(e); completed++; } catch (CancellationException e) { throw new InterruptedIOException(e.getMessage()); } catch (InterruptedException e) { throw new InterruptedIOException(e.getMessage()); } // submit call for the all of the secondaries at once // TODO: this may be an overkill for large region replication submitted += addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1); try { while (completed < submitted) { try { Future<Pair<Result[], ScannerCallable>> f = cs.take(); Pair<Result[], ScannerCallable> r = f.get(); if (r != null && r.getSecond() != null) { updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); } return r == null ? null : r.getFirst(); // great we got an answer } catch (ExecutionException e) { // if not cancel or interrupt, wait until all RPC's are done // one of the tasks failed. Save the exception for later. if (exceptions == null) exceptions = new ArrayList<ExecutionException>(rl.size()); exceptions.add(e); completed++; } } } catch (CancellationException e) { throw new InterruptedIOException(e.getMessage()); } catch (InterruptedException e) { throw new InterruptedIOException(e.getMessage()); } finally { // We get there because we were interrupted or because one or more of the // calls succeeded or failed. In all case, we stop all our tasks. cs.cancelAll(true); } if (exceptions != null && !exceptions.isEmpty()) { RpcRetryingCallerWithReadReplicas.throwEnrichedException( exceptions.get(0), retries); // just rethrow the first exception for now. } return null; // unreachable }
public HRegionInfo getHRegionInfo() { return currentScannerCallable.getHRegionInfo(); }
public void setServerHasMoreResults(boolean serverHasMoreResults) { currentScannerCallable.setServerHasMoreResults(serverHasMoreResults); }
public void setCaching(int caching) { currentScannerCallable.setCaching(caching); }
@Override public long sleep(long pause, int tries) { return callable.sleep(pause, tries); }
@Override public void throwable(Throwable t, boolean retrying) { callable.throwable(t, retrying); }
public void setHasMoreResultsContext(boolean serverHasMoreResultsContext) { currentScannerCallable.setHasMoreResultsContext(serverHasMoreResultsContext); }
/** * @return true when the most recent RPC response indicated that the response was a heartbeat * message. Heartbeat messages are sent back from the server when the processing of the scan * request exceeds a certain time threshold. Heartbeats allow the server to avoid timeouts * during long running scan operations. */ public boolean isHeartbeatMessage() { return currentScannerCallable != null && currentScannerCallable.isHeartbeatMessage(); }
public int getCaching() { return currentScannerCallable.getCaching(); }
public boolean getServerHasMoreResults() { return currentScannerCallable.getServerHasMoreResults(); }