1616
1717package  io .aiven .kafka .tieredstorage .chunkmanager ;
1818
19+ import  java .io .IOException ;
1920import  java .io .InputStream ;
2021import  java .util .List ;
2122import  java .util .Optional ;
23+ import  java .util .concurrent .CompletableFuture ;
24+ import  java .util .concurrent .CompletionException ;
25+ import  java .util .concurrent .Executor ;
26+ import  java .util .concurrent .ForkJoinPool ;
27+ import  java .util .function .Supplier ;
2228
2329import  io .aiven .kafka .tieredstorage .Chunk ;
30+ import  io .aiven .kafka .tieredstorage .chunkmanager .cache .ChunkCache ;
2431import  io .aiven .kafka .tieredstorage .manifest .SegmentEncryptionMetadata ;
2532import  io .aiven .kafka .tieredstorage .manifest .SegmentManifest ;
2633import  io .aiven .kafka .tieredstorage .security .AesEncryptionProvider ;
34+ import  io .aiven .kafka .tieredstorage .storage .BytesRange ;
2735import  io .aiven .kafka .tieredstorage .storage .ObjectFetcher ;
2836import  io .aiven .kafka .tieredstorage .storage .ObjectKey ;
2937import  io .aiven .kafka .tieredstorage .storage .StorageBackendException ;
3644public  class  DefaultChunkManager  implements  ChunkManager  {
3745    private  final  ObjectFetcher  fetcher ;
3846    private  final  AesEncryptionProvider  aesEncryptionProvider ;
47+     final  ChunkCache <?> chunkCache ;
48+     private  final  int  prefetchingSize ;
3949
40-     public  DefaultChunkManager (final  ObjectFetcher  fetcher , final  AesEncryptionProvider  aesEncryptionProvider ) {
50+     private  final  Executor  executor  = new  ForkJoinPool ();
51+ 
52+     public  DefaultChunkManager (final  ObjectFetcher  fetcher ,
53+                                final  AesEncryptionProvider  aesEncryptionProvider ,
54+                                final  ChunkCache <?> chunkCache ,
55+                                final  int  prefetchingSize ) {
4156        this .fetcher  = fetcher ;
4257        this .aesEncryptionProvider  = aesEncryptionProvider ;
58+         this .chunkCache  = chunkCache ;
59+         this .prefetchingSize  = prefetchingSize ;
4360    }
4461
4562    /** 
@@ -48,24 +65,60 @@ public DefaultChunkManager(final ObjectFetcher fetcher, final AesEncryptionProvi
4865     * @return an {@link InputStream} of the chunk, plain text (i.e., decrypted and decompressed). 
4966     */ 
5067    public  InputStream  getChunk (final  ObjectKey  objectKey , final  SegmentManifest  manifest ,
51-                                 final  int  chunkId ) throws  StorageBackendException  {
52-         final  Chunk  chunk  = manifest .chunkIndex ().chunks ().get (chunkId );
68+                                 final  int  chunkId ) throws  StorageBackendException , IOException  {
69+         final  var  currentChunk  = manifest .chunkIndex ().chunks ().get (chunkId );
70+         startPrefetching (objectKey , manifest , currentChunk .originalPosition  + currentChunk .originalSize );
71+ 
72+         final  ChunkKey  chunkKey  = new  ChunkKey (objectKey .value (), chunkId );
73+         return  chunkCache .getChunk (chunkKey , createChunkSupplier (objectKey , manifest , chunkId ));
74+     }
5375
54-         final  InputStream  chunkContent  = fetcher .fetch (objectKey , chunk .range ());
76+     private  void  startPrefetching (final  ObjectKey  segmentKey ,
77+                                   final  SegmentManifest  segmentManifest ,
78+                                   final  int  startPosition ) {
79+         if  (prefetchingSize  > 0 ) {
80+             final  BytesRange  prefetchingRange ;
81+             if  (Integer .MAX_VALUE  - startPosition  < prefetchingSize ) {
82+                 prefetchingRange  = BytesRange .of (startPosition , Integer .MAX_VALUE );
83+             } else  {
84+                 prefetchingRange  = BytesRange .ofFromPositionAndSize (startPosition , prefetchingSize );
85+             }
86+             final  var  chunks  = segmentManifest .chunkIndex ().chunksForRange (prefetchingRange );
87+             chunks .forEach (chunk  -> {
88+                 final  ChunkKey  chunkKey  = new  ChunkKey (segmentKey .value (), chunk .id );
89+                 chunkCache .supplyIfAbsent (chunkKey , createChunkSupplier (segmentKey , segmentManifest , chunk .id ));
90+             });
91+         }
92+     }
5593
56-         DetransformChunkEnumeration  detransformEnum  = new  BaseDetransformChunkEnumeration (chunkContent , List .of (chunk ));
57-         final  Optional <SegmentEncryptionMetadata > encryptionMetadata  = manifest .encryption ();
58-         if  (encryptionMetadata .isPresent ()) {
59-             detransformEnum  = new  DecryptionChunkEnumeration (
94+     private  Supplier <CompletableFuture <InputStream >> createChunkSupplier (final  ObjectKey  objectKey ,
95+                                                                          final  SegmentManifest  manifest ,
96+                                                                          final  int  chunkId ) {
97+         return  () -> CompletableFuture .supplyAsync (() -> {
98+             final  Chunk  chunk  = manifest .chunkIndex ().chunks ().get (chunkId );
99+ 
100+             final  InputStream  chunkContent ;
101+             try  {
102+                 chunkContent  = fetcher .fetch (objectKey , chunk .range ());
103+             } catch  (final  StorageBackendException  e ) {
104+                 throw  new  CompletionException (e );
105+             }
106+ 
107+             DetransformChunkEnumeration  detransformEnum  =
108+                 new  BaseDetransformChunkEnumeration (chunkContent , List .of (chunk ));
109+             final  Optional <SegmentEncryptionMetadata > encryptionMetadata  = manifest .encryption ();
110+             if  (encryptionMetadata .isPresent ()) {
111+                 detransformEnum  = new  DecryptionChunkEnumeration (
60112                    detransformEnum ,
61113                    encryptionMetadata .get ().ivSize (),
62114                    encryptedChunk  -> aesEncryptionProvider .decryptionCipher (encryptedChunk , encryptionMetadata .get ())
63-             );
64-         }
65-         if  (manifest .compression ()) {
66-             detransformEnum  = new  DecompressionChunkEnumeration (detransformEnum );
67-         }
68-         final  DetransformFinisher  detransformFinisher  = new  DetransformFinisher (detransformEnum );
69-         return  detransformFinisher .toInputStream ();
115+                 );
116+             }
117+             if  (manifest .compression ()) {
118+                 detransformEnum  = new  DecompressionChunkEnumeration (detransformEnum );
119+             }
120+             final  DetransformFinisher  detransformFinisher  = new  DetransformFinisher (detransformEnum );
121+             return  detransformFinisher .toInputStream ();
122+         }, executor );
70123    }
71124}
0 commit comments