1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
23 import java.util.ArrayList;
24 import java.util.HashSet;
25 import java.util.List;
26 import java.util.Set;
27 import java.util.concurrent.Callable;
28 import java.util.concurrent.CancellationException;
29 import java.util.concurrent.ExecutionException;
30 import java.util.concurrent.ExecutorService;
31 import java.util.concurrent.Future;
32 import java.util.concurrent.TimeUnit;
33 import java.util.concurrent.atomic.AtomicBoolean;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.hbase.classification.InterfaceAudience;
38 import org.apache.hadoop.conf.Configuration;
39 import org.apache.hadoop.hbase.HRegionInfo;
40 import org.apache.hadoop.hbase.RegionLocations;
41 import org.apache.hadoop.hbase.TableName;
42 import org.apache.hadoop.hbase.util.BoundedCompletionService;
43 import org.apache.hadoop.hbase.util.Pair;
44
45
46
47
48
49
50
51
52
53
54
55
56 @InterfaceAudience.Private
57 class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
58 private final Log LOG = LogFactory.getLog(this.getClass());
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
73 public ScannerCallableWithReplicas (TableName tableName, ClusterConnection cConnection,
74 ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan,
75 int retries, int scannerTimeout, int caching, Configuration conf,
76 RpcRetryingCaller<Result []> caller) {
77 this.currentScannerCallable = baseCallable;
78 this.cConnection = cConnection;
79 this.pool = pool;
80 if (timeBeforeReplicas < 0) {
81 throw new IllegalArgumentException("Invalid value of operation timeout on the primary");
82 }
83 this.timeBeforeReplicas = timeBeforeReplicas;
84 this.scan = scan;
85 this.retries = retries;
86 this.tableName = tableName;
87 this.conf = conf;
88 this.scannerTimeout = scannerTimeout;
89 this.caller = caller;
90 }
91
92 public void setClose() {
93 currentScannerCallable.setClose();
94 }
95
96 public void setCaching(int caching) {
97 currentScannerCallable.setCaching(caching);
98 }
99
100 public int getCaching() {
101 return currentScannerCallable.getCaching();
102 }
103
104 public HRegionInfo getHRegionInfo() {
105 return currentScannerCallable.getHRegionInfo();
106 }
107
108 public boolean getServerHasMoreResults() {
109 return currentScannerCallable.getServerHasMoreResults();
110 }
111
112 public void setServerHasMoreResults(boolean serverHasMoreResults) {
113 currentScannerCallable.setServerHasMoreResults(serverHasMoreResults);
114 }
115
116 public boolean hasMoreResultsContext() {
117 return currentScannerCallable.hasMoreResultsContext();
118 }
119
120 public void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
121 currentScannerCallable.setHasMoreResultsContext(serverHasMoreResultsContext);
122 }
123
124 @Override
125 public Result [] call(int timeout) throws IOException {
126
127
128
129
130 if (currentScannerCallable != null && currentScannerCallable.closed) {
131
132
133 if (LOG.isTraceEnabled()) {
134 LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId);
135 }
136 Result[] r = currentScannerCallable.call(timeout);
137 currentScannerCallable = null;
138 return r;
139 }
140
141
142
143
144
145
146
147 RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(true,
148 RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName,
149 currentScannerCallable.getRow());
150
151
152
153 BoundedCompletionService<Pair<Result[], ScannerCallable>> cs =
154 new BoundedCompletionService<Pair<Result[], ScannerCallable>>(pool, rl.size() * 5);
155
156 List<ExecutionException> exceptions = null;
157 int submitted = 0, completed = 0;
158 AtomicBoolean done = new AtomicBoolean(false);
159 replicaSwitched.set(false);
160
161 submitted += addCallsForCurrentReplica(cs, rl);
162 try {
163
164 Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeBeforeReplicas,
165 TimeUnit.MICROSECONDS);
166 if (f != null) {
167 Pair<Result[], ScannerCallable> r = f.get();
168 if (r != null && r.getSecond() != null) {
169 updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
170 }
171 return r == null ? null : r.getFirst();
172 }
173 } catch (ExecutionException e) {
174
175
176 exceptions = new ArrayList<ExecutionException>(rl.size());
177 exceptions.add(e);
178 completed++;
179 } catch (CancellationException e) {
180 throw new InterruptedIOException(e.getMessage());
181 } catch (InterruptedException e) {
182 throw new InterruptedIOException(e.getMessage());
183 }
184
185
186 submitted += addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1);
187 try {
188 while (completed < submitted) {
189 try {
190 Future<Pair<Result[], ScannerCallable>> f = cs.take();
191 Pair<Result[], ScannerCallable> r = f.get();
192 if (r != null && r.getSecond() != null) {
193 updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
194 }
195 return r == null ? null : r.getFirst();
196 } catch (ExecutionException e) {
197
198
199 if (exceptions == null) exceptions = new ArrayList<ExecutionException>(rl.size());
200 exceptions.add(e);
201 completed++;
202 }
203 }
204 } catch (CancellationException e) {
205 throw new InterruptedIOException(e.getMessage());
206 } catch (InterruptedException e) {
207 throw new InterruptedIOException(e.getMessage());
208 } finally {
209
210
211 cs.cancelAll(true);
212 }
213
214 if (exceptions != null && !exceptions.isEmpty()) {
215 RpcRetryingCallerWithReadReplicas.throwEnrichedException(exceptions.get(0),
216 retries);
217 }
218 return null;
219 }
220
221 private void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result,
222 AtomicBoolean done, ExecutorService pool) {
223 if (done.compareAndSet(false, true)) {
224 if (currentScannerCallable != scanner) replicaSwitched.set(true);
225 currentScannerCallable = scanner;
226
227 if (result != null && result.length != 0) this.lastResult = result[result.length - 1];
228 if (LOG.isTraceEnabled()) {
229 LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId +
230 " associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId());
231 }
232
233 outstandingCallables.remove(scanner);
234 for (ScannerCallable s : outstandingCallables) {
235 if (LOG.isTraceEnabled()) {
236 LOG.trace("Closing scanner id=" + s.scannerId +
237 ", replica=" + s.getHRegionInfo().getRegionId() +
238 " because slow and replica=" +
239 this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded");
240 }
241
242
243
244 s.setClose();
245 RetryingRPC r = new RetryingRPC(s);
246 pool.submit(r);
247 }
248
249 outstandingCallables.clear();
250 }
251 }
252
253
254
255
256
257
258 public boolean switchedToADifferentReplica() {
259 return replicaSwitched.get();
260 }
261
262 private int addCallsForCurrentReplica(
263 BoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl) {
264 RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable);
265 outstandingCallables.add(currentScannerCallable);
266 cs.submit(retryingOnReplica);
267 return 1;
268 }
269
270 private int addCallsForOtherReplicas(
271 BoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl, int min,
272 int max) {
273 if (scan.getConsistency() == Consistency.STRONG) {
274 return 0;
275 }
276 for (int id = min; id <= max; id++) {
277 if (currentScannerCallable.getHRegionInfo().getReplicaId() == id) {
278 continue;
279 }
280 ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
281 if (this.lastResult != null) {
282 s.getScan().setStartRow(this.lastResult.getRow());
283 }
284 outstandingCallables.add(s);
285 RetryingRPC retryingOnReplica = new RetryingRPC(s);
286 cs.submit(retryingOnReplica);
287 }
288 return max - min + 1;
289 }
290
291 class RetryingRPC implements Callable<Pair<Result[], ScannerCallable>> {
292 final ScannerCallable callable;
293
294 RetryingRPC(ScannerCallable callable) {
295 this.callable = callable;
296 }
297
298 @Override
299 public Pair<Result[], ScannerCallable> call() throws IOException {
300
301
302
303
304
305 RpcRetryingCaller<Result[]> caller = ScannerCallableWithReplicas.this.caller;
306 if (scan.getConsistency() == Consistency.TIMELINE) {
307 caller = new RpcRetryingCallerFactory(ScannerCallableWithReplicas.this.conf).
308 <Result[]>newCaller();
309 }
310 Result[] res = caller.callWithRetries(callable, scannerTimeout);
311 return new Pair<Result[], ScannerCallable>(res, callable);
312 }
313 }
314
315 @Override
316 public void prepare(boolean reload) throws IOException {
317 }
318
319 @Override
320 public void throwable(Throwable t, boolean retrying) {
321 currentScannerCallable.throwable(t, retrying);
322 }
323
324 @Override
325 public String getExceptionMessageAdditionalDetail() {
326 return currentScannerCallable.getExceptionMessageAdditionalDetail();
327 }
328
329 @Override
330 public long sleep(long pause, int tries) {
331 return currentScannerCallable.sleep(pause, tries);
332 }
333 }