[ 
https://issues.apache.org/jira/browse/HADOOP-19139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17838072#comment-17838072
 ] 

ASF GitHub Bot commented on HADOOP-19139:
-----------------------------------------

saxenapranav commented on code in PR #6699:
URL: https://github.com/apache/hadoop/pull/6699#discussion_r1568558190


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java:
##########
@@ -376,32 +439,48 @@ private int readLastBlock(final byte[] b, final int off, 
final int len)
     // data need to be copied to user buffer from index bCursor,
     // AbfsInutStream buffer is going to contain data from last block start. In
     // that case bCursor will be set to fCursor - lastBlockStart
-    long lastBlockStart = max(0, contentLength - footerReadSize);
+    if (!fileStatusInformationPresent.get()) {
+      long lastBlockStart = max(0, (fCursor + len) - footerReadSize);
+      bCursor = (int) (fCursor - lastBlockStart);
+      return optimisedRead(b, off, len, lastBlockStart, min(fCursor + len, 
footerReadSize), true);
+    }
+    long lastBlockStart = max(0, getContentLength() - footerReadSize);
     bCursor = (int) (fCursor - lastBlockStart);
     // 0 if contentlength is < buffersize
-    long actualLenToRead = min(footerReadSize, contentLength);
-    return optimisedRead(b, off, len, lastBlockStart, actualLenToRead);
+    long actualLenToRead = min(footerReadSize, getContentLength());
+    return optimisedRead(b, off, len, lastBlockStart, actualLenToRead, false);
   }
 
   private int optimisedRead(final byte[] b, final int off, final int len,
-      final long readFrom, final long actualLen) throws IOException {
+      final long readFrom, final long actualLen,
+      final boolean isReadWithoutContentLengthInformation) throws IOException {
     fCursor = readFrom;
     int totalBytesRead = 0;
     int lastBytesRead = 0;
     try {
       buffer = new byte[bufferSize];
+      boolean fileStatusInformationPresentBeforeRead = 
fileStatusInformationPresent.get();
       for (int i = 0;
-           i < MAX_OPTIMIZED_READ_ATTEMPTS && fCursor < contentLength; i++) {
+           i < MAX_OPTIMIZED_READ_ATTEMPTS && 
(!fileStatusInformationPresent.get()
+               || fCursor < getContentLength()); i++) {

Review Comment:
   Content length would not be available for the first optimized read in case
   of lazy head optimization in inputStream. In such case, read of the first 
optimized read
   would be done without the contentLength constraint. Post first call, the 
contentLength
   would be present and should be used for further reads.
   
   Have added it as comment.
   





> [ABFS]: No GetPathStatus call for opening AbfsInputStream
> ---------------------------------------------------------
>
>                 Key: HADOOP-19139
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19139
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/azure
>            Reporter: Pranav Saxena
>            Assignee: Pranav Saxena
>            Priority: Major
>              Labels: pull-request-available
>
> Read API gives contentLen and etag of the path. This information would be 
> used in future calls on that inputStream. Prior information of eTag is of not 
> much importance.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to