21
21
22
22
import java .io .IOException ;
23
23
import java .nio .ByteBuffer ;
24
+ import java .time .Duration ;
25
+ import java .time .Instant ;
24
26
import java .util .concurrent .CompletableFuture ;
25
27
import java .util .concurrent .Future ;
26
28
import java .util .concurrent .TimeUnit ;
31
33
import org .slf4j .Logger ;
32
34
import org .slf4j .LoggerFactory ;
33
35
36
+ import org .apache .hadoop .fs .statistics .DurationTracker ;
37
+
38
+ import static java .util .Objects .requireNonNull ;
39
+
34
40
import static org .apache .hadoop .io .IOUtils .cleanupWithLogger ;
35
41
36
42
/**
@@ -70,33 +76,37 @@ public abstract class CachingBlockManager extends BlockManager {
70
76
// Once set to true, any further caching requests will be ignored.
71
77
private final AtomicBoolean cachingDisabled ;
72
78
79
+ private final PrefetchingStatistics prefetchingStatistics ;
80
+
73
81
/**
74
82
* Constructs an instance of a {@code CachingBlockManager}.
75
83
*
76
84
* @param futurePool asynchronous tasks are performed in this pool.
77
85
* @param blockData information about each block of the underlying file.
78
86
* @param bufferPoolSize size of the in-memory cache in terms of number of blocks.
87
+ * @param prefetchingStatistics statistics for this stream.
79
88
*
80
- * @throws IllegalArgumentException if futurePool is null.
81
89
* @throws IllegalArgumentException if bufferPoolSize is zero or negative.
82
90
*/
83
91
public CachingBlockManager (
84
92
ExecutorServiceFuturePool futurePool ,
85
93
BlockData blockData ,
86
- int bufferPoolSize ) {
94
+ int bufferPoolSize ,
95
+ PrefetchingStatistics prefetchingStatistics ) {
87
96
super (blockData );
88
97
89
- Validate .checkNotNull (futurePool , "futurePool" );
90
98
Validate .checkPositiveInteger (bufferPoolSize , "bufferPoolSize" );
91
99
92
- this .futurePool = futurePool ;
100
+ this .futurePool = requireNonNull ( futurePool ) ;
93
101
this .bufferPoolSize = bufferPoolSize ;
94
102
this .numCachingErrors = new AtomicInteger ();
95
103
this .numReadErrors = new AtomicInteger ();
96
104
this .cachingDisabled = new AtomicBoolean ();
105
+ this .prefetchingStatistics = requireNonNull (prefetchingStatistics );
97
106
98
107
if (this .getBlockData ().getFileSize () > 0 ) {
99
- this .bufferPool = new BufferPool (bufferPoolSize , this .getBlockData ().getBlockSize ());
108
+ this .bufferPool = new BufferPool (bufferPoolSize , this .getBlockData ().getBlockSize (),
109
+ this .prefetchingStatistics );
100
110
this .cache = this .createCache ();
101
111
}
102
112
@@ -249,7 +259,7 @@ public void requestPrefetch(int blockNumber) {
249
259
}
250
260
251
261
BlockOperations .Operation op = this .ops .requestPrefetch (blockNumber );
252
- PrefetchTask prefetchTask = new PrefetchTask (data , this );
262
+ PrefetchTask prefetchTask = new PrefetchTask (data , this , Instant . now () );
253
263
Future <Void > prefetchFuture = this .futurePool .executeFunction (prefetchTask );
254
264
data .setPrefetch (prefetchFuture );
255
265
this .ops .end (op );
@@ -279,8 +289,10 @@ private void read(BufferData data) throws IOException {
279
289
}
280
290
}
281
291
282
- private void prefetch (BufferData data ) throws IOException {
292
+ private void prefetch (BufferData data , Instant taskQueuedStartTime ) throws IOException {
283
293
synchronized (data ) {
294
+ prefetchingStatistics .executorAcquired (
295
+ Duration .between (taskQueuedStartTime , Instant .now ()));
284
296
this .readBlock (
285
297
data ,
286
298
true ,
@@ -297,6 +309,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
297
309
}
298
310
299
311
BlockOperations .Operation op = null ;
312
+ DurationTracker tracker = null ;
300
313
301
314
synchronized (data ) {
302
315
try {
@@ -318,6 +331,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
318
331
}
319
332
320
333
if (isPrefetch ) {
334
+ tracker = prefetchingStatistics .prefetchOperationStarted ();
321
335
op = this .ops .prefetch (data .getBlockNumber ());
322
336
} else {
323
337
op = this .ops .getRead (data .getBlockNumber ());
@@ -333,13 +347,25 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
333
347
} catch (Exception e ) {
334
348
String message = String .format ("error during readBlock(%s)" , data .getBlockNumber ());
335
349
LOG .error (message , e );
350
+
351
+ if (isPrefetch && tracker != null ) {
352
+ tracker .failed ();
353
+ }
354
+
336
355
this .numReadErrors .incrementAndGet ();
337
356
data .setDone ();
338
357
throw e ;
339
358
} finally {
340
359
if (op != null ) {
341
360
this .ops .end (op );
342
361
}
362
+
363
+ if (isPrefetch ) {
364
+ prefetchingStatistics .prefetchOperationCompleted ();
365
+ if (tracker != null ) {
366
+ tracker .close ();
367
+ }
368
+ }
343
369
}
344
370
}
345
371
}
@@ -350,16 +376,18 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
350
376
private static class PrefetchTask implements Supplier <Void > {
351
377
private final BufferData data ;
352
378
private final CachingBlockManager blockManager ;
379
+ private final Instant taskQueuedStartTime ;
353
380
354
- PrefetchTask (BufferData data , CachingBlockManager blockManager ) {
381
+ PrefetchTask (BufferData data , CachingBlockManager blockManager , Instant taskQueuedStartTime ) {
355
382
this .data = data ;
356
383
this .blockManager = blockManager ;
384
+ this .taskQueuedStartTime = taskQueuedStartTime ;
357
385
}
358
386
359
387
@ Override
360
388
public Void get () {
361
389
try {
362
- this .blockManager .prefetch (data );
390
+ this .blockManager .prefetch (data , taskQueuedStartTime );
363
391
} catch (Exception e ) {
364
392
LOG .error ("error during prefetch" , e );
365
393
}
@@ -420,14 +448,18 @@ public void requestCaching(BufferData data) {
420
448
blockFuture = cf ;
421
449
}
422
450
423
- CachePutTask task = new CachePutTask (data , blockFuture , this );
451
+ CachePutTask task = new CachePutTask (data , blockFuture , this , Instant . now () );
424
452
Future <Void > actionFuture = this .futurePool .executeFunction (task );
425
453
data .setCaching (actionFuture );
426
454
this .ops .end (op );
427
455
}
428
456
}
429
457
430
- private void addToCacheAndRelease (BufferData data , Future <Void > blockFuture ) {
458
+ private void addToCacheAndRelease (BufferData data , Future <Void > blockFuture ,
459
+ Instant taskQueuedStartTime ) {
460
+ prefetchingStatistics .executorAcquired (
461
+ Duration .between (taskQueuedStartTime , Instant .now ()));
462
+
431
463
if (this .closed ) {
432
464
return ;
433
465
}
@@ -493,7 +525,7 @@ private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture) {
493
525
}
494
526
495
527
protected BlockCache createCache () {
496
- return new SingleFilePerBlockCache ();
528
+ return new SingleFilePerBlockCache (prefetchingStatistics );
497
529
}
498
530
499
531
protected void cachePut (int blockNumber , ByteBuffer buffer ) throws IOException {
@@ -513,18 +545,22 @@ private static class CachePutTask implements Supplier<Void> {
513
545
// Block manager that manages this block.
514
546
private final CachingBlockManager blockManager ;
515
547
548
+ private final Instant taskQueuedStartTime ;
549
+
516
550
CachePutTask (
517
551
BufferData data ,
518
552
Future <Void > blockFuture ,
519
- CachingBlockManager blockManager ) {
553
+ CachingBlockManager blockManager ,
554
+ Instant taskQueuedStartTime ) {
520
555
this .data = data ;
521
556
this .blockFuture = blockFuture ;
522
557
this .blockManager = blockManager ;
558
+ this .taskQueuedStartTime = taskQueuedStartTime ;
523
559
}
524
560
525
561
@ Override
526
562
public Void get () {
527
- this .blockManager .addToCacheAndRelease (this .data , this .blockFuture );
563
+ this .blockManager .addToCacheAndRelease (this .data , this .blockFuture , taskQueuedStartTime );
528
564
return null ;
529
565
}
530
566
}
0 commit comments