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

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

Copilot commented on code in PR #7801:
URL: https://github.com/apache/hadoop/pull/7801#discussion_r2225196847


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java:
##########
@@ -259,10 +259,46 @@ public final class ConfigurationKeys {
   public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = 
"fs.azure.shellkeyprovider.script";
 
   /**
-   * Enable or disable readahead buffer in AbfsInputStream.
+   * Enable or disable readahead V1 in AbfsInputStream.
    * Value: {@value}.
    */
   public static final String FS_AZURE_ENABLE_READAHEAD = 
"fs.azure.enable.readahead";
+  /**
+   * Enable or disable readahead V2 in AbfsInputStream. This will work 
independent of V1.
+   * Value: {@value}.
+   */
+  public static final String FS_AZURE_ENABLE_READAHEAD_V2 = 
"fs.azure.enable.readahead.v2";
+
+  /**
+   * Minimum number of prefetch threads in the thread pool for readahead V2.
+   * {@value }
+   */
+  public static final String FS_AZURE_READAHEAD_V2_MIN_THREAD_POOL_SIZE = 
"fs.azure.readahead.v2.min.thread.pool.size";
+  /**
+   * Maximum number of prefetch threads in the thread pool for readahead V2.
+   * {@value }
+   */
+  public static final String FS_AZURE_READAHEAD_V2_MAX_THREAD_POOL_SIZE = 
"fs.azure.readahead.v2.max.thread.pool.size";
+  /**
+   * Minimum size of the buffer pool for caching prefetched data for readahead 
V2.
+   * {@value }
+   */
+  public static final String FS_AZURE_READAHEAD_V2_MIN_BUFFER_POOL_SIZE = 
"fs.azure.readahead.v2.min.buffer.pool.size";
+  /**
+   * Maximum size of the buffer pool for caching prefetched data for readahead 
V2.
+   * {@value }
+   */
+  public static final String FS_AZURE_READAHEAD_V2_MAX_BUFFER_POOL_SIZE = 
"fs.azure.readahead.v2.max.buffer.pool.size";
+
+  /**
+   * TTL in milliseconds for the idle threads in executor service used by read 
ahead v2.
+   */
+  public static final String 
FS_AZURE_READAHEAD_V2_EXECUTOR_SERVICE_TTL_MILLISECONDS = 
"fs.azure.readahead.v2.executor.service.ttl.seconds";
+
+  /**
+   * TTL in milliseconds for the cached buffers in buffer pool used by read 
ahead v2.
+   */
+  public static final String 
FS_AZURE_READAHEAD_V2_CACHED_BUFFER_TTL_MILLISECONDS = 
"fs.azure.readahead.v2.cachec.buffer.ttl.milliseconds";

Review Comment:
   There's a typo in the configuration key: 'cachec' should be 'cached'.
   ```suggestion
     public static final String 
FS_AZURE_READAHEAD_V2_CACHED_BUFFER_TTL_MILLISECONDS = 
"fs.azure.readahead.v2.cached.buffer.ttl.milliseconds";
   ```



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java:
##########
@@ -0,0 +1,612 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.classification.VisibleForTesting;
+
+/**
+ * The Read Buffer Manager for Rest AbfsClient.
+ * V1 implementation of ReadBufferManager.
+ */
+final class ReadBufferManagerV1 extends ReadBufferManager {
+
+  private static final int NUM_BUFFERS = 16;
+  private static final int NUM_THREADS = 8;
+  private static final int DEFAULT_THRESHOLD_AGE_MILLISECONDS = 3000;
+
+  private Thread[] threads = new Thread[NUM_THREADS];
+  private byte[][] buffers;
+  private static  ReadBufferManagerV1 bufferManager;
+
+  // hide instance constructor
+  private ReadBufferManagerV1() {
+    LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch");
+  }
+
+  /**
+   * Sets the read buffer manager configurations.
+   * @param readAheadBlockSize the size of the read-ahead block in bytes
+   */
+  static void setReadBufferManagerConfigs(int readAheadBlockSize) {
+    if (bufferManager == null) {
+      LOGGER.debug(
+          "ReadBufferManagerV1 not initialized yet. Overriding 
readAheadBlockSize as {}",
+          readAheadBlockSize);
+      setReadAheadBlockSize(readAheadBlockSize);
+      setThresholdAgeMilliseconds(DEFAULT_THRESHOLD_AGE_MILLISECONDS);
+    }
+  }
+
+  /**
+   * Returns the singleton instance of ReadBufferManagerV1.
+   * @return the singleton instance of ReadBufferManagerV1
+   */
+  static ReadBufferManagerV1 getBufferManager() {
+    if (bufferManager == null) {
+      LOCK.lock();
+      try {
+        if (bufferManager == null) {
+          bufferManager = new ReadBufferManagerV1();
+          bufferManager.init();
+        }
+      } finally {
+        LOCK.unlock();
+      }
+    }
+    return bufferManager;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  void init() {
+    buffers = new byte[NUM_BUFFERS][];
+    for (int i = 0; i < NUM_BUFFERS; i++) {
+      buffers[i] = new byte[getReadAheadBlockSize()];  // same buffers are 
reused. The byte array never goes back to GC

Review Comment:
   Comment has grammatical error: 'The byte array never goes back to GC' should 
be 'The byte arrays never go back to GC' or 'These byte arrays are never 
garbage collected'.
   ```suggestion
         buffers[i] = new byte[getReadAheadBlockSize()];  // same buffers are 
reused. These byte arrays are never garbage collected.
   ```



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java:
##########
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+final class ReadBufferManagerV2 extends ReadBufferManager {
+
+  // Thread Pool Configurations
+  private static int minThreadPoolSize;
+  private static int maxThreadPoolSize;
+  private static int executorServiceKeepAliveTimeInMilliSec;
+  private ThreadPoolExecutor workerPool;
+
+  // Buffer Pool Configurations
+  private static int minBufferPoolSize;
+  private static int maxBufferPoolSize;
+  private int numberOfActiveBuffers = 0;
+  private byte[][] bufferPool;
+
+  private static ReadBufferManagerV2 bufferManager;
+
+  // hide instance constructor
+  private ReadBufferManagerV2() {
+    LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch");
+  }
+
+  /**
+   * Sets the read buffer manager configurations.
+   * @param readAheadBlockSize the size of the read-ahead block in bytes
+   * @param abfsConfiguration the AbfsConfiguration instance for other 
configurations
+   */
+  static void setReadBufferManagerConfigs(int readAheadBlockSize, 
AbfsConfiguration abfsConfiguration) {
+    if (bufferManager == null) {
+      minThreadPoolSize = abfsConfiguration.getMinReadAheadV2ThreadPoolSize();
+      maxThreadPoolSize = abfsConfiguration.getMaxReadAheadV2ThreadPoolSize();
+      executorServiceKeepAliveTimeInMilliSec = 
abfsConfiguration.getReadAheadExecutorServiceTTLInMilliSeconds();
+
+      minBufferPoolSize = abfsConfiguration.getMinReadAheadV2BufferPoolSize();
+      maxBufferPoolSize = abfsConfiguration.getMaxReadAheadV2BufferPoolSize();
+      
setThresholdAgeMilliseconds(abfsConfiguration.getReadAheadV2CachedBufferTTLMilliseconds());
+      setReadAheadBlockSize(readAheadBlockSize);
+    }
+  }
+
+  /**
+   * Returns the singleton instance of ReadBufferManagerV2.
+   * @return the singleton instance of ReadBufferManagerV2
+   */
+  static ReadBufferManagerV2 getBufferManager() {
+    if (bufferManager == null) {
+      LOCK.lock();
+      try {
+        if (bufferManager == null) {
+          bufferManager = new ReadBufferManagerV2();
+          bufferManager.init();
+        }
+      } finally {
+        LOCK.unlock();
+      }
+    }
+    return (ReadBufferManagerV2) bufferManager;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  void init() {
+    // Initialize Buffer Pool
+    bufferPool = new byte[maxBufferPoolSize][];
+    for (int i = 0; i < minBufferPoolSize; i++) {
+      bufferPool[i] = new byte[getReadAheadBlockSize()];  // same buffers are 
reused. The byte array never goes back to GC

Review Comment:
   Comment has grammatical error: 'The byte array never goes back to GC' should 
be 'The byte arrays never go back to GC' or 'These byte arrays are never 
garbage collected'.
   ```suggestion
         bufferPool[i] = new byte[getReadAheadBlockSize()];  // same buffers 
are reused. These byte arrays are never garbage collected
   ```



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java:
##########
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+final class ReadBufferManagerV2 extends ReadBufferManager {
+
+  // Thread Pool Configurations
+  private static int minThreadPoolSize;
+  private static int maxThreadPoolSize;
+  private static int executorServiceKeepAliveTimeInMilliSec;
+  private ThreadPoolExecutor workerPool;
+
+  // Buffer Pool Configurations
+  private static int minBufferPoolSize;
+  private static int maxBufferPoolSize;
+  private int numberOfActiveBuffers = 0;
+  private byte[][] bufferPool;
+
+  private static ReadBufferManagerV2 bufferManager;
+
+  // hide instance constructor
+  private ReadBufferManagerV2() {
+    LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch");
+  }
+
+  /**
+   * Sets the read buffer manager configurations.
+   * @param readAheadBlockSize the size of the read-ahead block in bytes
+   * @param abfsConfiguration the AbfsConfiguration instance for other 
configurations
+   */
+  static void setReadBufferManagerConfigs(int readAheadBlockSize, 
AbfsConfiguration abfsConfiguration) {
+    if (bufferManager == null) {
+      minThreadPoolSize = abfsConfiguration.getMinReadAheadV2ThreadPoolSize();
+      maxThreadPoolSize = abfsConfiguration.getMaxReadAheadV2ThreadPoolSize();
+      executorServiceKeepAliveTimeInMilliSec = 
abfsConfiguration.getReadAheadExecutorServiceTTLInMilliSeconds();
+
+      minBufferPoolSize = abfsConfiguration.getMinReadAheadV2BufferPoolSize();
+      maxBufferPoolSize = abfsConfiguration.getMaxReadAheadV2BufferPoolSize();
+      
setThresholdAgeMilliseconds(abfsConfiguration.getReadAheadV2CachedBufferTTLMilliseconds());
+      setReadAheadBlockSize(readAheadBlockSize);
+    }
+  }
+
+  /**
+   * Returns the singleton instance of ReadBufferManagerV2.
+   * @return the singleton instance of ReadBufferManagerV2
+   */
+  static ReadBufferManagerV2 getBufferManager() {
+    if (bufferManager == null) {
+      LOCK.lock();
+      try {
+        if (bufferManager == null) {
+          bufferManager = new ReadBufferManagerV2();
+          bufferManager.init();
+        }
+      } finally {
+        LOCK.unlock();
+      }
+    }
+    return (ReadBufferManagerV2) bufferManager;

Review Comment:
   The cast to ReadBufferManagerV2 is unsafe. The static field 'bufferManager' 
is of type ReadBufferManagerV2, but the return statement suggests it could be 
something else. Either change the field type or remove the unnecessary cast.
   ```suggestion
       return bufferManager;
   ```





> ABFS: [ReadAheadV2] Refactor ReadBufferManager to isolate new code with the 
> current working code
> ------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-19613
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19613
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/azure
>    Affects Versions: 3.5.0, 3.4.1
>            Reporter: Anuj Modi
>            Assignee: Anuj Modi
>            Priority: Major
>              Labels: pull-request-available
>
> Read Buffer Manager used today was introduced way back and has been stable 
> for quite a while.
> Read Buffer Manager to be introduced as part of 
> https://issues.apache.org/jira/browse/HADOOP-19596 will introduce many 
> changes incrementally over time. While the development goes on and we are 
> able to fully stabilise the optimized version we need the current flow to be 
> functional and undisturbed. 
> This work item is to isolate that from new code by refactoring 
> ReadBufferManager class to have 2 different implementations with same public 
> interfaces: ReadBufferManagerV1 and ReadBufferManagerV2.
> This will also introduce new configs that can be used to toggle between new 
> and old code. 



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

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

Reply via email to