/
ScannerCallableWithReplicas.java
423 lines (382 loc) · 16.5 KB
/
ScannerCallableWithReplicas.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.client.ClientScanner.createClosestRowBefore;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
/**
* This class has the logic for handling scanners for regions with and without replicas.
* 1. A scan is attempted on the default (primary) region
* 2. The scanner sends all the RPCs to the default region until it is done, or, there
* is a timeout on the default (a timeout of zero is disallowed).
* 3. If there is a timeout in (2) above, scanner(s) is opened on the non-default replica(s)
* 4. The results from the first successful scanner are taken, and it is stored which server
* returned the results.
* 5. The next RPCs are done on the above stored server until it is done or there is a timeout,
* in which case, the other replicas are queried (as in (3) above).
*
*/
@InterfaceAudience.Private
class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
private final Log LOG = LogFactory.getLog(this.getClass());
volatile ScannerCallable currentScannerCallable;
AtomicBoolean replicaSwitched = new AtomicBoolean(false);
final ClusterConnection cConnection;
protected final ExecutorService pool;
protected final int timeBeforeReplicas;
private final Scan scan;
private final int retries;
private Result lastResult;
private final RpcRetryingCaller<Result[]> caller;
private final TableName tableName;
private Configuration conf;
private int scannerTimeout;
private Set<ScannerCallable> outstandingCallables = new HashSet<ScannerCallable>();
private boolean someRPCcancelled = false; //required for testing purposes only
public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConnection,
ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan,
int retries, int scannerTimeout, int caching, Configuration conf,
RpcRetryingCaller<Result []> caller) {
this.currentScannerCallable = baseCallable;
this.cConnection = cConnection;
this.pool = pool;
if (timeBeforeReplicas < 0) {
throw new IllegalArgumentException("Invalid value of operation timeout on the primary");
}
this.timeBeforeReplicas = timeBeforeReplicas;
this.scan = scan;
this.retries = retries;
this.tableName = tableName;
this.conf = conf;
this.scannerTimeout = scannerTimeout;
this.caller = caller;
}
public void setClose() {
currentScannerCallable.setClose();
}
public void setCaching(int caching) {
currentScannerCallable.setCaching(caching);
}
public int getCaching() {
return currentScannerCallable.getCaching();
}
public HRegionInfo getHRegionInfo() {
return currentScannerCallable.getHRegionInfo();
}
public boolean getServerHasMoreResults() {
return currentScannerCallable.getServerHasMoreResults();
}
public void setServerHasMoreResults(boolean serverHasMoreResults) {
currentScannerCallable.setServerHasMoreResults(serverHasMoreResults);
}
public boolean hasMoreResultsContext() {
return currentScannerCallable.hasMoreResultsContext();
}
public void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
currentScannerCallable.setHasMoreResultsContext(serverHasMoreResultsContext);
}
@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.isTraceEnabled()) {
LOG.trace("Closing scanner id=" + 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)
ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs =
new ResultBoundedCompletionService<Pair<Result[], ScannerCallable>>(
new RpcRetryingCallerFactory(ScannerCallableWithReplicas.this.conf), 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();
}
if (exceptions != null && !exceptions.isEmpty()) {
RpcRetryingCallerWithReadReplicas.throwEnrichedException(exceptions.get(0),
retries); // just rethrow the first exception for now.
}
return null; // unreachable
}
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 id=" + currentScannerCallable.scannerId +
" associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId());
}
// close all outstanding replica scanners but the one we heard back from
outstandingCallables.remove(scanner);
for (ScannerCallable s : outstandingCallables) {
if (LOG.isTraceEnabled()) {
LOG.trace("Closing scanner id=" + s.scannerId +
", replica=" + s.getHRegionInfo().getRegionId() +
" because slow and replica=" +
this.currentScannerCallable.getHRegionInfo().getReplicaId() + " 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();
final RetryingRPC r = new RetryingRPC(s);
pool.submit(new Callable<Void>(){
@Override
public Void call() throws Exception {
r.call(scannerTimeout);
return null;
}
});
}
// now clear outstandingCallables since we scheduled a close for all the contained scanners
outstandingCallables.clear();
}
}
/**
* When a scanner switches in the middle of scanning (the 'next' call fails
* for example), the upper layer {@link ClientScanner} needs to know
* @return
*/
public boolean switchedToADifferentReplica() {
return replicaSwitched.get();
}
private int addCallsForCurrentReplica(
ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl) {
RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable);
outstandingCallables.add(currentScannerCallable);
cs.submit(retryingOnReplica, scannerTimeout, currentScannerCallable.id);
return 1;
}
private int addCallsForOtherReplicas(
ResultBoundedCompletionService<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.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);
}
return max - min + 1;
}
/**
* 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]));
}
}
}
@VisibleForTesting
boolean isAnyRPCcancelled() {
return someRPCcancelled;
}
class RetryingRPC implements RetryingCallable<Pair<Result[], ScannerCallable>>, Cancellable {
final ScannerCallable callable;
RpcRetryingCaller<Result[]> caller;
private volatile boolean cancelled = false;
RetryingRPC(ScannerCallable callable) {
this.callable = callable;
// For the Consistency.STRONG (default case), we reuse the caller
// to keep compatibility with what is done in the past
// For the Consistency.TIMELINE case, we can't reuse the caller
// since we could be making parallel RPCs (caller.callWithRetries is synchronized
// and we can't invoke it multiple times at the same time)
this.caller = ScannerCallableWithReplicas.this.caller;
if (scan.getConsistency() == Consistency.TIMELINE) {
this.caller = new RpcRetryingCallerFactory(ScannerCallableWithReplicas.this.conf).
<Result[]>newCaller();
}
}
@Override
public Pair<Result[], ScannerCallable> call(int callTimeout) throws IOException {
// since the retries is done within the ResultBoundedCompletionService,
// we don't invoke callWithRetries here
if (cancelled) {
return null;
}
Result[] res = this.caller.callWithoutRetries(this.callable, callTimeout);
return new Pair<Result[], ScannerCallable>(res, this.callable);
}
@Override
public void prepare(boolean reload) throws IOException {
if (cancelled) return;
if (Thread.interrupted()) {
throw new InterruptedIOException();
}
callable.prepare(reload);
}
@Override
public void throwable(Throwable t, boolean retrying) {
callable.throwable(t, retrying);
}
@Override
public String getExceptionMessageAdditionalDetail() {
return callable.getExceptionMessageAdditionalDetail();
}
@Override
public long sleep(long pause, int tries) {
return callable.sleep(pause, tries);
}
@Override
public void cancel() {
cancelled = true;
caller.cancel();
if (callable.getController() != null) {
callable.getController().startCancel();
}
someRPCcancelled = true;
}
@Override
public boolean isCancelled() {
return cancelled;
}
}
@Override
public void prepare(boolean reload) throws IOException {
}
@Override
public void throwable(Throwable t, boolean retrying) {
currentScannerCallable.throwable(t, retrying);
}
@Override
public String getExceptionMessageAdditionalDetail() {
return currentScannerCallable.getExceptionMessageAdditionalDetail();
}
@Override
public long sleep(long pause, int tries) {
return currentScannerCallable.sleep(pause, tries);
}
}