@@ -68,6 +68,9 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
6868  //                                                      of valid bytes in buffer) 
6969  private  boolean  closed  = false ;
7070
71+   /** Stream statistics. */ 
72+   private  final  AbfsInputStreamStatistics  streamStatistics ;
73+ 
7174  public  AbfsInputStream (
7275          final  AbfsClient  client ,
7376          final  Statistics  statistics ,
@@ -86,6 +89,7 @@ public AbfsInputStream(
8689    this .readAheadEnabled  = true ;
8790    this .cachedSasToken  = new  CachedSASToken (
8891        abfsInputStreamContext .getSasTokenRenewPeriodForStreamsInSeconds ());
92+     this .streamStatistics  = abfsInputStreamContext .getStreamStatistics ();
8993  }
9094
9195  public  String  getPath () {
@@ -105,10 +109,21 @@ public int read() throws IOException {
105109
106110  @ Override 
107111  public  synchronized  int  read (final  byte [] b , final  int  off , final  int  len ) throws  IOException  {
112+     // check if buffer is null before logging the length 
113+     if  (b  != null ) {
114+       LOG .debug ("read requested b.length = {} offset = {} len = {}" , b .length ,
115+           off , len );
116+     } else  {
117+       LOG .debug ("read requested b = null offset = {} len = {}" , off , len );
118+     }
119+ 
108120    int  currentOff  = off ;
109121    int  currentLen  = len ;
110122    int  lastReadBytes ;
111123    int  totalReadBytes  = 0 ;
124+     if  (streamStatistics  != null ) {
125+       streamStatistics .readOperationStarted (off , len );
126+     }
112127    incrementReadOps ();
113128    do  {
114129      lastReadBytes  = readOneBlock (b , currentOff , currentLen );
@@ -130,6 +145,8 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
130145    }
131146
132147    Preconditions .checkNotNull (b );
148+     LOG .debug ("read one block requested b.length = {} off {} len {}" , b .length ,
149+         off , len );
133150
134151    if  (len  == 0 ) {
135152      return  0 ;
@@ -155,6 +172,7 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
155172      bCursor  = 0 ;
156173      limit  = 0 ;
157174      if  (buffer  == null ) {
175+         LOG .debug ("created new buffer size {}" , bufferSize );
158176        buffer  = new  byte [bufferSize ];
159177      }
160178
@@ -183,6 +201,11 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
183201    if  (statistics  != null ) {
184202      statistics .incrementBytesRead (bytesToRead );
185203    }
204+     if  (streamStatistics  != null ) {
205+       // Bytes read from the local buffer. 
206+       streamStatistics .bytesReadFromBuffer (bytesToRead );
207+       streamStatistics .bytesRead (bytesToRead );
208+     }
186209    return  bytesToRead ;
187210  }
188211
@@ -200,8 +223,11 @@ private int readInternal(final long position, final byte[] b, final int offset,
200223      int  numReadAheads  = this .readAheadQueueDepth ;
201224      long  nextSize ;
202225      long  nextOffset  = position ;
226+       LOG .debug ("read ahead enabled issuing readheads num = {}" , numReadAheads );
203227      while  (numReadAheads  > 0  && nextOffset  < contentLength ) {
204228        nextSize  = Math .min ((long ) bufferSize , contentLength  - nextOffset );
229+         LOG .debug ("issuing read ahead requestedOffset = {} requested size {}" ,
230+             nextOffset , nextSize );
205231        ReadBufferManager .getBufferManager ().queueReadAhead (this , nextOffset , (int ) nextSize );
206232        nextOffset  = nextOffset  + nextSize ;
207233        numReadAheads --;
@@ -211,13 +237,15 @@ private int readInternal(final long position, final byte[] b, final int offset,
211237      receivedBytes  = ReadBufferManager .getBufferManager ().getBlock (this , position , length , b );
212238      if  (receivedBytes  > 0 ) {
213239        incrementReadOps ();
240+         LOG .debug ("Received data from read ahead, not doing remote read" );
214241        return  receivedBytes ;
215242      }
216243
217244      // got nothing from read-ahead, do our own read now 
218245      receivedBytes  = readRemote (position , b , offset , length );
219246      return  receivedBytes ;
220247    } else  {
248+       LOG .debug ("read ahead disabled, reading remote" );
221249      return  readRemote (position , b , offset , length );
222250    }
223251  }
@@ -247,6 +275,11 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti
247275      LOG .trace ("Trigger client.read for path={} position={} offset={} length={}" , path , position , offset , length );
248276      op  = client .read (path , position , b , offset , length , tolerateOobAppends  ? "*"  : eTag , cachedSasToken .get ());
249277      cachedSasToken .update (op .getSasToken ());
278+       if  (streamStatistics  != null ) {
279+         streamStatistics .remoteReadOperation ();
280+       }
281+       LOG .debug ("issuing HTTP GET request params position = {} b.length = {} " 
282+           + "offset = {} length = {}" , position , b .length , offset , length );
250283      perfInfo .registerResult (op .getResult ()).registerSuccess (true );
251284      incrementReadOps ();
252285    } catch  (AzureBlobFileSystemException  ex ) {
@@ -262,6 +295,7 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti
262295    if  (bytesRead  > Integer .MAX_VALUE ) {
263296      throw  new  IOException ("Unexpected Content-Length" );
264297    }
298+     LOG .debug ("HTTP request read bytes = {}" , bytesRead );
265299    return  (int ) bytesRead ;
266300  }
267301
@@ -282,6 +316,7 @@ private void incrementReadOps() {
282316   */ 
283317  @ Override 
284318  public  synchronized  void  seek (long  n ) throws  IOException  {
319+     LOG .debug ("requested seek to position {}" , n );
285320    if  (closed ) {
286321      throw  new  IOException (FSExceptionMessages .STREAM_IS_CLOSED );
287322    }
@@ -292,13 +327,21 @@ public synchronized void seek(long n) throws IOException {
292327      throw  new  EOFException (FSExceptionMessages .CANNOT_SEEK_PAST_EOF );
293328    }
294329
330+     if  (streamStatistics  != null ) {
331+       streamStatistics .seek (n , fCursor );
332+     }
333+ 
295334    if  (n >=fCursor -limit  && n <=fCursor ) { // within buffer 
296335      bCursor  = (int ) (n -(fCursor -limit ));
336+       if  (streamStatistics  != null ) {
337+         streamStatistics .seekInBuffer ();
338+       }
297339      return ;
298340    }
299341
300342    // next read will read from here 
301343    fCursor  = n ;
344+     LOG .debug ("set fCursor to {}" , fCursor );
302345
303346    //invalidate buffer 
304347    limit  = 0 ;
@@ -390,6 +433,7 @@ public boolean seekToNewSource(long l) throws IOException {
390433  public  synchronized  void  close () throws  IOException  {
391434    closed  = true ;
392435    buffer  = null ; // de-reference the buffer so it can be GC'ed sooner 
436+     LOG .debug ("Closing {}" , this );
393437  }
394438
395439  /** 
@@ -443,4 +487,28 @@ protected void setCachedSasToken(final CachedSASToken cachedSasToken) {
443487    this .cachedSasToken  = cachedSasToken ;
444488  }
445489
490+   /** 
491+    * Getter for AbfsInputStreamStatistics. 
492+    * 
493+    * @return an instance of AbfsInputStreamStatistics. 
494+    */ 
495+   @ VisibleForTesting 
496+   public  AbfsInputStreamStatistics  getStreamStatistics () {
497+     return  streamStatistics ;
498+   }
499+ 
500+   /** 
501+    * Get the statistics of the stream. 
502+    * @return a string value. 
503+    */ 
504+   @ Override 
505+   public  String  toString () {
506+     final  StringBuilder  sb  = new  StringBuilder (super .toString ());
507+     if  (streamStatistics  != null ) {
508+       sb .append ("AbfsInputStream@(" ).append (this .hashCode ()).append ("){" );
509+       sb .append (streamStatistics .toString ());
510+       sb .append ("}" );
511+     }
512+     return  sb .toString ();
513+   }
446514}
0 commit comments