[
https://issues.apache.org/jira/browse/HADOOP-19139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17840665#comment-17840665
]
ASF GitHub Bot commented on HADOOP-19139:
-----------------------------------------
anmolanmol1234 commented on code in PR #6699:
URL: https://github.com/apache/hadoop/pull/6699#discussion_r1578933240
##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java:
##########
@@ -231,7 +237,17 @@ public void testAbfsHttpResponseStatistics() throws
IOException {
// 1 read request = 1 connection and 1 get response
expectedConnectionsMade++;
expectedGetResponses++;
- expectedBytesReceived += bytesWrittenToFile;
+ if (!getConfiguration().getHeadOptimizationForInputStream()) {
+ expectedBytesReceived += bytesWrittenToFile;
+ } else {
+ /*
+ * With head optimization enabled, the abfsInputStream is not aware
+ * of the contentLength and hence, it would only read data for which
the range
+ * is provided. With the first remote call done, the inputStream will
get
+ * aware of the contentLength and would be able to use it for further
reads.
+ */
+ expectedBytesReceived += 1;
Review Comment:
That should mean +1 for operation, why would the bytes received be 1. Bytes
received should be equal to the range it read right
##########
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++) {
lastBytesRead = readInternal(fCursor, buffer, limit,
(int) actualLen - limit, true);
if (lastBytesRead > 0) {
totalBytesRead += lastBytesRead;
+ boolean shouldBreak = !fileStatusInformationPresentBeforeRead
+ && totalBytesRead == (int) actualLen;
limit += lastBytesRead;
fCursor += lastBytesRead;
fCursorAfterLastRead = fCursor;
+ if (shouldBreak) {
+ break;
+ }
}
}
} catch (IOException e) {
+ if (isNonRetriableOptimizedReadException(e)) {
+ throw e;
Review Comment:
Can you explain this a bit more, on the exception only for FileNotFound
being thrown ?
##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemAuthorization.java:
##########
@@ -328,13 +328,13 @@ private void executeOp(Path reqPath, AzureBlobFileSystem
fs,
fs.open(reqPath);
break;
case Open:
- InputStream is = fs.open(reqPath);
- if (getConfiguration().getHeadOptimizationForInputStream()) {
- try {
- is.read();
- } catch (IOException ex) {
- is.close();
- throw (IOException) ex.getCause();
+ try(InputStream is = fs.open(reqPath)) {
+ if (getConfiguration().isInputStreamLazyOptimizationEnabled()) {
+ try {
+ is.read();
+ } catch (IOException ex) {
+ throw (IOException) ex.getCause();
+ }
Review Comment:
finally block to close the input stream is missing
> [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]