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 @Override
109 public Result [] call(int timeout) throws IOException {
110
111
112
113
114 if (currentScannerCallable != null && currentScannerCallable.closed) {
115
116
117 if (LOG.isTraceEnabled()) {
118 LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId);
119 }
120 Result[] r = currentScannerCallable.call(timeout);
121 currentScannerCallable = null;
122 return r;
123 }
124
125
126
127
128
129
130
131 RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(true,
132 RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName,
133 currentScannerCallable.getRow());
134
135
136
137 BoundedCompletionService<Pair<Result[], ScannerCallable>> cs =
138 new BoundedCompletionService<Pair<Result[], ScannerCallable>>(pool, rl.size() * 5);
139
140 List<ExecutionException> exceptions = null;
141 int submitted = 0, completed = 0;
142 AtomicBoolean done = new AtomicBoolean(false);
143 replicaSwitched.set(false);
144
145 submitted += addCallsForCurrentReplica(cs, rl);
146 try {
147
148 Future<Pair<Result[], ScannerCallable>> f = cs.poll(timeBeforeReplicas,
149 TimeUnit.MICROSECONDS);
150 if (f != null) {
151 Pair<Result[], ScannerCallable> r = f.get();
152 if (r != null && r.getSecond() != null) {
153 updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
154 }
155 return r == null ? null : r.getFirst();
156 }
157 } catch (ExecutionException e) {
158
159
160 exceptions = new ArrayList<ExecutionException>(rl.size());
161 exceptions.add(e);
162 completed++;
163 } catch (CancellationException e) {
164 throw new InterruptedIOException(e.getMessage());
165 } catch (InterruptedException e) {
166 throw new InterruptedIOException(e.getMessage());
167 }
168
169
170 submitted += addCallsForOtherReplicas(cs, rl, 0, rl.size() - 1);
171 try {
172 while (completed < submitted) {
173 try {
174 Future<Pair<Result[], ScannerCallable>> f = cs.take();
175 Pair<Result[], ScannerCallable> r = f.get();
176 if (r != null && r.getSecond() != null) {
177 updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool);
178 }
179 return r == null ? null : r.getFirst();
180 } catch (ExecutionException e) {
181
182
183 if (exceptions == null) exceptions = new ArrayList<ExecutionException>(rl.size());
184 exceptions.add(e);
185 completed++;
186 }
187 }
188 } catch (CancellationException e) {
189 throw new InterruptedIOException(e.getMessage());
190 } catch (InterruptedException e) {
191 throw new InterruptedIOException(e.getMessage());
192 } finally {
193
194
195 cs.cancelAll(true);
196 }
197
198 if (exceptions != null && !exceptions.isEmpty()) {
199 RpcRetryingCallerWithReadReplicas.throwEnrichedException(exceptions.get(0),
200 retries);
201 }
202 return null;
203 }
204
205 private void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result,
206 AtomicBoolean done, ExecutorService pool) {
207 if (done.compareAndSet(false, true)) {
208 if (currentScannerCallable != scanner) replicaSwitched.set(true);
209 currentScannerCallable = scanner;
210
211 if (result != null && result.length != 0) this.lastResult = result[result.length - 1];
212 if (LOG.isTraceEnabled()) {
213 LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId +
214 " associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId());
215 }
216
217 outstandingCallables.remove(scanner);
218 for (ScannerCallable s : outstandingCallables) {
219 if (LOG.isTraceEnabled()) {
220 LOG.trace("Closing scanner id=" + s.scannerId +
221 ", replica=" + s.getHRegionInfo().getRegionId() +
222 " because slow and replica=" +
223 this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded");
224 }
225
226
227
228 s.setClose();
229 RetryingRPC r = new RetryingRPC(s);
230 pool.submit(r);
231 }
232
233 outstandingCallables.clear();
234 }
235 }
236
237
238
239
240
241
242 public boolean switchedToADifferentReplica() {
243 return replicaSwitched.get();
244 }
245
246 private int addCallsForCurrentReplica(
247 BoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl) {
248 RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable);
249 outstandingCallables.add(currentScannerCallable);
250 cs.submit(retryingOnReplica);
251 return 1;
252 }
253
254 private int addCallsForOtherReplicas(
255 BoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl, int min,
256 int max) {
257 if (scan.getConsistency() == Consistency.STRONG) {
258 return 0;
259 }
260 for (int id = min; id <= max; id++) {
261 if (currentScannerCallable.getHRegionInfo().getReplicaId() == id) {
262 continue;
263 }
264 ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
265 if (this.lastResult != null) {
266 s.getScan().setStartRow(this.lastResult.getRow());
267 }
268 outstandingCallables.add(s);
269 RetryingRPC retryingOnReplica = new RetryingRPC(s);
270 cs.submit(retryingOnReplica);
271 }
272 return max - min + 1;
273 }
274
275 class RetryingRPC implements Callable<Pair<Result[], ScannerCallable>> {
276 final ScannerCallable callable;
277
278 RetryingRPC(ScannerCallable callable) {
279 this.callable = callable;
280 }
281
282 @Override
283 public Pair<Result[], ScannerCallable> call() throws IOException {
284
285
286
287
288
289 RpcRetryingCaller<Result[]> caller = ScannerCallableWithReplicas.this.caller;
290 if (scan.getConsistency() == Consistency.TIMELINE) {
291 caller = new RpcRetryingCallerFactory(ScannerCallableWithReplicas.this.conf).
292 <Result[]>newCaller();
293 }
294 Result[] res = caller.callWithRetries(callable, scannerTimeout);
295 return new Pair<Result[], ScannerCallable>(res, callable);
296 }
297 }
298
299 @Override
300 public void prepare(boolean reload) throws IOException {
301 }
302
303 @Override
304 public void throwable(Throwable t, boolean retrying) {
305 currentScannerCallable.throwable(t, retrying);
306 }
307
308 @Override
309 public String getExceptionMessageAdditionalDetail() {
310 return currentScannerCallable.getExceptionMessageAdditionalDetail();
311 }
312
313 @Override
314 public long sleep(long pause, int tries) {
315 return currentScannerCallable.sleep(pause, tries);
316 }
317 }