You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2022/11/07 17:39:19 UTC

[GitHub] [hadoop] steveloughran opened a new pull request, #5117: HADOOP-18521. ABFS ReadBufferManager does not reuse in-progress buffers

steveloughran opened a new pull request, #5117:
URL: https://github.com/apache/hadoop/pull/5117

   
   Addresses the issue by not trying to cancel in-progress reads when a stream
   is closed()...they are allowed to continue and then their data discarded.
   To enable discarding, AbfsInputStreams export their `closed` state in
   which is now AtomicBool internally so reader threads can probe it.
   
   The shared buffers now have owner tracking, which will reject
   * attempts to acquire an owned buffer
   * attempts to return a buffer not owned
   Plus
   * Lots of other invariants added to validate the state
   * useful to string values
   
   Also adds path and stream capability probe for the fix;
   cloudstore "pathcapability" probe can report this.
   Hadoop 3.3.2 added the path capability
   "fs.capability.paths.acls", so two probes can
   determine if abfs is exposed: 
   
   not vulnerable
     !hasPathCability("fs.capability.paths.acls")
     || hasPathCability("fs.azure.capability.prefetch.safe")
   
   vulnerable
     hasPathCability("fs.capability.paths.acls")
     && !hasPathCability("fs.azure.capability.prefetch.safe")
   
   It can also be demanded in an openFile() call.
   That block the code ever working on a version without
   the race condition. Possibly a bit excessive.
   
   ### How was this patch tested?
   
   needs more tests with multi GB csv files to validate the patch.
   Unable to come up with good tests to recreate the failure condition.
   
   
   ### For code changes:
   
   - [X] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
   - [X] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033420242


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java:
##########
@@ -555,7 +579,7 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t
           throw new FileNotFoundException(ere.getMessage());
         }
       }
-      throw new IOException(ex);
+      throw ex;

Review Comment:
   Any specific reason for changing the exception type from IOException to AzureBlobFileSystemException ? 



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java:
##########
@@ -851,4 +880,67 @@ private void resetReadBufferManager(int bufferSize, int threshold) {
     // by successive tests can lead to OOM based on the dev VM/machine capacity.
     System.gc();
   }
+
+  /**
+   * The first readahead closes the stream.
+   */
+  @Test
+  public void testStreamCloseInFirstReadAhead() throws Exception {
+    describe("close a stream during prefetch, verify outcome is good");
+
+    AbfsClient client = getMockAbfsClient();
+    AbfsRestOperation successOp = getMockRestOp();
+
+    AbfsInputStream inputStream = getAbfsInputStream(client, getMethodName());
+    ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
+
+    final long initialInProgressBlocksDiscarded = bufferManager.getInProgressBlocksDiscarded();
+
+    // on first read, the op succeeds but the stream is closed, which
+    // means that the request should be considered a failure
+    doAnswer(invocation -> {
+      LOG.info("in read call with {}", inputStream);
+      inputStream.close();
+      return successOp;
+    }).doReturn(successOp)
+        .when(client)
+        .read(any(String.class), any(Long.class), any(byte[].class),
+            any(Integer.class), any(Integer.class), any(String.class),
+            any(String.class), any(TracingContext.class));
+
+    // kick these off before the read() to avoid various race conditions.
+    queueReadAheads(inputStream);
+
+    // AbfsInputStream Read would have waited for the read-ahead for the requested offset
+    // as we are testing from ReadAheadManager directly, sleep for a sec to
+    // get the read ahead threads to complete
+    waitForPrefetchCompletion();
+
+    // this triggers prefetching, which closes the stream while the read
+    // is queued. which causes the prefetch to not return.
+    // which triggers a blocking read, which will then fail.
+    intercept(PathIOException.class, FSExceptionMessages.STREAM_IS_CLOSED, () -> {
+      // should fail
+      int bytesRead = inputStream.read(new byte[ONE_KB]);
+      // diagnostics info if failure wasn't raised
+      return "read " + bytesRead + " bytes from " + inputStream;
+    });
+
+    Assertions.assertThat(bufferManager.getCompletedReadListCopy())
+        .filteredOn(rb -> rb.getStream() == inputStream)
+        .describedAs("list of completed reads")
+        .isEmpty();
+    IOStatisticsStore ios = inputStream.getIOStatistics();
+    assertThatStatisticCounter(ios, STREAM_READ_PREFETCH_BLOCKS_DISCARDED)
+        .describedAs("blocks discarded by %s", inputStream)
+        .isGreaterThan(0);
+
+    // at least one of the blocks was discarded in progress.
+    // this is guaranteed because the mockito callback must have been invoked
+    // by the prefetcher
+    Assertions.assertThat(bufferManager.getInProgressBlocksDiscarded())
+        .describedAs("in progress blocks discarded")
+        .isGreaterThan(initialInProgressBlocksDiscarded);
+  }
+
 }

Review Comment:
   nit: Add line at the end of the file.



##########
hadoop-tools/hadoop-azure/src/test/resources/log4j.properties:
##########
@@ -26,6 +26,8 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
 log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG
 log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE
 log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG
+log4j.logger.org.apache.hadoop.fs.azurebfs.services.ReadBufferManager=TRACE

Review Comment:
   Was this added for testing as this might add a lot of logging ?



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java:
##########
@@ -162,4 +182,120 @@ public void setAnyByteConsumed(boolean isAnyByteConsumed) {
     this.isAnyByteConsumed = isAnyByteConsumed;
   }
 
+  @Override
+  public String toString() {
+    return super.toString() +
+        "{ status=" + status +
+        ", offset=" + offset +
+        ", length=" + length +
+        ", requestedLength=" + requestedLength +
+        ", bufferindex=" + bufferindex +
+        ", timeStamp=" + timeStamp +
+        ", isFirstByteConsumed=" + isFirstByteConsumed +
+        ", isLastByteConsumed=" + isLastByteConsumed +
+        ", isAnyByteConsumed=" + isAnyByteConsumed +
+        ", errException=" + errException +
+        ", stream=" + (stream != null ? stream.getStreamID() : "none") +
+        ", stream closed=" + isStreamClosed() +
+        ", latch=" + latch +
+        '}';
+  }
+
+  /**
+   * Is the stream closed.
+   * @return stream closed status.
+   */
+  public boolean isStreamClosed() {
+    return stream != null && stream.isClosed();
+  }
+
+  /**
+   * IOStatistics of stream.
+   * @return the stream's IOStatisticsStore.
+   */
+  public IOStatisticsStore getStreamIOStatistics() {
+    return stream.getIOStatistics();
+  }
+
+  /**
+   * Start using the buffer.
+   * Sets the byte consumption flags as appriopriate, then
+   * updates the stream statistics with the use of this buffer.
+   * @param offset offset in buffer where copy began
+   * @param bytesCopied bytes copied.
+   */
+  void dataConsumedByStream(int offset, int bytesCopied) {
+    boolean isFirstUse = !isAnyByteConsumed;
+    setAnyByteConsumed(true);
+    if (offset == 0) {
+      setFirstByteConsumed(true);
+    }
+    if (offset + bytesCopied == getLength()) {
+      setLastByteConsumed(true);
+    }
+    IOStatisticsStore iostats = getStreamIOStatistics();
+    if (isFirstUse) {
+      // first use, update the use
+      iostats.incrementCounter(STREAM_READ_PREFETCH_BLOCKS_USED, 1);
+    }
+    // every use, update the count of bytes read
+    iostats.incrementCounter(STREAM_READ_PREFETCH_BYTES_USED, bytesCopied);
+  }
+
+  /**
+   * The (completed) buffer was evicted; update stream statistics
+   * as appropriate.
+   */
+  void evicted() {
+    IOStatisticsStore iostats = getStreamIOStatistics();
+    iostats.incrementCounter(STREAM_READ_PREFETCH_BLOCKS_EVICTED, 1);
+    if (!isAnyByteConsumed()) {
+      // nothing was read, so consider it discarded.
+      iostats.incrementCounter(STREAM_READ_PREFETCH_BLOCKS_DISCARDED, 1);
+      iostats.incrementCounter(STREAM_READ_PREFETCH_BYTES_DISCARDED, getLength());
+    }
+  }
+
+  /**
+   * The (completed) buffer was discarded; no data was read.
+   */
+  void discarded() {
+    if (getBufferindex() >= 0) {
+      IOStatisticsStore iostats = getStreamIOStatistics();
+      iostats.incrementCounter(STREAM_READ_PREFETCH_BLOCKS_DISCARDED, 1);
+      iostats.incrementCounter(STREAM_READ_PREFETCH_BYTES_DISCARDED, getLength());
+    }
+  }
+
+  /**
+   * Release the buffer: update fields as appropriate.
+   */
+  void releaseBuffer() {
+    setBuffer(null);
+    setBufferindex(-1);
+  }
+
+

Review Comment:
   nit: Extra line



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1331371867

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 51s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  16m  5s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 16s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  23m 32s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |  20m 53s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   4m 12s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   3m 13s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 44s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m 24s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 42s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 46s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 41s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 41s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |  22m 41s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 46s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |  20m 46s |  |  the patch passed  |
   | -1 :x: |  blanks  |   0m  0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/8/artifact/out/blanks-eol.txt) |  The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | -0 :warning: |  checkstyle  |   4m 11s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/8/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 5 new + 1 unchanged - 0 fixed = 6 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   3m  5s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   2m 41s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m  8s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 59s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m 29s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  18m 40s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 41s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 18s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 239m 51s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux 7ef67244d4ab 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / a261c9ac1cab9b2e1f95ef9ff59e1b02c07c5c4d |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/8/testReport/ |
   | Max. process+thread count | 3108 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-azure U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/8/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
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 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033433849


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -302,33 +325,33 @@ private synchronized boolean tryEvict() {
   }
 
   private boolean evict(final ReadBuffer buf) {
-    // As failed ReadBuffers (bufferIndx = -1) are saved in completedReadList,
-    // avoid adding it to freeList.
-    if (buf.getBufferindex() != -1) {
-      freeList.push(buf.getBufferindex());
-    }
-
-    completedReadList.remove(buf);
     buf.setTracingContext(null);
     if (LOGGER.isTraceEnabled()) {
       LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
           buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
     }
+    completedReadList.remove(buf);

Review Comment:
   Can you please highlight again why should we not remove the buffer from the completed list after it has been added to the free list ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1327598664

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 5 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  15m 56s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 18s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  23m 35s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |  20m 56s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   4m  7s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   3m  4s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 43s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   1m 58s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 58s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 42s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 46s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |  22m 46s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m  1s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |  21m  1s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 54s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/5/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 37 new + 1 unchanged - 0 fixed = 38 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   2m 58s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   2m 19s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m 13s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 31s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m  3s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  18m 36s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 45s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 22s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 237m 48s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux d6a7789fd8d3 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5415ba36518ee06540d75cc5248d989120e59fa8 |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/5/testReport/ |
   | Max. process+thread count | 3149 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-azure U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1307460345

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  41m 44s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 43s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   0m 48s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m  3s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m 22s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  23m 54s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 37s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 22s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/3/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 22 new + 1 unchanged - 0 fixed = 23 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   0m 35s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m  9s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  23m 15s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  7s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 40s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 104m 34s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux 1cf2fc9c924a 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c295fb241de7c160a1cba5b955f29f2f30376520 |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/3/testReport/ |
   | Max. process+thread count | 606 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033833635


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java:
##########
@@ -555,7 +579,7 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t
           throw new FileNotFoundException(ere.getMessage());
         }
       }
-      throw new IOException(ex);
+      throw ex;

Review Comment:
   i haven't
   ```
   class AzureBlobFileSystemException extends IOException 
   ```
   just removed one layer of wrapping so hive is less likely to lose the stack trace



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1019839338


##########
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:
   Lets not add buffer in completedList in the cases where we are going to add in freeList(due to byteRead == 0).
   
   - Got exception:
   
   ```
   2022-11-10 20:48:22,516 TRACE [ABFS-prefetch-7]: services.ReadBufferManager (ReadBufferManager.java:doneReading(591)) - ReadBufferWorker completed file /testfilefb393e327a88 for offset 4194304 bytes 0; org.apache.hadoop.fs.azurebfs.services.ReadBuffer@6777a743{ status=READING_IN_PROGRESS, offset=4194304, length=0, requestedLength=4194304, bufferindex=0, timeStamp=0, isFirstByteConsumed=false, isLastByteConsumed=false, isAnyByteConsumed=false, errException=null, stream=9d85521627a8, stream closed=false}
   2022-11-10 20:48:22,516 TRACE [ABFS-prefetch-7]: services.ReadBufferManager (ReadBufferManager.java:doneReading(633)) - Adding buffer to completed list org.apache.hadoop.fs.azurebfs.services.ReadBuffer@6777a743{ status=AVAILABLE, offset=4194304, length=0, requestedLength=4194304, bufferindex=0, timeStamp=86052487, isFirstByteConsumed=false, isLastByteConsumed=false, isAnyByteConsumed=false, errException=null, stream=9d85521627a8, stream closed=false}
   2022-11-10 20:48:22,516 DEBUG [ABFS-prefetch-7]: services.ReadBufferManager (ReadBufferManager.java:placeBufferOnFreeList(407)) - Returning buffer index 0 to free list for 'failed read'; owner org.apache.hadoop.fs.azurebfs.services.ReadBuffer@6777a743{ status=AVAILABLE, offset=4194304, length=0, requestedLength=4194304, bufferindex=0, timeStamp=86052487, isFirstByteConsumed=false, isLastByteConsumed=false, isAnyByteConsumed=false, errException=null, stream=9d85521627a8, stream closed=false}
   2022-11-10 20:48:22,517 TRACE [ABFS-prefetch-7]: services.ReadBufferWorker (ReadBufferWorker.java:run(95)) - Exception received: 
   org.apache.hadoop.fs.PathIOException: `/testfilefb393e327a88': Input/output error: Buffer index 0 found in buffer collection completedReadList
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferWorker.run(ReadBufferWorker.java:93)
   	at java.lang.Thread.run(Thread.java:750)
   Caused by: java.lang.IllegalStateException: Buffer index 0 found in buffer collection completedReadList
   	at org.apache.hadoop.util.Preconditions.checkState(Preconditions.java:298)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.verifyByteBufferNotInCollection(ReadBufferManager.java:471)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.verifyByteBufferNotInUse(ReadBufferManager.java:457)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.placeBufferOnFreeList(ReadBufferManager.java:413)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.doneReading(ReadBufferManager.java:646)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferWorker.run(ReadBufferWorker.java:87)
   	... 1 more
   ```
   Reason: https://github.com/steveloughran/hadoop/blob/azure/HADOOP-18521-buffer-manager/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L629 -> https://github.com/steveloughran/hadoop/blob/azure/HADOOP-18521-buffer-manager/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L641 -> https://github.com/steveloughran/hadoop/blob/1ee18eeb4922d18168bd1fc8ec4a5c75610447cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L413 -> https://github.com/steveloughran/hadoop/blob/1ee18eeb4922d18168bd1fc8ec4a5c75610447cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L457 -> exception.
   
   -  Double addition in freeList:
   -- Let in doneReading, prefetch-thread reaches https://github.com/steveloughran/hadoop/blob/1ee18eeb4922d18168bd1fc8ec4a5c75610447cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L415 and somehow contextSwitch happens and this thread doesn't get CPU for some time. Meanwhile, the buffer added in completedList gets picked up for eviction and its run and added in freeList. Now, the prefetchThread gets CPU again and runs https://github.com/steveloughran/hadoop/blob/1ee18eeb4922d18168bd1fc8ec4a5c75610447cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L415-L422 and adds in freeList.
   -- Wrote an experiment for the same: https://github.com/pranavsaxena-microsoft/hadoop/commit/7b6ac1558fa12c46217a296f197bf51a8b86c10b
   ```
   2022-11-10 22:22:57,147 TRACE [Thread-28]: services.ReadBufferManager (ReadBufferManager.java:lambda$init$0(147)) - INCONSISTENCY!! on index 8
   
   
   Exception in thread "Thread-16" java.lang.AssertionError: At index4194304
   	at org.junit.Assert.fail(Assert.java:89)
   	at org.junit.Assert.assertTrue(Assert.java:42)
   	at org.junit.Assert.assertFalse(Assert.java:65)
   	at org.apache.hadoop.fs.azurebfs.ITestPartialRead.lambda$purgeIssue$0(ITestPartialRead.java:154)
   	at java.lang.Thread.run(Thread.java:750)
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1329588076

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 55s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  16m 10s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  29m  8s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  25m 32s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |  22m 17s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   4m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 53s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 17s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 22s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  25m  4s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 49s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  27m 24s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |  27m 24s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  26m 57s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |  26m 57s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   5m  1s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/7/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 40 new + 1 unchanged - 0 fixed = 41 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   3m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   2m 49s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m  4s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   5m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  25m 44s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  18m 44s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 40s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 19s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 263m 49s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux 9fca8013fa2f 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / b224f33813755c2e5b2d73d64486ad4b0482feb6 |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/7/testReport/ |
   | Max. process+thread count | 3134 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-azure U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/7/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1020173444


##########
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:
   Agree with this because the current flow might lead to double addition in free list or inconsistency during addition.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1020487323


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -302,33 +325,33 @@ private synchronized boolean tryEvict() {
   }
 
   private boolean evict(final ReadBuffer buf) {
-    // As failed ReadBuffers (bufferIndx = -1) are saved in completedReadList,
-    // avoid adding it to freeList.
-    if (buf.getBufferindex() != -1) {
-      freeList.push(buf.getBufferindex());
-    }
-
-    completedReadList.remove(buf);
     buf.setTracingContext(null);
     if (LOGGER.isTraceEnabled()) {
       LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
           buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
     }
+    completedReadList.remove(buf);

Review Comment:
   ok. i'd positioned where they were so the invariant "not in use" held.
   maybe the validateReadManagerState() should go in at the end of the eviction



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1312096951

   thx for the comments; been away from this on the doc/replication side of this "issue" all week.  have been able to replicate the problem with avro parsing, though there it always fails at the parser; on csv/text records that's not guaranteed


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran closed pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by "steveloughran (via GitHub)" <gi...@apache.org>.
steveloughran closed pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers
URL: https://github.com/apache/hadoop/pull/5117


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1020491125


##########
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);

Review Comment:
   i've moved that check down. and with the move to an interface for abfs interaction, now in a position to create a test to simulate the conditions, including "available but empty" and "io error" on normal and prefetch reads



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1020171703


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -302,33 +325,33 @@ private synchronized boolean tryEvict() {
   }
 
   private boolean evict(final ReadBuffer buf) {
-    // As failed ReadBuffers (bufferIndx = -1) are saved in completedReadList,
-    // avoid adding it to freeList.
-    if (buf.getBufferindex() != -1) {
-      freeList.push(buf.getBufferindex());
-    }
-
-    completedReadList.remove(buf);
     buf.setTracingContext(null);
     if (LOGGER.isTraceEnabled()) {
       LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
           buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
     }
+    completedReadList.remove(buf);

Review Comment:
   Buffer should be removed from completed list after it has been added to the free list. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033415579


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferStreamOperations.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+
+/**
+ * Interface which is required for read buffer stream
+ * calls.
+ * Extracted from {@code AbfsInputStream} to make testing
+ * easier and to isolate what operations the read buffer
+ * makes of the streams using it.
+ */
+interface ReadBufferStreamOperations {
+
+  /**
+   * Read a block from the store.
+   * @param position position in file
+   * @param b destination buffer.
+   * @param offset offset in buffer
+   * @param length length of read
+   * @param tracingContext trace context
+   * @return count of bytes read.
+   * @throws IOException failure.
+   */
+  int readRemote(long position,
+      byte[] b,
+      int offset,
+      int length,
+      TracingContext tracingContext) throws IOException;
+
+  /**
+   * Is the stream closed?
+   * This must be thread safe as prefetch operations in
+   * different threads probe this before closure.
+   * @return true if the stream has been closed.
+   */
+  boolean isClosed();
+
+  String getStreamID();
+
+  IOStatisticsStore getIOStatistics();
+
+  /**
+   * Get the stream path as a string.
+   * @return path string.
+   */
+  String getPath();
+

Review Comment:
   Additional line, can be removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1313988205

   see also #5134 which is the "disable readahead" patch for 3.3.5


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033835902


##########
hadoop-tools/hadoop-azure/src/test/resources/log4j.properties:
##########
@@ -26,6 +26,8 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
 log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG
 log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE
 log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG
+log4j.logger.org.apache.hadoop.fs.azurebfs.services.ReadBufferManager=TRACE

Review Comment:
   it was...but we run the rest of the tests logging at debug, and i don't see the prefetcher being any chattier than the rest of the stack.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1344656088

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 37s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  17m 19s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  31m 23s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  24m 34s |  |  trunk passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  compile  |  21m  1s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   3m 40s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 35s |  |  trunk passed  |
   | -1 :x: |  javadoc  |   1m 12s | [/branch-javadoc-hadoop-common-project_hadoop-common-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/branch-javadoc-hadoop-common-project_hadoop-common-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-common in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 44s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   1m 27s |  |  trunk passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   4m  2s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 48s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  25m 54s |  |  the patch passed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04  |
   | +1 :green_heart: |  javac  |  25m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m 47s |  |  the patch passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  javac  |  21m 47s |  |  the patch passed  |
   | -1 :x: |  blanks  |   0m  0s | [/blanks-eol.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/blanks-eol.txt) |  The patch has 1 line(s) that end in blanks. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | -0 :warning: |  checkstyle  |   3m 57s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 5 new + 1 unchanged - 0 fixed = 6 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   2m 32s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   1m  4s | [/patch-javadoc-hadoop-common-project_hadoop-common-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/patch-javadoc-hadoop-common-project_hadoop-common-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-common in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | -1 :x: |  javadoc  |   0m 41s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  |  the patch passed with JDK Private Build-1.8.0_352-8u352-ga-1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   4m  8s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  22m 50s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  19m  5s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 28s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 59s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 241m 48s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux dc177bc99f6a 4.15.0-200-generic #211-Ubuntu SMP Thu Nov 24 18:16:04 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 940c9b629508e3e298c727f2e4da9625d11bf65a |
   | Default Java | Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.17+8-post-Ubuntu-1ubuntu220.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_352-8u352-ga-1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/testReport/ |
   | Max. process+thread count | 1263 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-azure U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/9/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1309518862

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 47s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  16m 25s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m  8s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  23m 24s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |  20m 48s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   4m 17s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   3m  2s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 41s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m 17s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 42s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  22m 13s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 44s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 41s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |  22m 41s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 46s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |  20m 46s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   3m 57s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/4/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 36 new + 1 unchanged - 0 fixed = 37 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   3m 15s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   2m 33s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m 17s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 37s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 48s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  18m 49s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 45s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 16s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 239m  1s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux 1cea54f46634 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 1ee18eeb4922d18168bd1fc8ec4a5c75610447cc |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/4/testReport/ |
   | Max. process+thread count | 1263 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-azure U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/4/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by "steveloughran (via GitHub)" <gi...@apache.org>.
steveloughran commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1702530486

   not working on this; 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033198624


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -454,31 +597,86 @@ 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);
-        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.
+      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, such as
+          // double invocation of this method with the same buffer.
+          String message =
+              String.format("Read completed from an operation not declared as in progress %s",
+                  buffer);
+          LOGGER.warn(message);
+          if (buffer.hasIndexedBuffer()) {
+            // release the buffer (which may raise an exception)
+            placeBufferOnFreeList("read not in progress", buffer);
+          }
+          // report the failure
+          throw new IllegalStateException(message);
         }
-        // completed list also contains FAILED read buffers
-        // for sending exception message to clients.
+
+        // should the read buffer be added to the completed list?
+        boolean addCompleted = true;
+        // flag to indicate buffer should be freed
+        boolean shouldFreeBuffer = false;
+        // and the reason (for logging)
+        String freeBufferReason = "";
+
         buffer.setStatus(result);
         buffer.setTimeStamp(currentTimeMillis());
-        completedReadList.add(buffer);
+        // did the read return any data?
+        if (result == ReadBufferStatus.AVAILABLE
+            && bytesActuallyRead > 0) {
+          // successful read of data; update buffer state.
+          buffer.setLength(bytesActuallyRead);
+        } else {
+          // 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.
+          // NOTE: checks for closed state may update this.
+          addCompleted = result == ReadBufferStatus.READ_FAILED;

Review Comment:
   Should we not add the buffer in completedList. Reason being it will always be removed from the list via oldFailedBuffers removal. Kindly suggest if there is a reason for adding it. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033415579


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferStreamOperations.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+
+/**
+ * Interface which is required for read buffer stream
+ * calls.
+ * Extracted from {@code AbfsInputStream} to make testing
+ * easier and to isolate what operations the read buffer
+ * makes of the streams using it.
+ */
+interface ReadBufferStreamOperations {
+
+  /**
+   * Read a block from the store.
+   * @param position position in file
+   * @param b destination buffer.
+   * @param offset offset in buffer
+   * @param length length of read
+   * @param tracingContext trace context
+   * @return count of bytes read.
+   * @throws IOException failure.
+   */
+  int readRemote(long position,
+      byte[] b,
+      int offset,
+      int length,
+      TracingContext tracingContext) throws IOException;
+
+  /**
+   * Is the stream closed?
+   * This must be thread safe as prefetch operations in
+   * different threads probe this before closure.
+   * @return true if the stream has been closed.
+   */
+  boolean isClosed();
+
+  String getStreamID();
+
+  IOStatisticsStore getIOStatistics();
+
+  /**
+   * Get the stream path as a string.
+   * @return path string.
+   */
+  String getPath();
+

Review Comment:
   nit: Additional line, can be removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1306078980

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   2m 50s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  39m 34s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 54s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 51s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   0m 43s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 53s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 46s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m 23s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 45s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 25s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/1/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 22 new + 3 unchanged - 0 fixed = 25 total (was 3)  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m 10s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m 39s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 15s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 43s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 100m 14s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux f5eb08c3f643 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 86a8177aedb9f311384bd28038ffdc464c188168 |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/1/testReport/ |
   | Max. process+thread count | 699 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] anmolanmol1234 commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
anmolanmol1234 commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1311623500

   The checkstyle errors needs fixing. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] snvijaya commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
snvijaya commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1313734822

   Hi @steveloughran, Wanted to get your opinion on below change as a possible replacement for this change :
   [https://github.com/apache/hadoop/pull/5133](url)
   
   A ReadBuffer with a valid Buffer assigned to it can be in certain states when stream is closed, and with the above change, I am trying to address it as below :
   1. Is in QueueReadAheadList - No change, the earlier purge takes care of it
   2. Is in CompletedList - No change again, the earlier purge takes care of it
   3. Is InProgressList but yet to make the network call - If stream is closed, stop network call and move the ReadBuffer as a failure into completed list
   4. Is InProgressList , just finished the network call - If stream is closed, network call was successful or not, move the ReadBuffer as a failure into completed list
   
   Now, when in state 3 or 4, the purge method might not pick it as it might have executed first. In that case, to prioritize these ReadBuffers for eviction, have added the check for stream is closed in the eviction code as well. 
   
   Please let me know if you see value in this fix and I could pursue further changes to incorporate validation code at queuing time and when getBlock finds a hit in completed list, and will also add related test code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] steveloughran commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
steveloughran commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1307225146

   going to have to rebase this.
   
   i also intend to have prefetch threads update the input streams with
   * gauge of active prefetches
   * amount of data/duration
   measuring how much data was discarded for a stream would be interesting too, but a bit fiddlier
   
   reader thread jsut gets iostats off the stream, casts to IOStatisticsStore then uses it as a duration tracker and gauge stats source; straightforward


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1020040801


##########
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:
   It would be better if we do placeBufferOnFreeList before trying to add in completedList, since, we would add it in freeList before completeListAddition (which can try itself to add in freeList on evict()) -> this will force never addition to freeList from completedList.



##########
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);

Review Comment:
   In case of READ_FAILED, it will make buffer.bufferIndex = -1. Now, when it goes to placeBufferOnFreeList, at https://github.com/steveloughran/hadoop/blob/1ee18eeb4922d18168bd1fc8ec4a5c75610447cc/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L406, it will be index = -1 -> it will break the flow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1327901575

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 56s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  1s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +0 :ok: |  mvndep  |  16m 56s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 26s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  23m 44s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |  21m  1s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   4m 17s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   3m 21s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m 37s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   2m  9s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 34s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 50s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 39s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  22m 36s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |  22m 36s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 55s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |  20m 55s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   4m  1s | [/results-checkstyle-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/6/artifact/out/results-checkstyle-root.txt) |  root: The patch generated 38 new + 1 unchanged - 0 fixed = 39 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   3m  7s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   2m 30s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   1m 59s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   4m 33s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 47s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  18m 42s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 43s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 24s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 239m 28s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux 8beba5d39166 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f72ad51e28eb52321c5be354db1f2cc08771a25d |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/6/testReport/ |
   | Max. process+thread count | 1263 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-azure U: . |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/6/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1033198624


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -454,31 +597,86 @@ 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);
-        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.
+      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, such as
+          // double invocation of this method with the same buffer.
+          String message =
+              String.format("Read completed from an operation not declared as in progress %s",
+                  buffer);
+          LOGGER.warn(message);
+          if (buffer.hasIndexedBuffer()) {
+            // release the buffer (which may raise an exception)
+            placeBufferOnFreeList("read not in progress", buffer);
+          }
+          // report the failure
+          throw new IllegalStateException(message);
         }
-        // completed list also contains FAILED read buffers
-        // for sending exception message to clients.
+
+        // should the read buffer be added to the completed list?
+        boolean addCompleted = true;
+        // flag to indicate buffer should be freed
+        boolean shouldFreeBuffer = false;
+        // and the reason (for logging)
+        String freeBufferReason = "";
+
         buffer.setStatus(result);
         buffer.setTimeStamp(currentTimeMillis());
-        completedReadList.add(buffer);
+        // did the read return any data?
+        if (result == ReadBufferStatus.AVAILABLE
+            && bytesActuallyRead > 0) {
+          // successful read of data; update buffer state.
+          buffer.setLength(bytesActuallyRead);
+        } else {
+          // 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.
+          // NOTE: checks for closed state may update this.
+          addCompleted = result == ReadBufferStatus.READ_FAILED;

Review Comment:
   Should we not add the buffer in completedList. Reason being it will always be removed from the list via oldFailedBuffers removal. Kindly suggest if there is a reason for adding it. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#issuecomment-1306093551

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 52s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  detsecrets  |   0m  0s |  |  detect-secrets was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 4 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  42m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  compile  |   0m 41s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 47s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 46s |  |  trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m 25s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m  2s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javac  |   0m 37s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 21s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/2/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 22 new + 3 unchanged - 0 fixed = 25 total (was 3)  |
   | +1 :green_heart: |  mvnsite  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m 11s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  23m 32s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  7s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 38s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 104m 39s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/5117 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets |
   | uname | Linux d1321bc3e889 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / ddd1a32528a819c8f05eb87624148eb24b324931 |
   | Default Java | Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/2/testReport/ |
   | Max. process+thread count | 594 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-5117/2/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] pranavsaxena-microsoft commented on a diff in pull request #5117: HADOOP-18521. ABFS ReadBufferManager must not reuse in-progress buffers

Posted by GitBox <gi...@apache.org>.
pranavsaxena-microsoft commented on code in PR #5117:
URL: https://github.com/apache/hadoop/pull/5117#discussion_r1017458506


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -534,17 +676,31 @@ void callTryEvict() {
     tryEvict();
   }
 
-
   /**
    * Purging the buffers associated with an {@link AbfsInputStream}
    * from {@link ReadBufferManager} when stream is closed.
+   * Before HADOOP-18521 this would purge in progress reads, which
+   * would return the active buffer to the free pool while it was
+   * still in use.
    * @param stream input stream.
    */
   public synchronized void purgeBuffersForStream(AbfsInputStream stream) {
     LOGGER.debug("Purging stale buffers for AbfsInputStream {} ", stream);
+
+    // remove from the queue
+    int before = readAheadQueue.size();
     readAheadQueue.removeIf(readBuffer -> readBuffer.getStream() == stream);
-    purgeList(stream, completedReadList);
-    purgeList(stream, inProgressList);
+    int readaheadPurged = readAheadQueue.size() - before;

Review Comment:
   By the thread reaches this line, maybe some more blocks would be added in readAheadQueue, this may bloat the metric. Also, before should >= readAheadQueue.size() (in case no additional blocks are ahead), this would result in negative addition.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -454,31 +588,39 @@ 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);
+    }
+    try {
+      synchronized (this) {
+        checkState(inProgressList.remove(buffer),
+            "Read completed from an operation not declared as in progress %s", buffer);
+        // If this buffer has already been purged during
+        // close of InputStream then we don't update the lists.
         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.
+          // there is no data, so it is immediately returned to the free list.
+          placeBufferOnFreeList("failed read", buffer);

Review Comment:
   This may result in IllegalStateException propogating to AbfsInputStream.
   
   This line will add the buffer into freeList, from which this index shall be taken by readBuffer b1.
   Now, after sometime, let this buffer from completedList needs to be evicted, it would come to https://github.com/steveloughran/hadoop/blob/azure/HADOOP-18521-buffer-manager/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L408, two things can happen:
   
   1. freeList still has this index: it will throw IllegalStateException
   2. freeList doesn't have: it will throw IllegalStateException from https://github.com/steveloughran/hadoop/blob/azure/HADOOP-18521-buffer-manager/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java#L411. 



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -454,31 +588,39 @@ 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);
+    }
+    try {
+      synchronized (this) {
+        checkState(inProgressList.remove(buffer),
+            "Read completed from an operation not declared as in progress %s", buffer);
+        // If this buffer has already been purged during
+        // close of InputStream then we don't update the lists.
         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.
+          // there is no data, so it is immediately returned to the free list.
+          placeBufferOnFreeList("failed read", buffer);

Review Comment:
   Made a suggestive-change, which prevents this:
   https://github.com/pranavsaxena-microsoft/hadoop/commit/0d09a0de501bdc928139263075f82feb064fd6bc



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java:
##########
@@ -454,31 +588,39 @@ 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);
+    }
+    try {
+      synchronized (this) {
+        checkState(inProgressList.remove(buffer),
+            "Read completed from an operation not declared as in progress %s", buffer);
+        // If this buffer has already been purged during
+        // close of InputStream then we don't update the lists.
         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.
+          // there is no data, so it is immediately returned to the free list.
+          placeBufferOnFreeList("failed read", buffer);

Review Comment:
   Test for the same: https://github.com/pranavsaxena-microsoft/hadoop/commit/18da3752f3f72a953cecba0525a01bfab6be89ee.
   
   In seperate run:
   ```
   java.lang.IllegalStateException: Buffer 14 returned to free buffer list by non-owner ReadBuffer{status=AVAILABLE, offset=4194304, length=0, requestedLength=4194304, bufferindex=14, timeStamp=46807492, isFirstByteConsumed=false, isLastByteConsumed=false, isAnyByteConsumed=false, errException=org.apache.hadoop.fs.PathIOException: `/testfilef6b6f93ac245': Input/output error: Buffer index 14 found in buffer collection completedReadList, stream=org.apache.hadoop.fs.azurebfs.services.AbfsInputStream@652e2419{counters=((stream_read_bytes_backwards_on_seek=0) (stream_read_operations=1) (remote_read_op=2) (stream_read_seek_backward_operations=0) (action_http_get_request.failures=0) (action_http_get_request=0) (bytes_read_buffer=0) (stream_read_bytes=0) (seek_in_buffer=0) (remote_bytes_read=0) (stream_read_seek_bytes_skipped=0) (stream_read_seek_operations=2) (read_ahead_bytes_read=0) (stream_read_seek_forward_operations=2));
   gauges=();
   minimums=((action_http_get_request.failures.min=-1) (action_http_get_request.min=-1));
   maximums=((action_http_get_request.max=-1) (action_http_get_request.failures.max=-1));
   means=((action_http_get_request.failures.mean=(samples=0, sum=0, mean=0.0000)) (action_http_get_request.mean=(samples=0, sum=0, mean=0.0000)));
   }AbfsInputStream@(1697522713){StreamStatistics{counters=((remote_bytes_read=0) (stream_read_seek_backward_operations=0) (remote_read_op=2) (stream_read_seek_forward_operations=2) (bytes_read_buffer=0) (seek_in_buffer=0) (stream_read_bytes=0) (stream_read_operations=1) (read_ahead_bytes_read=0) (stream_read_bytes_backwards_on_seek=0) (stream_read_seek_operations=2) (action_http_get_request.failures=0) (stream_read_seek_bytes_skipped=0) (action_http_get_request=0));
   gauges=();
   minimums=((action_http_get_request.min=-1) (action_http_get_request.failures.min=-1));
   maximums=((action_http_get_request.failures.max=-1) (action_http_get_request.max=-1));
   means=((action_http_get_request.mean=(samples=0, sum=0, mean=0.0000)) (action_http_get_request.failures.mean=(samples=0, sum=0, mean=0.0000)));
   }}}
   
   	at org.apache.hadoop.util.Preconditions.checkState(Preconditions.java:298)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.verifyReadOwnsBufferAtIndex(ReadBufferManager.java:430)
   	at org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.placeBufferOnFreeList(ReadBufferManager.java:411)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org