[ https://issues.apache.org/jira/browse/HADOOP-18521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17632535#comment-17632535 ]
ASF GitHub Bot commented on HADOOP-18521: ----------------------------------------- steveloughran commented on code in PR #5117: URL: https://github.com/apache/hadoop/pull/5117#discussion_r1020503409 ########## hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java: ########## @@ -454,31 +588,65 @@ ReadBuffer getNextBlockToRead() throws InterruptedException { */ void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) { if (LOGGER.isTraceEnabled()) { - LOGGER.trace("ReadBufferWorker completed read file {} for offset {} outcome {} bytes {}", - buffer.getStream().getPath(), buffer.getOffset(), result, bytesActuallyRead); - } - synchronized (this) { - // If this buffer has already been purged during - // close of InputStream then we don't update the lists. - if (inProgressList.contains(buffer)) { - inProgressList.remove(buffer); + LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}; {}", + buffer.getStream().getPath(), buffer.getOffset(), bytesActuallyRead, buffer); + } + // decrement counter. + buffer.prefetchFinished(); + + try { + synchronized (this) { + // remove from the list + if (!inProgressList.remove(buffer)) { + // this is a sign of inconsistent state, so a major problem + String message = + String.format("Read completed from an operation not declared as in progress %s", + buffer); + LOGGER.warn(message); + // release the buffer (which may raise an exception) + placeBufferOnFreeList("read not in progress", buffer); + // report the failure + throw new IllegalStateException(message); + } + + boolean shouldFreeBuffer = false; + String freeBufferReason = ""; if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) { buffer.setStatus(ReadBufferStatus.AVAILABLE); buffer.setLength(bytesActuallyRead); } else { - freeList.push(buffer.getBufferindex()); - // buffer will be deleted as per the eviction policy. + // read failed or there was no data, -the buffer can be returned to the free list. + shouldFreeBuffer = true; + freeBufferReason = "failed read"; } // completed list also contains FAILED read buffers // for sending exception message to clients. buffer.setStatus(result); buffer.setTimeStamp(currentTimeMillis()); - completedReadList.add(buffer); + if (!buffer.isStreamClosed()) { + // completed reads are added to the list. + LOGGER.trace("Adding buffer to completed list {}", buffer); + completedReadList.add(buffer); Review Comment: making some changes. this is a complex bit of code and why I plan to write some unit tests to explore the results; i will take what you've done too @pranavsaxena-microsoft > ABFS ReadBufferManager buffer sharing across concurrent HTTP requests > --------------------------------------------------------------------- > > Key: HADOOP-18521 > URL: https://issues.apache.org/jira/browse/HADOOP-18521 > Project: Hadoop Common > Issue Type: Bug > Components: fs/azure > Affects Versions: 3.3.2, 3.3.3, 3.3.4 > Reporter: Steve Loughran > Assignee: Steve Loughran > Priority: Critical > Labels: pull-request-available > > AbfsInputStream.close() can trigger the return of buffers used for active > prefetch GET requests into the ReadBufferManager free buffer pool. > A subsequent prefetch by a different stream in the same process may acquire > this same buffer. This can lead to risk of corruption of its own prefetched > data, data which may then be returned to that other thread. > On releases without the fix for this (3.3.2 to 3.3.4), the bug can be avoided > by disabling all prefetching > {code} > fs.azure.readaheadqueue.depth > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org