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 2020/10/07 22:43:28 UTC

[GitHub] [hadoop] snvijaya opened a new pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

snvijaya opened a new pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368


   Customers migrating from Gen1 to Gen2 often are observing different read patterns for the same workload. The optimization in Gen2 which reads only requested data size once detected as random read pattern is usually the cause of difference.
   
   In this PR, config option to force Gen2 driver to read always in buffer size even for random is being introduced. With this enabled the read pattern for the job will be similar to Gen1 and be full buffer sizes to backend.
   
   Have also accommodated the request to config control the readahead size to help cases such as small row groups in parquet files, where more data can be captured.
   
   These configs are not determined to be performant on the official parquet recommended row group sizes of 512-1024 MB and hence will not be enabled by default. 
   
   Tests are added to verify various combinations of config values. Also modified tests in file ITestAzureBlobFileSystemRandomRead which were using same file and hence test debugging was getting harder.


----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   Test results from accounts on East US 2 region:
   
   ### NON-HNS:
   	SharedKey:
   		[INFO] Results:
   		[INFO] 
   		[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 245
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 24
   ### HNS:
   	SharedKey:
   		[INFO] Results:
   		[INFO] 
   		[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 24
   	OAuth:
   		[INFO] Results:
   		[INFO] 
   		[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 140
   


----------------------------------------------------------------
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.

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 merged pull request #2368: HADOOP -17296. ABFS: Force reads to be always of buffer size.

Posted by GitBox <gi...@apache.org>.
steveloughran merged pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368


   


----------------------------------------------------------------
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.

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] mukund-thakur commented on a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r504018690



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
##########
@@ -448,15 +477,119 @@ public void testRandomReadPerformance() throws Exception {
             ratio < maxAcceptableRatio);
   }
 
+  /**
+   * With this test we should see a full buffer read being triggered in case
+   * alwaysReadBufferSize is on, else only the requested buffer size.
+   * Hence a seek done few bytes away from last read position will trigger
+   * a network read when alwaysReadBufferSize is off, whereas it will return
+   * from the internal buffer when it is on.
+   * Reading a full buffer size is the Gen1 behaviour.
+   * @throws Throwable
+   */
+  @Test
+  public void testAlwaysReadBufferSizeConfig() throws Throwable {
+    testAlwaysReadBufferSizeConfig(false);
+    testAlwaysReadBufferSizeConfig(true);
+  }
+
+  private void assertStatistics(AzureBlobFileSystem fs,

Review comment:
       Why creating a new method here if we are just doing a passthrough?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java
##########
@@ -447,4 +490,168 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception {
     checkEvictedStatus(inputStream, 0, true);
   }
 
+  /**
+   * Test readahead with different config settings for request request size and
+   * readAhead block size
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
+    // Set requestRequestSize = 4MB and readAheadBufferSize=8MB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FOUR_MB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    testReadAheadConfigs(FOUR_MB, TEST_READAHEAD_DEPTH_4, false, EIGHT_MB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(SIXTEEN_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    AbfsInputStream inputStream = testReadAheadConfigs(SIXTEEN_KB,
+        TEST_READAHEAD_DEPTH_2, true, SIXTEEN_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, SIXTEEN_KB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=48KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(SIXTEEN_KB, TEST_READAHEAD_DEPTH_2, true,
+        FORTY_EIGHT_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, FORTY_EIGHT_KB);
+
+    // Test for requestRequestSize =48KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(FORTY_EIGHT_KB, TEST_READAHEAD_DEPTH_2,
+        true,
+        SIXTEEN_KB);
+    testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
+  }
+
+
+  private void testReadAheads(AbfsInputStream inputStream,
+      int readRequestSize,
+      int readAheadRequestSize)
+      throws Exception {
+    if (readRequestSize > readAheadRequestSize) {
+      readAheadRequestSize = readRequestSize;
+    }
+
+    byte[] firstReadBuffer = new byte[readRequestSize];
+    byte[] secondReadBuffer = new byte[readAheadRequestSize];
+
+    // get the expected bytes to compare
+    byte[] expectedFirstReadAheadBufferContents = new byte[readRequestSize];
+    byte[] expectedSecondReadAheadBufferContents = new byte[readAheadRequestSize];
+    getExpectedBufferData(0, readRequestSize, expectedFirstReadAheadBufferContents);
+    getExpectedBufferData(readRequestSize, readAheadRequestSize,
+        expectedSecondReadAheadBufferContents);
+
+    assertTrue("Read should be of exact requested size",
+              inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
+    assertTrue("Data mismatch found in RAH1",
+        Arrays.equals(firstReadBuffer,
+            expectedFirstReadAheadBufferContents));
+
+
+    assertTrue("Read should be of exact requested size",
+        inputStream.read(secondReadBuffer, 0, readAheadRequestSize) == readAheadRequestSize);
+    assertTrue("Data mismatch found in RAH2",

Review comment:
       Better to use assert equals here inspite of assertTrue no?

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java
##########
@@ -447,4 +490,168 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception {
     checkEvictedStatus(inputStream, 0, true);
   }
 
+  /**
+   * Test readahead with different config settings for request request size and
+   * readAhead block size
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
+    // Set requestRequestSize = 4MB and readAheadBufferSize=8MB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FOUR_MB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    testReadAheadConfigs(FOUR_MB, TEST_READAHEAD_DEPTH_4, false, EIGHT_MB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(SIXTEEN_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    AbfsInputStream inputStream = testReadAheadConfigs(SIXTEEN_KB,
+        TEST_READAHEAD_DEPTH_2, true, SIXTEEN_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, SIXTEEN_KB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=48KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(SIXTEEN_KB, TEST_READAHEAD_DEPTH_2, true,
+        FORTY_EIGHT_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, FORTY_EIGHT_KB);
+
+    // Test for requestRequestSize =48KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(FORTY_EIGHT_KB, TEST_READAHEAD_DEPTH_2,
+        true,
+        SIXTEEN_KB);
+    testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
+  }
+
+
+  private void testReadAheads(AbfsInputStream inputStream,
+      int readRequestSize,
+      int readAheadRequestSize)
+      throws Exception {
+    if (readRequestSize > readAheadRequestSize) {
+      readAheadRequestSize = readRequestSize;
+    }
+
+    byte[] firstReadBuffer = new byte[readRequestSize];
+    byte[] secondReadBuffer = new byte[readAheadRequestSize];
+
+    // get the expected bytes to compare
+    byte[] expectedFirstReadAheadBufferContents = new byte[readRequestSize];
+    byte[] expectedSecondReadAheadBufferContents = new byte[readAheadRequestSize];
+    getExpectedBufferData(0, readRequestSize, expectedFirstReadAheadBufferContents);
+    getExpectedBufferData(readRequestSize, readAheadRequestSize,
+        expectedSecondReadAheadBufferContents);
+
+    assertTrue("Read should be of exact requested size",
+              inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
+    assertTrue("Data mismatch found in RAH1",
+        Arrays.equals(firstReadBuffer,
+            expectedFirstReadAheadBufferContents));
+
+
+    assertTrue("Read should be of exact requested size",
+        inputStream.read(secondReadBuffer, 0, readAheadRequestSize) == readAheadRequestSize);
+    assertTrue("Data mismatch found in RAH2",
+        Arrays.equals(secondReadBuffer,
+            expectedSecondReadAheadBufferContents));
+  }
+
+  public AbfsInputStream testReadAheadConfigs(int readRequestSize,
+      int readAheadQueueDepth,
+      boolean alwaysReadBufferSizeEnabled,
+      int readAheadBlockSize) throws Exception {
+    Configuration
+        config = new Configuration(
+        this.getRawConfiguration());
+    config.set("fs.azure.read.request.size", Integer.toString(readRequestSize));
+    config.set("fs.azure.readaheadqueue.depth",
+        Integer.toString(readAheadQueueDepth));
+    config.set("fs.azure.read.alwaysReadBufferSize",
+        Boolean.toString(alwaysReadBufferSizeEnabled));
+    config.set("fs.azure.read.readahead.blocksize",
+        Integer.toString(readAheadBlockSize));
+    if (readRequestSize > readAheadBlockSize) {
+      readAheadBlockSize = readRequestSize;
+    }
+
+    Path testPath = new Path(
+        "/testReadAheadConfigs");
+    final AzureBlobFileSystem fs = createTestFile(testPath,
+        ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE, config);
+    byte[] byteBuffer = new byte[ONE_MB];
+    AbfsInputStream inputStream = this.getAbfsStore(fs)
+        .openFileForRead(testPath, null);
+
+    assertEquals("Unexpected AbfsInputStream buffer size", readRequestSize,
+        inputStream.getBufferSize());
+    assertEquals("Unexpected ReadAhead queue depth", readAheadQueueDepth,
+        inputStream.getReadAheadQueueDepth());
+    assertEquals("Unexpected AlwaysReadBufferSize settings",
+        alwaysReadBufferSizeEnabled,
+        inputStream.shouldAlwaysReadBufferSize());
+    assertEquals("Unexpected readAhead block size", readAheadBlockSize,
+        ReadBufferManager.getBufferManager().getReadAheadBlockSize());
+
+    return inputStream;
+  }
+
+  private void getExpectedBufferData(int offset, int length, byte[] b) {
+    boolean startFillingIn = false;
+    int indexIntoBuffer = 0;
+    char character = 'a';
+
+    for (int i = 0; i < (offset + length); i++) {
+      if (i == offset) {
+        startFillingIn = true;
+      }
+
+      if ((startFillingIn) && (indexIntoBuffer < length)) {
+        b[indexIntoBuffer] = (byte) character;
+        indexIntoBuffer++;
+      }
+
+      character = (character == 'z') ? 'a' : (char) ((int) character + 1);
+    }
+  }
+
+  private AzureBlobFileSystem createTestFile(Path testFilePath, long testFileSize,

Review comment:
       See if you can reuse the data generation and new file creation code from ContractTestUtils.dataset() and ContractTestUtils.createFile. 

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java
##########
@@ -447,4 +490,168 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception {
     checkEvictedStatus(inputStream, 0, true);
   }
 
+  /**
+   * Test readahead with different config settings for request request size and
+   * readAhead block size
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
+    // Set requestRequestSize = 4MB and readAheadBufferSize=8MB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FOUR_MB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    testReadAheadConfigs(FOUR_MB, TEST_READAHEAD_DEPTH_4, false, EIGHT_MB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(SIXTEEN_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    AbfsInputStream inputStream = testReadAheadConfigs(SIXTEEN_KB,
+        TEST_READAHEAD_DEPTH_2, true, SIXTEEN_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, SIXTEEN_KB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=48KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(SIXTEEN_KB, TEST_READAHEAD_DEPTH_2, true,
+        FORTY_EIGHT_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, FORTY_EIGHT_KB);
+
+    // Test for requestRequestSize =48KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(FORTY_EIGHT_KB, TEST_READAHEAD_DEPTH_2,
+        true,
+        SIXTEEN_KB);
+    testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
+  }
+
+
+  private void testReadAheads(AbfsInputStream inputStream,
+      int readRequestSize,
+      int readAheadRequestSize)
+      throws Exception {
+    if (readRequestSize > readAheadRequestSize) {
+      readAheadRequestSize = readRequestSize;
+    }
+
+    byte[] firstReadBuffer = new byte[readRequestSize];
+    byte[] secondReadBuffer = new byte[readAheadRequestSize];
+
+    // get the expected bytes to compare
+    byte[] expectedFirstReadAheadBufferContents = new byte[readRequestSize];
+    byte[] expectedSecondReadAheadBufferContents = new byte[readAheadRequestSize];
+    getExpectedBufferData(0, readRequestSize, expectedFirstReadAheadBufferContents);
+    getExpectedBufferData(readRequestSize, readAheadRequestSize,
+        expectedSecondReadAheadBufferContents);
+
+    assertTrue("Read should be of exact requested size",
+              inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
+    assertTrue("Data mismatch found in RAH1",
+        Arrays.equals(firstReadBuffer,

Review comment:
       AssertJ has rich api's to tackle these kind of assertions. Try that     
   example : Assertions.assertThat(list)
               .hasSameElementsAs(list2) 




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   
   HNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141
   
   HNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 24
   
   NonHNS-SharedKey
   ========================
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   TestAbfsClientThrottlingAnalyzer.testManySuccessAndErrorsAndWaiting:171->fuzzyValidate:49 The actual value 20 is not within the expected range: [5.60, 8.40].
   [INFO] 
   [ERROR] Tests run: 89, Failures: 1, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 245
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 24
   
   NonHNS-OAuth
   ========================
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 249
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141
   
   Test failures reported in Yetus are not related to ABFS driver. Have created JIRA for tracking the failures.
   https://issues.apache.org/jira/browse/HADOOP-17325


----------------------------------------------------------------
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.

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 removed a comment on pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#issuecomment-705294313






----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r530769116



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -49,21 +51,37 @@
   private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
   private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
   private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
-  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+  private static ReadBufferManager bufferManager; // singleton, initialized in static initialization block
+  private static final ReentrantLock LOCK = new ReentrantLock();
 
-  static {
-    BUFFER_MANAGER = new ReadBufferManager();
-    BUFFER_MANAGER.init();
+  static ReadBufferManager getBufferManager() {

Review comment:
       Retaining the change as current change has no functional issues.




----------------------------------------------------------------
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.

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] mukund-thakur commented on a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r501750162



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -89,9 +91,24 @@ public AbfsInputStream(
     this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
     this.eTag = eTag;
     this.readAheadEnabled = true;
+    this.alwaysReadBufferSize
+        = abfsInputStreamContext.shouldReadBufferSizeAlways();
     this.cachedSasToken = new CachedSASToken(
         abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
     this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
+    readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
+    if (this.bufferSize > readAheadBlockSize) {

Review comment:
       Can this LOG/validation be moved to AbfsInputStreamContext.build() ?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -37,10 +39,10 @@
   private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
 
   private static final int NUM_BUFFERS = 16;
-  private static final int BLOCK_SIZE = 4 * 1024 * 1024;
   private static final int NUM_THREADS = 8;
   private static final int DEFAULT_THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
 
+  private static int blockSize = 4 * 1024 * 1024;

Review comment:
       nit: use 4 * ONE_MB consistent as everywhere else.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -89,9 +91,24 @@ public AbfsInputStream(
     this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
     this.eTag = eTag;
     this.readAheadEnabled = true;
+    this.alwaysReadBufferSize
+        = abfsInputStreamContext.shouldReadBufferSizeAlways();
     this.cachedSasToken = new CachedSASToken(
         abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
     this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
+    readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
+    if (this.bufferSize > readAheadBlockSize) {
+      LOG.debug(
+          "fs.azure.read.request.size[={}] is configured for higher size than "
+              + "fs.azure.read.readahead.blocksize[={}]. Auto-align "
+              + "readAhead block size to be same as readRequestSize.",
+          bufferSize, readAheadBlockSize);
+      readAheadBlockSize = this.bufferSize;
+    }
+
+    // Propagate the config values to ReadBufferManager so that the first instance
+    // to initialize it get can set the readAheadBlockSize

Review comment:
       nit: typo? initialize it get can set

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -178,11 +195,15 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
         buffer = new byte[bufferSize];
       }
 
-      // Enable readAhead when reading sequentially
-      if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
+      if (alwaysReadBufferSize) {
         bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);

Review comment:
       JIRA and PR description says we are trying to read till bufferSize always rather than just the requested length but as per this line we are enabling the buffer manager readahead as well which is bypassed in random read in gen2 as per line 205 below. PS: I have never seen gen1 code though.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -49,21 +51,37 @@
   private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
   private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
   private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
-  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+  private static ReadBufferManager bufferManager; // singleton, initialized in static initialization block
+  private static final ReentrantLock LOCK = new ReentrantLock();
 
-  static {
-    BUFFER_MANAGER = new ReadBufferManager();
-    BUFFER_MANAGER.init();
+  static ReadBufferManager getBufferManager() {

Review comment:
       Why all these changes ? Why not just initilize the blockSize in init() ? 

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
##########
@@ -74,6 +74,9 @@
   public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = "";
 
   public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
+  public static final boolean DEFAULT_ALWAYS_READ_BUFFER_SIZE = false;

Review comment:
       I think putting these config together with DEFAULT_READ_BUFFER_SIZE would make code more readable. Also use 4 * ONE_MB as used above.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -464,4 +483,53 @@ int getCompletedReadListSize() {
   void callTryEvict() {
     tryEvict();
   }
+
+  @VisibleForTesting
+  void testResetReadBufferManager() {

Review comment:
       please add some reasoning/docs around these changes. Thanks.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       Would like to understand the reasoning behind this. 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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r503689712



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -89,9 +91,24 @@ public AbfsInputStream(
     this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
     this.eTag = eTag;
     this.readAheadEnabled = true;
+    this.alwaysReadBufferSize
+        = abfsInputStreamContext.shouldReadBufferSizeAlways();
     this.cachedSasToken = new CachedSASToken(
         abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
     this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
+    readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
+    if (this.bufferSize > readAheadBlockSize) {

Review comment:
       Done




----------------------------------------------------------------
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.

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] mukund-thakur commented on a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r507647774



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       What do you mean by gaps/holes in the readAhead range done here?
   Have you done any experiments on this readAheadBlockSize config? If so, please share. 




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   Test results from accounts in East US 2 regions:
   
   ### NON-HNS:
   
   	SharedKey:
   		[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 245
   		[ERROR] Errors: 
   		[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testComplexDirActions:153->AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir:191 » IllegalState
   		[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testListStatusIteratorFile:366 » IllegalState
   		[ERROR] Tests run: 208, Failures: 0, Errors: 2, Skipped: 24
   			ERROR] testComplexDirActions(org.apache.hadoop.fs.azurebfs.contract.ITestAbfsFileSystemContractGetFileStatus)  Time elapsed: 31.122 s  <<< ERROR!
   			java.lang.IllegalStateException: No more items in iterator
   				at com.google.common.base.Preconditions.checkState(Preconditions.java:507)
   				at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2232)
   				at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2205)
   				at org.apache.hadoop.fs.contract.ContractTestUtils.iteratorToListThroughNextCallsAlone(ContractTestUtils.java:1494)
   				at org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir(AbstractContractGetFileStatusTest.java:191)
   				at org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest.testComplexDirActions(AbstractContractGetFileStatusTest.java:153)
   				at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   				at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   				at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   				at java.lang.reflect.Method.invoke(Method.java:498)
   				at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   				at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   				at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   				at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   				at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   				at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   				at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
   				at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
   				at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
   				at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   				at java.lang.Thread.run(Thread.java:748)
   
   			[ERROR] testListStatusIteratorFile(org.apache.hadoop.fs.azurebfs.contract.ITestAbfsFileSystemContractGetFileStatus)  Time elapsed: 3.038 s  <<< ERROR!
   			java.lang.IllegalStateException: No more items in iterator
   				at com.google.common.base.Preconditions.checkState(Preconditions.java:507)
   				at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2232)
   				at org.apache.hadoop.fs.FileSystem$DirListingIterator.next(FileSystem.java:2205)
   				at org.apache.hadoop.fs.contract.ContractTestUtils.iteratorToListThroughNextCallsAlone(ContractTestUtils.java:1494)
   				at org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest.testListStatusIteratorFile(AbstractContractGetFileStatusTest.java:366)
   				at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   				at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   				at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   				at java.lang.reflect.Method.invoke(Method.java:498)
   				at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   				at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   				at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   				at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   				at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   				at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   				at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
   				at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
   				at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
   				at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   				at java.lang.Thread.run(Thread.java:748)
   
   	OAuth:
   		INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
   		[ERROR] Errors: 
   		[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testComplexDirActions:153->AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir:191 » IllegalState
   		[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testListStatusIteratorFile:366 » IllegalState
   		[ERROR] Tests run: 208, Failures: 0, Errors: 2, Skipped: 24
   ### HNS:
   
   	SharedKey:
   		[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 24
   		ERROR] Errors:
   		[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testComplexDirActions:153->AbstractContractGetFileStatusTest.checkListStatusIteratorComplexDir:191 » IllegalState
   		[ERROR]   ITestAbfsFileSystemContractGetFileStatus>AbstractContractGetFileStatusTest.testListStatusIteratorFile:366 » IllegalState
   		[INFO]
   		[ERROR] Tests run: 208, Failures: 0, Errors: 2, Skipped: 24
   
   
   	OAuth: * In this run the ListIterator test got skipped due to isSecure config being off
   		[INFO] Tests run: 88, Failures: 0, Errors: 0, Skipped: 0
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
   		[WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141
   
   2 failures seen in most combination call fs.listStatusIterator() APIs. These APIs are not supported by ABFS FS currently. Need to check if a recent change to test introduced this call.


----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r530768704



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       
   Read buffer size config being available to be modified, fixed read ahead buffer size and issuing read aheads by buffer size is the current prod behaviour and will function as the picture attached. This will need fixing.
   And as for deprecating read buffer size config and only use the new read ahead buffer size config. The config has been available since GA, and hence deprecating it would not be feasible. (Also for clients who are disabling readAheads to use readahead buffer size for reads might be confusing too).
   As for the number of different configs present for read, 1 and 2 configs already were present while this PR is introducing 3 and 4. So total of 4 configs.
   1.	fs.azure.read.request.size
   2.	fs.azure.readaheadqueue.depth
   3.	fs.azure.read.alwaysReadBufferSize => For Gen1 migrating customers
   4.	fs.azure.read.readahead.blocksize => Was one that needed fixing long back as there is no validation on 4 MB being the right size for all workloads. Just the way read buffer size can be modified.
    
   All these changes are being added based on various customer issues and experiences that we are dealing with. Instead of spending our time in providing patches that can enable them to test various combinations, having these options over a config for their testing saves our dev time to improve the service. As you can see in the PR, the defaults introduced by these configs will retain the current prod behavior.
   ![ReadAheadBufferConfigExplanation](https://user-images.githubusercontent.com/13376553/100308566-b87e8580-2fce-11eb-8643-748848a69dbd.png)
   




----------------------------------------------------------------
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.

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] mukund-thakur commented on pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#issuecomment-705634731


   Reviewed the production code. Will review the test code as well once I get some time. 


----------------------------------------------------------------
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.

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 removed a comment on pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#issuecomment-714616055


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  8s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  30m 10s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 46s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 25s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   1m 28s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/5/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 57s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.azure.TestNativeAzureFileSystemMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemConcurrency |
   |   | hadoop.fs.azure.TestWasbFsck |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemOperationsMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemFileNameCheck |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemContractMocked |
   |   | hadoop.fs.azure.TestOutOfBandAzureBlobOperations |
   |   | hadoop.fs.azure.TestBlobMetadata |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint |
   | uname | Linux ec475f03c972 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 7435604a91a |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/5/testReport/ |
   | Max. process+thread count | 420 (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-2368/5/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.1.3 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 57s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 56s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 42s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 36s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  0s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 59s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 47s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  5s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/1/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 32s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  80m 49s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Write to static field org.apache.hadoop.fs.azurebfs.services.AbfsInputStream.readAheadBlockSize from instance method new org.apache.hadoop.fs.azurebfs.services.AbfsInputStream(AbfsClient, FileSystem$Statistics, String, long, AbfsInputStreamContext, String)  At AbfsInputStream.java:from instance method new org.apache.hadoop.fs.azurebfs.services.AbfsInputStream(AbfsClient, FileSystem$Statistics, String, long, AbfsInputStreamContext, String)  At AbfsInputStream.java:[line 99] |
   |  |  Write to static field org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.blockSize from instance method org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.testResetReadBufferManager(int, int)  At ReadBufferManager.java:from instance method org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.testResetReadBufferManager(int, int)  At ReadBufferManager.java:[line 513] |
   |  |  Write to static field org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.thresholdAgeMilliseconds from instance method org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.testResetReadBufferManager(int, int)  At ReadBufferManager.java:from instance method org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.testResetReadBufferManager(int, int)  At ReadBufferManager.java:[line 514] |
   |  |  Write to static field org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.bufferManager from instance method org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.testResetReadBufferManager()  At ReadBufferManager.java:from instance method org.apache.hadoop.fs.azurebfs.services.ReadBufferManager.testResetReadBufferManager()  At ReadBufferManager.java:[line 507] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 44a045e9418c 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / df4006eb813 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/1/testReport/ |
   | Max. process+thread count | 414 (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-2368/1/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r531628039



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
##########
@@ -99,12 +115,14 @@ public void testBasicRead() throws Exception {
   public void testRandomRead() throws Exception {
     Assume.assumeFalse("This test does not support namespace enabled account",
             this.getFileSystem().getIsNamespaceEnabled());
-    assumeHugeFileExists();
+    Path testPath = new Path(TEST_FILE_PREFIX + "_testRandomRead");

Review comment:
       FWIW I use a Junit rule to get the method name, then you can hava a `path()` method which dynamically creates the unique path, including when you use parameterized tests.




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  8s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  30m 10s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 46s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 25s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   1m 28s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/5/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 57s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.azure.TestNativeAzureFileSystemMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemConcurrency |
   |   | hadoop.fs.azure.TestWasbFsck |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemOperationsMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemFileNameCheck |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemContractMocked |
   |   | hadoop.fs.azure.TestOutOfBandAzureBlobOperations |
   |   | hadoop.fs.azure.TestBlobMetadata |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint |
   | uname | Linux ec475f03c972 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 7435604a91a |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/5/testReport/ |
   | Max. process+thread count | 420 (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-2368/5/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.1.3 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   @mukund-thakur  - Will appreciate if you can confirm your review status on this PR. Have provided updates for your comments.


----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m  0s |  |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  4s |  |  https://github.com/apache/hadoop/pull/2368 does not apply to trunk. Rebase required? Wrong Branch? See https://wiki.apache.org/hadoop/HowToContribute for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/4/console |
   | versions | git=2.17.1 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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] mukund-thakur commented on a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r501750162



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -89,9 +91,24 @@ public AbfsInputStream(
     this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
     this.eTag = eTag;
     this.readAheadEnabled = true;
+    this.alwaysReadBufferSize
+        = abfsInputStreamContext.shouldReadBufferSizeAlways();
     this.cachedSasToken = new CachedSASToken(
         abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
     this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
+    readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
+    if (this.bufferSize > readAheadBlockSize) {

Review comment:
       Can this LOG/validation be moved to AbfsInputStreamContext.build() ?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -37,10 +39,10 @@
   private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
 
   private static final int NUM_BUFFERS = 16;
-  private static final int BLOCK_SIZE = 4 * 1024 * 1024;
   private static final int NUM_THREADS = 8;
   private static final int DEFAULT_THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
 
+  private static int blockSize = 4 * 1024 * 1024;

Review comment:
       nit: use 4 * ONE_MB consistent as everywhere else.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -89,9 +91,24 @@ public AbfsInputStream(
     this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
     this.eTag = eTag;
     this.readAheadEnabled = true;
+    this.alwaysReadBufferSize
+        = abfsInputStreamContext.shouldReadBufferSizeAlways();
     this.cachedSasToken = new CachedSASToken(
         abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
     this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
+    readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
+    if (this.bufferSize > readAheadBlockSize) {
+      LOG.debug(
+          "fs.azure.read.request.size[={}] is configured for higher size than "
+              + "fs.azure.read.readahead.blocksize[={}]. Auto-align "
+              + "readAhead block size to be same as readRequestSize.",
+          bufferSize, readAheadBlockSize);
+      readAheadBlockSize = this.bufferSize;
+    }
+
+    // Propagate the config values to ReadBufferManager so that the first instance
+    // to initialize it get can set the readAheadBlockSize

Review comment:
       nit: typo? initialize it get can set

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -178,11 +195,15 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
         buffer = new byte[bufferSize];
       }
 
-      // Enable readAhead when reading sequentially
-      if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
+      if (alwaysReadBufferSize) {
         bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);

Review comment:
       JIRA and PR description says we are trying to read till bufferSize always rather than just the requested length but as per this line we are enabling the buffer manager readahead as well which is bypassed in random read in gen2 as per line 205 below. PS: I have never seen gen1 code though.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -49,21 +51,37 @@
   private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
   private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
   private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
-  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+  private static ReadBufferManager bufferManager; // singleton, initialized in static initialization block
+  private static final ReentrantLock LOCK = new ReentrantLock();
 
-  static {
-    BUFFER_MANAGER = new ReadBufferManager();
-    BUFFER_MANAGER.init();
+  static ReadBufferManager getBufferManager() {

Review comment:
       Why all these changes ? Why not just initilize the blockSize in init() ? 

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
##########
@@ -74,6 +74,9 @@
   public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = "";
 
   public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
+  public static final boolean DEFAULT_ALWAYS_READ_BUFFER_SIZE = false;

Review comment:
       I think putting these config together with DEFAULT_READ_BUFFER_SIZE would make code more readable. Also use 4 * ONE_MB as used above.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -464,4 +483,53 @@ int getCompletedReadListSize() {
   void callTryEvict() {
     tryEvict();
   }
+
+  @VisibleForTesting
+  void testResetReadBufferManager() {

Review comment:
       please add some reasoning/docs around these changes. Thanks.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       Would like to understand the reasoning behind this. 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.

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] mukund-thakur commented on pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#issuecomment-705634731


   Reviewed the production code. Will review the test code as well once I get some time. 


----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r510800015



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       If the buffer gets overwritten by config to 16MB, the readAhead buffer size will still remain to be 4MB as it was a code static. The loop done will start issuing readAheads in 16 MB buffer sizes, the request to readAhead will be:
   offset=0, Length=16MB
   offset=16MB, Length=32MB
   
   But the readAhead buffer size is stuck at 4 MB. so it will read only:
   offset=0 Length=4MB
   offset=16MB Length=4MB
   
   Gap being at 4MB to 16MB here.
   
   This bug is getting fixed. Tests for all possible combinations here has been added to the tests of this PR.




----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r503693637



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       Couple of things here:
   1. The earlier code allowed bufferSize to be configurable whereas ReadAhead buffer size was fixed. And each time loop is done, read issued was always for bufferSize which can lead to gaps/holes in the readAhead range done.
   2. There is no validation for 4MB as a fixed size for readAhead is optimal for all sequential reads. Having a higher readAhead range for apps like DFSIO which are guaranteed sequential and doing higher readAhead ranges in background can be performant. 
   In this PR, the bug in point 1 is fixed and also a provision to configure readAhead buffer size is provided.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -37,10 +39,10 @@
   private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
 
   private static final int NUM_BUFFERS = 16;
-  private static final int BLOCK_SIZE = 4 * 1024 * 1024;
   private static final int NUM_THREADS = 8;
   private static final int DEFAULT_THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
 
+  private static int blockSize = 4 * 1024 * 1024;

Review comment:
       Done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -464,4 +483,53 @@ int getCompletedReadListSize() {
   void callTryEvict() {
     tryEvict();
   }
+
+  @VisibleForTesting
+  void testResetReadBufferManager() {

Review comment:
       Done




----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r504647658



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -49,21 +51,37 @@
   private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
   private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
   private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
-  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+  private static ReadBufferManager bufferManager; // singleton, initialized in static initialization block
+  private static final ReentrantLock LOCK = new ReentrantLock();
 
-  static {
-    BUFFER_MANAGER = new ReadBufferManager();
-    BUFFER_MANAGER.init();
+  static ReadBufferManager getBufferManager() {

Review comment:
       For singleton classes its a common practice to lock around the new instance creation within the getInstance() method. Also, didnt want to make any changes to init method.




----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r503691466



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -178,11 +195,15 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
         buffer = new byte[bufferSize];
       }
 
-      // Enable readAhead when reading sequentially
-      if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
+      if (alwaysReadBufferSize) {
         bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);

Review comment:
       AlwaysReadBufferSize helped the IO pattern to match the Gen1 run. But to be performant readAhead had to be enabled. For the customer scenario explained in the JIRA , for the small row groups for an overall small parquet file size, reading whole buffer size along with readAhead bought good performance.




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   this will need to be rebased now that #2369


----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r504685349



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
##########
@@ -448,15 +477,119 @@ public void testRandomReadPerformance() throws Exception {
             ratio < maxAcceptableRatio);
   }
 
+  /**
+   * With this test we should see a full buffer read being triggered in case
+   * alwaysReadBufferSize is on, else only the requested buffer size.
+   * Hence a seek done few bytes away from last read position will trigger
+   * a network read when alwaysReadBufferSize is off, whereas it will return
+   * from the internal buffer when it is on.
+   * Reading a full buffer size is the Gen1 behaviour.
+   * @throws Throwable
+   */
+  @Test
+  public void testAlwaysReadBufferSizeConfig() throws Throwable {
+    testAlwaysReadBufferSizeConfig(false);
+    testAlwaysReadBufferSizeConfig(true);
+  }
+
+  private void assertStatistics(AzureBlobFileSystem fs,

Review comment:
       Redundant method 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  31m  5s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  markdownlint  |   0m  0s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 20s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 23s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 14s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 57s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 55s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 12s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 24s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 15s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  16m 34s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 59s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 25s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 29s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 112m 53s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint |
   | uname | Linux fd8c2773496c 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 235947e2822 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/7/testReport/ |
   | Max. process+thread count | 622 (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-2368/7/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   ### HNS-OAuth
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 446, Failures: 0, Errors: 0, Skipped: 66
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 141
   ### NonHNS-OAuth
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 249
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 206, Failures: 0, Errors: 0, Skipped: 141
   ### HNS-SharedKey
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [ERROR] Errors: 
   [ERROR]   ITestAzureBlobFileSystemDelegationSAS.testCheckAccess:94 » AbfsRestOperation O...
   [ERROR]   ITestAzureBlobFileSystemDelegationSAS.testRootPath:343 » AbfsRestOperation Ope...
   [INFO] 
   [ERROR] Tests run: 446, Failures: 0, Errors: 2, Skipped: 24
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractRename>AbstractContractRenameTest.testRenameFileOverExistingFile:131->Assert.fail:88 expected rename(abfs://gen2contracttest@snvijayacontracttest.dfs.core.windows.net/fork-6/test/source-256.txt, abfs://gen2contracttest@snvijayacontracttest.dfs.core.windows.net/fork-6/test/dest-512.txt) to be rejected with exception, but got false
   [INFO] 
   [ERROR] Tests run: 196, Failures: 1, Errors: 0, Skipped: 24
   ### NonHNS-SharedKey
   [INFO] Results:
   [INFO] 
   [INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   [INFO] Results:
   [INFO] 
   [WARNING] Tests run: 424, Failures: 0, Errors: 0, Skipped: 242
   [INFO] Results:
   [INFO] 
   [ERROR] Failures: 
   [ERROR]   ITestAbfsFileSystemContractRename>AbstractContractRenameTest.testRenameFileOverExistingFile:131->Assert.fail:88 expected rename(abfs://gen2contracttest@snvijayanonhnstest.dfs.core.windows.net/fork-7/test/source-256.txt, abfs://gen2contracttest@snvijayanonhnstest.dfs.core.windows.net/fork-7/test/dest-512.txt) to be rejected with exception, but got false
   [INFO] 
   [ERROR] Tests run: 208, Failures: 1, Errors: 0, Skipped: 24
   
   Rename Test failure is tracked in JIRA: https://issues.apache.org/jira/browse/HADOOP-17396
   SAS Test failure is tracked in JIRA: https://issues.apache.org/jira/browse/HADOOP-17397


----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   Hi @mukund-thakur, Kindly request your review, so that I can freeze on comments from your side.


----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  33m 22s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  30m 16s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 14s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 57s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 14s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 28s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 105m 23s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux ec39d768a392 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / b3786d6c3cc |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/3/testReport/ |
   | Max. process+thread count | 437 (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-2368/3/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.1.3 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: HADOOP-17296. ABFS: Force reads to be always of buffer size.

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


   ok, merged into trunk. For 3.3.x can you do the backport, retest and put up a new PR for me to merge in there? That branch doesn't have guava under the thirdparty packaging, so a safe cherrypick isn't enough to prove it will compile/run. Don't worry about reordering the import references, just edit out the o.a.h.thirdparty prefixes and leave as is.


----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r503689799



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -89,9 +91,24 @@ public AbfsInputStream(
     this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
     this.eTag = eTag;
     this.readAheadEnabled = true;
+    this.alwaysReadBufferSize
+        = abfsInputStreamContext.shouldReadBufferSizeAlways();
     this.cachedSasToken = new CachedSASToken(
         abfsInputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
     this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
+    readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
+    if (this.bufferSize > readAheadBlockSize) {
+      LOG.debug(
+          "fs.azure.read.request.size[={}] is configured for higher size than "
+              + "fs.azure.read.readahead.blocksize[={}]. Auto-align "
+              + "readAhead block size to be same as readRequestSize.",
+          bufferSize, readAheadBlockSize);
+      readAheadBlockSize = this.bufferSize;
+    }
+
+    // Propagate the config values to ReadBufferManager so that the first instance
+    // to initialize it get can set the readAheadBlockSize

Review comment:
       Fixed

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
##########
@@ -74,6 +74,9 @@
   public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = "";
 
   public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
+  public static final boolean DEFAULT_ALWAYS_READ_BUFFER_SIZE = false;

Review comment:
       Done




----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r530768877



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       Have synced with @mukund-thakur over mail thread further to clear the understanding.




----------------------------------------------------------------
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.

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 removed a comment on pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#issuecomment-718737087


   :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: |  markdownlint  |   0m  1s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |   2m  7s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | +1 :green_heart: |  compile  |   2m 52s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 10s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 54s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 53s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 13s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 16s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   1m 31s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  51m 12s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.azure.TestNativeAzureFileSystemOperationsMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemContractMocked |
   |   | hadoop.fs.azure.TestOutOfBandAzureBlobOperations |
   |   | hadoop.fs.azure.TestBlobMetadata |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemFileNameCheck |
   |   | hadoop.fs.azure.TestWasbFsck |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemConcurrency |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint |
   | uname | Linux fc250b1997db 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f17e067d527 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/testReport/ |
   | Max. process+thread count | 455 (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-2368/6/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.1.3 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   @mukund-thakur   - Have addressed the review comments. Please have a look.


----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  29m 27s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  3s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  1s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 29s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  71m 32s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux e8b21cb8b3f6 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 52db86b0bb4 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/2/testReport/ |
   | Max. process+thread count | 424 (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-2368/2/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   Re-ran the oAuth tests with the latest changes:
   	### OAuth:
   		[INFO] Results:
   		[INFO] 
   		[INFO] Tests run: 89, Failures: 0, Errors: 0, Skipped: 0
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 458, Failures: 0, Errors: 0, Skipped: 66
   		[INFO] Results:
   		[INFO] 
   		[WARNING] Tests run: 207, Failures: 0, Errors: 0, Skipped: 140


----------------------------------------------------------------
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.

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 change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r531625370



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
##########
@@ -49,21 +51,37 @@
   private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
   private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
   private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
-  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+  private static ReadBufferManager bufferManager; // singleton, initialized in static initialization block
+  private static final ReentrantLock LOCK = new ReentrantLock();
 
-  static {
-    BUFFER_MANAGER = new ReadBufferManager();
-    BUFFER_MANAGER.init();
+  static ReadBufferManager getBufferManager() {

Review comment:
       FWIW, synchronize on the Class object is a standard pattern here, but the lock is fine, just a bit of extra coding




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  29m 27s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  3s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m  1s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 29s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  71m 32s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux e8b21cb8b3f6 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 52db86b0bb4 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/2/testReport/ |
   | Max. process+thread count | 424 (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-2368/2/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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] mukund-thakur commented on a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r522187925



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
##########
@@ -223,16 +244,19 @@ private int readInternal(final long position, final byte[] b, final int offset,
 
       // queue read-aheads
       int numReadAheads = this.readAheadQueueDepth;
-      long nextSize;
       long nextOffset = position;
+      // First read to queue needs to be of readBufferSize and later

Review comment:
       I don't think there is any bug in the current production code as such. As far as I understand the code the change is introduced becuase new config is introduced. 
   Now my question is why not use readAheadBlockSize for the first call as well? The calls would be like 
   offset=0 Length=4MB 
   offset=4MB Length=4MB
   
   Sorry to say this but honestly speaking, introducing so many configs is making the code complex and confusing.  




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   Test failures reported in Yetus are not related to ABFS driver. Have created JIRA for tracking the failures.
   https://issues.apache.org/jira/browse/HADOOP-17325
   


----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r504650837



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java
##########
@@ -447,4 +490,168 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception {
     checkEvictedStatus(inputStream, 0, true);
   }
 
+  /**
+   * Test readahead with different config settings for request request size and
+   * readAhead block size
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
+    // Set requestRequestSize = 4MB and readAheadBufferSize=8MB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FOUR_MB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    testReadAheadConfigs(FOUR_MB, TEST_READAHEAD_DEPTH_4, false, EIGHT_MB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(SIXTEEN_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    AbfsInputStream inputStream = testReadAheadConfigs(SIXTEEN_KB,
+        TEST_READAHEAD_DEPTH_2, true, SIXTEEN_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, SIXTEEN_KB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=48KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(SIXTEEN_KB, TEST_READAHEAD_DEPTH_2, true,
+        FORTY_EIGHT_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, FORTY_EIGHT_KB);
+
+    // Test for requestRequestSize =48KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(FORTY_EIGHT_KB, TEST_READAHEAD_DEPTH_2,
+        true,
+        SIXTEEN_KB);
+    testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
+  }
+
+
+  private void testReadAheads(AbfsInputStream inputStream,
+      int readRequestSize,
+      int readAheadRequestSize)
+      throws Exception {
+    if (readRequestSize > readAheadRequestSize) {
+      readAheadRequestSize = readRequestSize;
+    }
+
+    byte[] firstReadBuffer = new byte[readRequestSize];
+    byte[] secondReadBuffer = new byte[readAheadRequestSize];
+
+    // get the expected bytes to compare
+    byte[] expectedFirstReadAheadBufferContents = new byte[readRequestSize];
+    byte[] expectedSecondReadAheadBufferContents = new byte[readAheadRequestSize];
+    getExpectedBufferData(0, readRequestSize, expectedFirstReadAheadBufferContents);
+    getExpectedBufferData(readRequestSize, readAheadRequestSize,
+        expectedSecondReadAheadBufferContents);
+
+    assertTrue("Read should be of exact requested size",
+              inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
+    assertTrue("Data mismatch found in RAH1",
+        Arrays.equals(firstReadBuffer,
+            expectedFirstReadAheadBufferContents));
+
+
+    assertTrue("Read should be of exact requested size",
+        inputStream.read(secondReadBuffer, 0, readAheadRequestSize) == readAheadRequestSize);
+    assertTrue("Data mismatch found in RAH2",
+        Arrays.equals(secondReadBuffer,
+            expectedSecondReadAheadBufferContents));
+  }
+
+  public AbfsInputStream testReadAheadConfigs(int readRequestSize,
+      int readAheadQueueDepth,
+      boolean alwaysReadBufferSizeEnabled,
+      int readAheadBlockSize) throws Exception {
+    Configuration
+        config = new Configuration(
+        this.getRawConfiguration());
+    config.set("fs.azure.read.request.size", Integer.toString(readRequestSize));
+    config.set("fs.azure.readaheadqueue.depth",
+        Integer.toString(readAheadQueueDepth));
+    config.set("fs.azure.read.alwaysReadBufferSize",
+        Boolean.toString(alwaysReadBufferSizeEnabled));
+    config.set("fs.azure.read.readahead.blocksize",
+        Integer.toString(readAheadBlockSize));
+    if (readRequestSize > readAheadBlockSize) {
+      readAheadBlockSize = readRequestSize;
+    }
+
+    Path testPath = new Path(
+        "/testReadAheadConfigs");
+    final AzureBlobFileSystem fs = createTestFile(testPath,
+        ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE, config);
+    byte[] byteBuffer = new byte[ONE_MB];
+    AbfsInputStream inputStream = this.getAbfsStore(fs)
+        .openFileForRead(testPath, null);
+
+    assertEquals("Unexpected AbfsInputStream buffer size", readRequestSize,
+        inputStream.getBufferSize());
+    assertEquals("Unexpected ReadAhead queue depth", readAheadQueueDepth,
+        inputStream.getReadAheadQueueDepth());
+    assertEquals("Unexpected AlwaysReadBufferSize settings",
+        alwaysReadBufferSizeEnabled,
+        inputStream.shouldAlwaysReadBufferSize());
+    assertEquals("Unexpected readAhead block size", readAheadBlockSize,
+        ReadBufferManager.getBufferManager().getReadAheadBlockSize());
+
+    return inputStream;
+  }
+
+  private void getExpectedBufferData(int offset, int length, byte[] b) {
+    boolean startFillingIn = false;
+    int indexIntoBuffer = 0;
+    char character = 'a';
+
+    for (int i = 0; i < (offset + length); i++) {
+      if (i == offset) {
+        startFillingIn = true;
+      }
+
+      if ((startFillingIn) && (indexIntoBuffer < length)) {
+        b[indexIntoBuffer] = (byte) character;
+        indexIntoBuffer++;
+      }
+
+      character = (character == 'z') ? 'a' : (char) ((int) character + 1);
+    }
+  }
+
+  private AzureBlobFileSystem createTestFile(Path testFilePath, long testFileSize,

Review comment:
       I need the file created to also have a specific pattern of data ingested and have another method which needs to return the expected data content from offset to range. This is being used to cross check with readahead buffer data. Will retain for these tests. But thanks, hadnt observed the other APIs, will default to them where there isnt a specific needs on the file content.




----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r504649012



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java
##########
@@ -447,4 +490,168 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception {
     checkEvictedStatus(inputStream, 0, true);
   }
 
+  /**
+   * Test readahead with different config settings for request request size and
+   * readAhead block size
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
+    // Set requestRequestSize = 4MB and readAheadBufferSize=8MB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FOUR_MB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    testReadAheadConfigs(FOUR_MB, TEST_READAHEAD_DEPTH_4, false, EIGHT_MB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(SIXTEEN_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    AbfsInputStream inputStream = testReadAheadConfigs(SIXTEEN_KB,
+        TEST_READAHEAD_DEPTH_2, true, SIXTEEN_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, SIXTEEN_KB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=48KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(SIXTEEN_KB, TEST_READAHEAD_DEPTH_2, true,
+        FORTY_EIGHT_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, FORTY_EIGHT_KB);
+
+    // Test for requestRequestSize =48KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(FORTY_EIGHT_KB, TEST_READAHEAD_DEPTH_2,
+        true,
+        SIXTEEN_KB);
+    testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
+  }
+
+
+  private void testReadAheads(AbfsInputStream inputStream,
+      int readRequestSize,
+      int readAheadRequestSize)
+      throws Exception {
+    if (readRequestSize > readAheadRequestSize) {
+      readAheadRequestSize = readRequestSize;
+    }
+
+    byte[] firstReadBuffer = new byte[readRequestSize];
+    byte[] secondReadBuffer = new byte[readAheadRequestSize];
+
+    // get the expected bytes to compare
+    byte[] expectedFirstReadAheadBufferContents = new byte[readRequestSize];
+    byte[] expectedSecondReadAheadBufferContents = new byte[readAheadRequestSize];
+    getExpectedBufferData(0, readRequestSize, expectedFirstReadAheadBufferContents);
+    getExpectedBufferData(readRequestSize, readAheadRequestSize,
+        expectedSecondReadAheadBufferContents);
+
+    assertTrue("Read should be of exact requested size",
+              inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
+    assertTrue("Data mismatch found in RAH1",
+        Arrays.equals(firstReadBuffer,
+            expectedFirstReadAheadBufferContents));
+
+
+    assertTrue("Read should be of exact requested size",
+        inputStream.read(secondReadBuffer, 0, readAheadRequestSize) == readAheadRequestSize);
+    assertTrue("Data mismatch found in RAH2",

Review comment:
       :) Just a force of habit. Will try to use the assertions.
   Have modified in the other test code areas of this PR as well.




----------------------------------------------------------------
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.

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] mukund-thakur commented on pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
mukund-thakur commented on pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#issuecomment-723746656


   > @mukund-thakur - Will appreciate if you can confirm your review status on this PR. Have provided updates for your comments.
   
   I was on a vacation. Will check once I get some time. 


----------------------------------------------------------------
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.

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 a change in pull request #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

Posted by GitBox <gi...@apache.org>.
snvijaya commented on a change in pull request #2368:
URL: https://github.com/apache/hadoop/pull/2368#discussion_r504648104



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java
##########
@@ -447,4 +490,168 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception {
     checkEvictedStatus(inputStream, 0, true);
   }
 
+  /**
+   * Test readahead with different config settings for request request size and
+   * readAhead block size
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
+    // Set requestRequestSize = 4MB and readAheadBufferSize=8MB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FOUR_MB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    testReadAheadConfigs(FOUR_MB, TEST_READAHEAD_DEPTH_4, false, EIGHT_MB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(SIXTEEN_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    AbfsInputStream inputStream = testReadAheadConfigs(SIXTEEN_KB,
+        TEST_READAHEAD_DEPTH_2, true, SIXTEEN_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, SIXTEEN_KB);
+
+    // Test for requestRequestSize =16KB and readAheadBufferSize=48KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(SIXTEEN_KB, TEST_READAHEAD_DEPTH_2, true,
+        FORTY_EIGHT_KB);
+    testReadAheads(inputStream, SIXTEEN_KB, FORTY_EIGHT_KB);
+
+    // Test for requestRequestSize =48KB and readAheadBufferSize=16KB
+    ReadBufferManager.getBufferManager()
+        .testResetReadBufferManager(FORTY_EIGHT_KB, INCREASED_READ_BUFFER_AGE_THRESHOLD);
+    inputStream = testReadAheadConfigs(FORTY_EIGHT_KB, TEST_READAHEAD_DEPTH_2,
+        true,
+        SIXTEEN_KB);
+    testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
+  }
+
+
+  private void testReadAheads(AbfsInputStream inputStream,
+      int readRequestSize,
+      int readAheadRequestSize)
+      throws Exception {
+    if (readRequestSize > readAheadRequestSize) {
+      readAheadRequestSize = readRequestSize;
+    }
+
+    byte[] firstReadBuffer = new byte[readRequestSize];
+    byte[] secondReadBuffer = new byte[readAheadRequestSize];
+
+    // get the expected bytes to compare
+    byte[] expectedFirstReadAheadBufferContents = new byte[readRequestSize];
+    byte[] expectedSecondReadAheadBufferContents = new byte[readAheadRequestSize];
+    getExpectedBufferData(0, readRequestSize, expectedFirstReadAheadBufferContents);
+    getExpectedBufferData(readRequestSize, readAheadRequestSize,
+        expectedSecondReadAheadBufferContents);
+
+    assertTrue("Read should be of exact requested size",
+              inputStream.read(firstReadBuffer, 0, readRequestSize) == readRequestSize);
+    assertTrue("Data mismatch found in RAH1",
+        Arrays.equals(firstReadBuffer,

Review comment:
       Thanks. Have updated except for the content check assert.




----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   :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: |  markdownlint  |   0m  1s |  |  markdownlint was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 3 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |   2m  7s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | +1 :green_heart: |  compile  |   2m 52s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 10s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 54s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 53s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 13s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 16s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   1m 31s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  51m 12s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.fs.azure.TestNativeAzureFileSystemOperationsMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemContractMocked |
   |   | hadoop.fs.azure.TestOutOfBandAzureBlobOperations |
   |   | hadoop.fs.azure.TestBlobMetadata |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemFileNameCheck |
   |   | hadoop.fs.azure.TestWasbFsck |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemMocked |
   |   | hadoop.fs.azure.TestNativeAzureFileSystemConcurrency |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.40 ServerAPI=1.40 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2368 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle markdownlint |
   | uname | Linux fc250b1997db 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f17e067d527 |
   | Default Java | Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.8+10-post-Ubuntu-0ubuntu118.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_265-8u265-b01-0ubuntu2~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2368/6/testReport/ |
   | Max. process+thread count | 455 (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-2368/6/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.1.3 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT 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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   @mukund-thakur Thanks for your review. I have updated the PR with suggestions. Kindly request you to review.


----------------------------------------------------------------
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.

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 #2368: Hadoop-17296. ABFS: Force reads to be always of buffer size

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


   @mukund-thakur Have synced with latest trunk. Please have a look.


----------------------------------------------------------------
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.

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