From 37898b427299110abb527858bb8335578b0595ea Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Mon, 28 Jul 2025 21:44:29 -0700 Subject: [PATCH 01/13] RBMV2 Implementation --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 75 +- .../azurebfs/constants/ConfigurationKeys.java | 38 + .../constants/FileSystemConfigurations.java | 8 + .../fs/azurebfs/services/AbfsInputStream.java | 4 + .../fs/azurebfs/services/ReadBuffer.java | 39 +- .../azurebfs/services/ReadBufferManager.java | 2 +- .../services/ReadBufferManagerV2.java | 667 +++++++++++++++--- .../azurebfs/services/ReadBufferWorker.java | 16 +- .../services/ITestReadBufferManager.java | 235 +++--- .../services/ITestReadBufferManagerV2.java | 148 ++++ .../services/TestAbfsInputStream.java | 4 +- 11 files changed, 1030 insertions(+), 206 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 1242122f03015..69952ad888149 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -386,6 +386,11 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ENABLE_READAHEAD_V2) private boolean isReadAheadV2Enabled; + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING, + DefaultValue = DEFAULT_ENABLE_READAHEAD_V2_DYNAMIC_SCALING) + private boolean isReadAheadV2DynamicScalingEnabled; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_READAHEAD_V2_MIN_THREAD_POOL_SIZE, DefaultValue = DEFAULT_READAHEAD_V2_MIN_THREAD_POOL_SIZE) @@ -406,6 +411,26 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_READAHEAD_V2_MAX_BUFFER_POOL_SIZE) private int maxReadAheadV2BufferPoolSize; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_READAHEAD_V2_CPU_MONITORING_INTERVAL_MILLIS, + DefaultValue = DEFAULT_READAHEAD_V2_CPU_MONITORING_INTERVAL_MILLIS) + private int readAheadV2CpuMonitoringIntervalMillis; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_READAHEAD_V2_THREAD_POOL_UPSCALE_PERCENTAGE, + DefaultValue = DEFAULT_READAHEAD_V2_THREAD_POOL_UPSCALE_PERCENTAGE) + private int readAheadV2ThreadPoolUpscalePercentage; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_READAHEAD_V2_THREAD_POOL_DOWNSCALE_PERCENTAGE, + DefaultValue = DEFAULT_READAHEAD_V2_THREAD_POOL_DOWNSCALE_PERCENTAGE) + private int readAheadV2ThreadPoolDownscalePercentage; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_READAHEAD_V2_MEMORY_MONITORING_INTERVAL_MILLIS, + DefaultValue = DEFAULT_READAHEAD_V2_MEMORY_MONITORING_INTERVAL_MILLIS) + private int readAheadV2MemoryMonitoringIntervalMillis; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_READAHEAD_V2_EXECUTOR_SERVICE_TTL_MILLIS, DefaultValue = DEFAULT_READAHEAD_V2_EXECUTOR_SERVICE_TTL_MILLIS) @@ -416,6 +441,16 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_READAHEAD_V2_CACHED_BUFFER_TTL_MILLIS) private int readAheadV2CachedBufferTTLMillis; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_READAHEAD_V2_CPU_USAGE_THRESHOLD_PERCENT, + DefaultValue = DEFAULT_READAHEAD_V2_CPU_USAGE_THRESHOLD_PERCENT) + private int readAheadV2CpuUsageThresholdPercent; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_READAHEAD_V2_MEMORY_USAGE_THRESHOLD_PERCENT, + DefaultValue = DEFAULT_READAHEAD_V2_MEMORY_USAGE_THRESHOLD_PERCENT) + private int readAheadV2MemoryUsageThresholdPercent; + @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS, MinValue = 0, DefaultValue = DEFAULT_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS_IN_SECONDS) @@ -1403,6 +1438,18 @@ public boolean isReadAheadEnabled() { return this.enabledReadAhead; } + /** + * Checks if the read-ahead v2 feature is enabled by user. + * @return true if read-ahead v2 is enabled, false otherwise. + */ + public boolean isReadAheadV2Enabled() { + return this.isReadAheadV2Enabled; + } + + public boolean isReadAheadV2DynamicScalingEnabled() { + return isReadAheadV2DynamicScalingEnabled; + } + public int getMinReadAheadV2ThreadPoolSize() { if (minReadAheadV2ThreadPoolSize <= 0) { // If the minReadAheadV2ThreadPoolSize is not set, use the default value @@ -1435,6 +1482,22 @@ public int getMaxReadAheadV2BufferPoolSize() { return maxReadAheadV2BufferPoolSize; } + public int getReadAheadV2CpuMonitoringIntervalMillis() { + return readAheadV2CpuMonitoringIntervalMillis; + } + + public int getReadAheadV2ThreadPoolUpscalePercentage() { + return readAheadV2ThreadPoolUpscalePercentage; + } + + public int getReadAheadV2ThreadPoolDownscalePercentage() { + return readAheadV2ThreadPoolDownscalePercentage; + } + + public int getReadAheadV2MemoryMonitoringIntervalMillis() { + return readAheadV2MemoryMonitoringIntervalMillis; + } + public int getReadAheadExecutorServiceTTLInMillis() { return readAheadExecutorServiceTTLMillis; } @@ -1443,12 +1506,12 @@ public int getReadAheadV2CachedBufferTTLMillis() { return readAheadV2CachedBufferTTLMillis; } - /** - * Checks if the read-ahead v2 feature is enabled by user. - * @return true if read-ahead v2 is enabled, false otherwise. - */ - public boolean isReadAheadV2Enabled() { - return this.isReadAheadV2Enabled; + public int getReadAheadV2CpuUsageThresholdPercent() { + return readAheadV2CpuUsageThresholdPercent; + } + + public int getReadAheadV2MemoryUsageThresholdPercent() { + return readAheadV2MemoryUsageThresholdPercent; } @VisibleForTesting diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 50a88ab4e4587..520e0d8831f29 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -269,6 +269,12 @@ public final class ConfigurationKeys { */ public static final String FS_AZURE_ENABLE_READAHEAD_V2 = "fs.azure.enable.readahead.v2"; + /** + * Enable or disable dynamic scaling of thread pool and buffer pool of readahead V2. + * Value: {@value}. + */ + public static final String FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING = "fs.azure.enable.readahead.v2.dynamic.scaling"; + /** * Minimum number of prefetch threads in the thread pool for readahead V2. * {@value } @@ -290,6 +296,28 @@ public final class ConfigurationKeys { */ public static final String FS_AZURE_READAHEAD_V2_MAX_BUFFER_POOL_SIZE = "fs.azure.readahead.v2.max.buffer.pool.size"; + /** + * Interval in milliseconds for periodic monitoring of CPU usage and up/down scaling thread pool size accordingly. + * {@value } + */ + public static final String FS_AZURE_READAHEAD_V2_CPU_MONITORING_INTERVAL_MILLIS = "fs.azure.readahead.v2.cpu.monitoring.interval.millis"; + + /** + * Percentage by which the thread pool size should be upscaled when CPU usage is low. + */ + public static final String FS_AZURE_READAHEAD_V2_THREAD_POOL_UPSCALE_PERCENTAGE = "fs.azure.readahead.v2.thread.pool.upscale.percentage"; + + /** + * Percentage by which the thread pool size should be downscaled when CPU usage is high. + */ + public static final String FS_AZURE_READAHEAD_V2_THREAD_POOL_DOWNSCALE_PERCENTAGE = "fs.azure.readahead.v2.thread.pool.downscale.percentage"; + + /** + * Interval in milliseconds for periodic monitoring of memory usage and up/down scaling buffer pool size accordingly. + * {@value } + */ + public static final String FS_AZURE_READAHEAD_V2_MEMORY_MONITORING_INTERVAL_MILLIS = "fs.azure.readahead.v2.memory.monitoring.interval.millis"; + /** * TTL in milliseconds for the idle threads in executor service used by read ahead v2. */ @@ -300,6 +328,16 @@ public final class ConfigurationKeys { */ public static final String FS_AZURE_READAHEAD_V2_CACHED_BUFFER_TTL_MILLIS = "fs.azure.readahead.v2.cached.buffer.ttl.millis"; + /** + * Threshold percentage for CPU usage to scale up/down the thread pool size in read ahead v2. + */ + public static final String FS_AZURE_READAHEAD_V2_CPU_USAGE_THRESHOLD_PERCENT = "fs.azure.readahead.v2.cpu.usage.threshold.percent"; + + /** + * Threshold percentage for memory usage to scale up/down the buffer pool size in read ahead v2. + */ + public static final String FS_AZURE_READAHEAD_V2_MEMORY_USAGE_THRESHOLD_PERCENT = "fs.azure.readahead.v2.memory.usage.threshold.percent"; + /** Setting this true will make the driver use it's own RemoteIterator implementation */ public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator"; /** Server side encryption key encoded in Base6format {@value}.*/ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 824a4c9701ea0..82c58b41c9fb8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -128,12 +128,19 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_ENABLE_READAHEAD = true; public static final boolean DEFAULT_ENABLE_READAHEAD_V2 = false; + public static final boolean DEFAULT_ENABLE_READAHEAD_V2_DYNAMIC_SCALING = true; public static final int DEFAULT_READAHEAD_V2_MIN_THREAD_POOL_SIZE = -1; public static final int DEFAULT_READAHEAD_V2_MAX_THREAD_POOL_SIZE = -1; public static final int DEFAULT_READAHEAD_V2_MIN_BUFFER_POOL_SIZE = -1; public static final int DEFAULT_READAHEAD_V2_MAX_BUFFER_POOL_SIZE = -1; + public static final int DEFAULT_READAHEAD_V2_CPU_MONITORING_INTERVAL_MILLIS = 6_000; + public static final int DEFAULT_READAHEAD_V2_THREAD_POOL_UPSCALE_PERCENTAGE = 20; + public static final int DEFAULT_READAHEAD_V2_THREAD_POOL_DOWNSCALE_PERCENTAGE = 30; + public static final int DEFAULT_READAHEAD_V2_MEMORY_MONITORING_INTERVAL_MILLIS = 6_000; public static final int DEFAULT_READAHEAD_V2_EXECUTOR_SERVICE_TTL_MILLIS = 3_000; public static final int DEFAULT_READAHEAD_V2_CACHED_BUFFER_TTL_MILLIS = 6_000; + public static final int DEFAULT_READAHEAD_V2_CPU_USAGE_THRESHOLD_PERCENT = 50; + public static final int DEFAULT_READAHEAD_V2_MEMORY_USAGE_THRESHOLD_PERCENT = 50; public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING; public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN"; @@ -209,6 +216,7 @@ public final class FileSystemConfigurations { public static final int ZERO = 0; public static final int HUNDRED = 100; + public static final double ONE_HUNDRED = 100.0; public static final long THOUSAND = 1000L; public static final HttpOperationType DEFAULT_NETWORKING_LIBRARY diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 3dc7f88e52911..5adb350991b23 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -813,6 +813,10 @@ public String getStreamID() { return inputStreamId; } + public String getETag() { + return eTag; + } + /** * Getter for AbfsInputStreamStatistics. * diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java index 9ce926d841c84..930482b2825d8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java @@ -20,15 +20,18 @@ import java.io.IOException; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus.READ_FAILED; -class ReadBuffer { +public class ReadBuffer { private AbfsInputStream stream; + private String eTag; + private String path; // path of the file this buffer is for private long offset; // offset within the file for the buffer private int length; // actual length, set after the buffer is filles private int requestedLength; // requested length of the read @@ -44,6 +47,7 @@ class ReadBuffer { private boolean isFirstByteConsumed = false; private boolean isLastByteConsumed = false; private boolean isAnyByteConsumed = false; + private AtomicInteger refCount = new AtomicInteger(0); private IOException errException = null; @@ -51,10 +55,26 @@ public AbfsInputStream getStream() { return stream; } + public String getETag() { + return eTag; + } + + public String getPath() { + return path; + } + public void setStream(AbfsInputStream stream) { this.stream = stream; } + public void setETag(String eTag) { + this.eTag = eTag; + } + + public void setPath(String path) { + this.path = path; + } + public void setTracingContext(TracingContext tracingContext) { this.tracingContext = tracingContext; } @@ -122,6 +142,20 @@ public void setStatus(ReadBufferStatus status) { } } + public void startReading() { + refCount.getAndIncrement(); + } + + public void endReading() { + if (refCount.decrementAndGet() < 0) { + throw new IllegalStateException("ReadBuffer refCount cannot be negative"); + } + } + + public int getRefCount() { + return refCount.get(); + } + public CountDownLatch getLatch() { return latch; } @@ -162,4 +196,7 @@ public void setAnyByteConsumed(boolean isAnyByteConsumed) { this.isAnyByteConsumed = isAnyByteConsumed; } + public boolean isFullyConsumed() { + return isFirstByteConsumed() && isLastByteConsumed(); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 9ee128fbc3275..9b1b108aac43f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -264,7 +264,7 @@ protected synchronized List getReadAheadQueueCopy() { * @return a list of in-progress {@link ReadBuffer} objects */ @VisibleForTesting - protected synchronized List getInProgressCopiedList() { + protected synchronized List getInProgressListCopy() { return new ArrayList<>(inProgressList); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 9cce860127dae..0f80beac32b87 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -17,87 +17,137 @@ */ package org.apache.hadoop.fs.azurebfs.services; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; + +import com.sun.management.OperatingSystemMXBean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.MemoryUsage; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.Stack; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; -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; +import org.apache.hadoop.classification.VisibleForTesting; + +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_HUNDRED; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +/** + * The Improved Read Buffer Manager for Rest AbfsClient. + */ final class ReadBufferManagerV2 extends ReadBufferManager { + // Internal constants + private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManagerV2.class); + private static final ReentrantLock LOCK = new ReentrantLock(); // Thread Pool Configurations private static int minThreadPoolSize; private static int maxThreadPoolSize; + private static int cpuMonitoringIntervalInMilliSec; + private static int cpuThreshold; + private static int threadPoolUpscalePercentage; + private static int threadPoolDownscalePercentage; private static int executorServiceKeepAliveTimeInMilliSec; + private static boolean isDynamicScalingEnabled; + private ThreadPoolExecutor workerPool; + private final List workerRefs = new ArrayList<>(); // Buffer Pool Configurations private static int minBufferPoolSize; private static int maxBufferPoolSize; + private static int memoryMonitoringIntervalInMilliSec; + private static double memoryThreshold; private int numberOfActiveBuffers = 0; private byte[][] bufferPool; + // Buffer Manager Structures 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 + * Private constructor to prevent instantiation as this needs to be singleton. */ - static void setReadBufferManagerConfigs(int readAheadBlockSize, AbfsConfiguration abfsConfiguration) { - if (bufferManager == null) { - minThreadPoolSize = abfsConfiguration.getMinReadAheadV2ThreadPoolSize(); - maxThreadPoolSize = abfsConfiguration.getMaxReadAheadV2ThreadPoolSize(); - executorServiceKeepAliveTimeInMilliSec = abfsConfiguration.getReadAheadExecutorServiceTTLInMillis(); - - minBufferPoolSize = abfsConfiguration.getMinReadAheadV2BufferPoolSize(); - maxBufferPoolSize = abfsConfiguration.getMaxReadAheadV2BufferPoolSize(); - setThresholdAgeMilliseconds(abfsConfiguration.getReadAheadV2CachedBufferTTLMillis()); - setReadAheadBlockSize(readAheadBlockSize); - } + private ReadBufferManagerV2() { + printTraceLog("Creating Read Buffer Manager V2 with HADOOP-18546 patch"); } - /** - * Returns the singleton instance of ReadBufferManagerV2. - * @return the singleton instance of ReadBufferManagerV2 - */ - static ReadBufferManagerV2 getBufferManager() { + public static ReadBufferManagerV2 getBufferManager() { if (bufferManager == null) { LOCK.lock(); try { if (bufferManager == null) { bufferManager = new ReadBufferManagerV2(); bufferManager.init(); + LOGGER.trace("ReadBufferManagerV2 singleton initialized"); } } finally { LOCK.unlock(); } } - return bufferManager; + return (ReadBufferManagerV2) bufferManager; + } + + /** + * Set the ReadBufferManagerV2 configurations based on the provided before singleton initialization. + * @param abfsConfiguration the configuration to set for the ReadBufferManagerV2. + */ + public static void setReadBufferManagerConfigs(final int readAheadBlockSize, + final AbfsConfiguration abfsConfiguration) { + // Set Configs only before initializations. + if (bufferManager == null) { + minThreadPoolSize = abfsConfiguration.getMinReadAheadV2ThreadPoolSize(); + maxThreadPoolSize = abfsConfiguration.getMaxReadAheadV2ThreadPoolSize(); + cpuMonitoringIntervalInMilliSec = abfsConfiguration.getReadAheadV2CpuMonitoringIntervalMillis(); + cpuThreshold = abfsConfiguration.getReadAheadV2CpuUsageThresholdPercent(); + threadPoolUpscalePercentage = abfsConfiguration.getReadAheadV2ThreadPoolUpscalePercentage(); + threadPoolDownscalePercentage = abfsConfiguration.getReadAheadV2ThreadPoolDownscalePercentage(); + executorServiceKeepAliveTimeInMilliSec = abfsConfiguration.getReadAheadExecutorServiceTTLInMillis(); + + minBufferPoolSize = abfsConfiguration.getMinReadAheadV2BufferPoolSize(); + maxBufferPoolSize = abfsConfiguration.getMaxReadAheadV2BufferPoolSize(); + memoryMonitoringIntervalInMilliSec = abfsConfiguration.getReadAheadV2MemoryMonitoringIntervalMillis(); + memoryThreshold = abfsConfiguration.getReadAheadV2MemoryUsageThresholdPercent(); + setThresholdAgeMilliseconds(abfsConfiguration.getReadAheadV2CachedBufferTTLMillis()); + isDynamicScalingEnabled = abfsConfiguration.isReadAheadV2DynamicScalingEnabled(); + setReadAheadBlockSize(readAheadBlockSize); + } } /** - * {@inheritDoc} + * Initialize the singleton ReadBufferManagerV2. */ @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. These byte arrays are never garbage collected + bufferPool[i] = new byte[getReadAheadBlockSize()]; // same buffers are reused. The byte array never goes back to GC getFreeList().add(i); numberOfActiveBuffers++; } + ScheduledExecutorService memoryMonitorThread + = Executors.newSingleThreadScheduledExecutor(); + memoryMonitorThread.scheduleAtFixedRate(this::scheduledEviction, + memoryMonitoringIntervalInMilliSec, memoryMonitoringIntervalInMilliSec, TimeUnit.MILLISECONDS); // Initialize a Fixed Size Thread Pool with minThreadPoolSize threads workerPool = new ThreadPoolExecutor( @@ -109,104 +159,556 @@ void init() { namedThreadFactory); workerPool.allowCoreThreadTimeOut(true); for (int i = 0; i < minThreadPoolSize; i++) { - ReadBufferWorker worker = new ReadBufferWorker(i, this); + ReadBufferWorker worker = new ReadBufferWorker(i, getBufferManager()); + workerRefs.add(worker); workerPool.submit(worker); } ReadBufferWorker.UNLEASH_WORKERS.countDown(); + + if (isDynamicScalingEnabled) { + ScheduledExecutorService cpuMonitorThread + = Executors.newSingleThreadScheduledExecutor(); + cpuMonitorThread.scheduleAtFixedRate(this::adjustThreadPool, + cpuMonitoringIntervalInMilliSec, cpuMonitoringIntervalInMilliSec, + TimeUnit.MILLISECONDS); + } + + printTraceLog("ReadBufferManagerV2 initialized with {} buffers and {} worker threads", + numberOfActiveBuffers, workerRefs.size()); } /** - * {@inheritDoc} + * {@link AbfsInputStream} calls this method to queueing read-ahead. + * @param stream which read-ahead is requested from. + * @param requestedOffset The offset in the file which should be read. + * @param requestedLength The length to read. */ @Override - public void queueReadAhead(final AbfsInputStream stream, - final long requestedOffset, - final int requestedLength, - final TracingContext tracingContext) { - // TODO: To be implemented + public void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, + final int requestedLength, TracingContext tracingContext) { + printTraceLog("Start Queueing readAhead for file: {}, with eTag: {}, offset: {}, length: {}, triggered by stream: {}", + stream.getPath(), stream.getETag(), requestedOffset, requestedLength, stream.hashCode()); + ReadBuffer buffer; + synchronized (this) { + if (isAlreadyQueued(stream.getETag(), requestedOffset)) { + // Already queued for this offset, so skip queuing. + printTraceLog("Skipping queuing readAhead for file: {}, with eTag: {}, offset: {}, triggered by stream: {} as it is already queued", + stream.getPath(), stream.getETag(), requestedOffset, stream.hashCode()); + return; + } + if (getFreeList().isEmpty() && !tryMemoryUpscale() && !tryEvict()) { + // No buffers are available and more buffers cannot be created. Skip queuing. + printTraceLog("Skipping queuing readAhead for file: {}, with eTag: {}, offset: {}, triggered by stream: {} as no buffers are available", + stream.getPath(), stream.getETag(), requestedOffset, stream.hashCode()); + return; + } + + // Create a new ReadBuffer to keep the prefetched data and queue. + buffer = new ReadBuffer(); + buffer.setStream(stream); // To map buffer with stream that requested it + buffer.setETag(stream.getETag()); // To map buffer with file it belongs to + buffer.setPath(stream.getPath()); + buffer.setOffset(requestedOffset); + buffer.setLength(0); + buffer.setRequestedLength(requestedLength); + buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE); + buffer.setLatch(new CountDownLatch(1)); + buffer.setTracingContext(tracingContext); + + if (getFreeList().empty()) { + /* + * By now there should be at least one buffer available. + * This is to double sure that after upscaling or eviction, + * we still have free buffer available. If not, we skip queueing. + */ + return; + } + Integer bufferIndex = getFreeList().pop(); + buffer.setBuffer(bufferPool[bufferIndex]); + buffer.setBufferindex(bufferIndex); + getReadAheadQueue().add(buffer); + notifyAll(); + printTraceLog("Done q-ing readAhead for file: {}, with eTag:{}, offset: {}, buffer idx: {}, triggered by stream: {}", + stream.getPath(), stream.getETag(), requestedOffset, buffer.getBufferindex(), stream.hashCode()); + } } /** - * {@inheritDoc} + * {@link AbfsInputStream} calls this method read any bytes already available in a buffer (thereby saving a + * remote read). This returns the bytes if the data already exists in buffer. If there is a buffer that is reading + * the requested offset, then this method blocks until that read completes. If the data is queued in a read-ahead + * but not picked up by a worker thread yet, then it cancels that read-ahead and reports cache miss. This is because + * depending on worker thread availability, the read-ahead may take a while - the calling thread can do its own + * read to get the data faster (compared to the read waiting in queue for an indeterminate amount of time). + * + * @param stream of the file to read bytes for + * @param position the offset in the file to do a read for + * @param length the length to read + * @param buffer the buffer to read data into. Note that the buffer will be written into from offset 0. + * @return the number of bytes read */ @Override - public int getBlock(final AbfsInputStream stream, - final long position, - final int length, - final byte[] buffer) throws IOException { - // TODO: To be implemented + public int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) + throws IOException { + // not synchronized, so have to be careful with locking + printTraceLog("getBlock request for file: {}, with eTag: {}, for position: {} for length: {} received from stream: {}", + stream.getPath(), stream.getETag(), position, length, stream.hashCode()); + + String requestedETag = stream.getETag(); + + // Wait for any in-progress read to complete. + waitForProcess(requestedETag, position); + + int bytesRead = 0; + synchronized (this) { + bytesRead = getBlockFromCompletedQueue(requestedETag, position, length, buffer); + } + if (bytesRead > 0) { + printTraceLog("Done read from Cache for the file with eTag: {}, position: {}, length: {}, requested by stream: {}", + requestedETag, position, bytesRead, stream.hashCode()); + return bytesRead; + } + + // otherwise, just say we got nothing - calling thread can do its own read return 0; } /** - * {@inheritDoc} + * {@link ReadBufferWorker} thread calls this to get the next buffer that it should work on. + * @return {@link ReadBuffer} + * @throws InterruptedException if thread is interrupted */ @Override public ReadBuffer getNextBlockToRead() throws InterruptedException { - // TODO: To be implemented - return null; + ReadBuffer buffer = null; + synchronized (this) { + // Blocking Call to wait for prefetch to be queued. + while (getReadAheadQueue().size() == 0) { + wait(); + } + + buffer = getReadAheadQueue().remove(); + notifyAll(); + if (buffer == null) { + return null; + } + buffer.setStatus(ReadBufferStatus.READING_IN_PROGRESS); + getInProgressList().add(buffer); + } + printTraceLog("ReadBufferWorker picked file: {}, with eTag: {}, for offset: {}, queued by stream: {}", + buffer.getPath(), buffer.getETag(), buffer.getOffset(), buffer.getStream().hashCode()); + return buffer; } /** - * {@inheritDoc} + * {@link ReadBufferWorker} thread calls this method to post completion. * + * @param buffer the buffer whose read was completed + * @param result the {@link ReadBufferStatus} after the read operation in the worker thread + * @param bytesActuallyRead the number of bytes that the worker thread was actually able to read */ @Override - public void doneReading(final ReadBuffer buffer, - final ReadBufferStatus result, + public void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) { - // TODO: To be implemented + printTraceLog("ReadBufferWorker completed prefetch for file: {} with eTag: {}, for offset: {}, queued by stream: {}, with status: {} and bytes read: {}", + buffer.getPath(), buffer.getETag(), buffer.getOffset(), buffer.getStream().hashCode(), result, bytesActuallyRead); + synchronized (this) { + // If this buffer has already been purged during + // close of InputStream then we don't update the lists. + if (getInProgressList().contains(buffer)) { + getInProgressList().remove(buffer); + if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) { + // Successful read, so update the buffer status and length + buffer.setStatus(ReadBufferStatus.AVAILABLE); + buffer.setLength(bytesActuallyRead); + } else { + // Failed read, reuse buffer for next read, this buffer will be + // evicted later based on eviction policy. + getFreeList().push(buffer.getBufferindex()); + } + // completed list also contains FAILED read buffers + // for sending exception message to clients. + buffer.setStatus(result); + buffer.setTimeStamp(currentTimeMillis()); + getCompletedReadList().add(buffer); + } + } + + //outside the synchronized, since anyone receiving a wake-up from the latch must see safe-published results + buffer.getLatch().countDown(); // wake up waiting threads (if any) } /** - * {@inheritDoc} + * Purging the buffers associated with an {@link AbfsInputStream} + * from {@link ReadBufferManagerV2} when stream is closed. + * @param stream input stream. */ - @Override - public void purgeBuffersForStream(final AbfsInputStream stream) { - // TODO: To be implemented + public synchronized void purgeBuffersForStream(AbfsInputStream stream) { + printDebugLog("Purging stale buffers for AbfsInputStream {} ", stream); + getReadAheadQueue().removeIf(readBuffer -> readBuffer.getStream() == stream); + purgeList(stream, getCompletedReadList()); + } + + private boolean isAlreadyQueued(final String eTag, final long requestedOffset) { + // returns true if any part of the buffer is already queued + return (isInList(getReadAheadQueue(), eTag, requestedOffset) + || isInList(getInProgressList(), eTag, requestedOffset) + || isInList(getCompletedReadList(), eTag, requestedOffset)); + } + + private boolean isInList(final Collection list, final String eTag, + final long requestedOffset) { + return (getFromList(list, eTag, requestedOffset) != null); + } + + private ReadBuffer getFromList(final Collection list, final String eTag, + final long requestedOffset) { + for (ReadBuffer buffer : list) { + if (eTag.equals(buffer.getETag())) { + if (buffer.getStatus() == ReadBufferStatus.AVAILABLE + && requestedOffset >= buffer.getOffset() + && requestedOffset < buffer.getOffset() + buffer.getLength()) { + return buffer; + } else if (requestedOffset >= buffer.getOffset() + && requestedOffset + < buffer.getOffset() + buffer.getRequestedLength()) { + return buffer; + } + } + } + return null; } /** - * {@inheritDoc} + * If any buffer in the completed list can be reclaimed then reclaim it and return the buffer to free list. + * The objective is to find just one buffer - there is no advantage to evicting more than one. + * @return whether the eviction succeeded - i.e., were we able to free up one buffer */ - @VisibleForTesting - @Override - public int getNumBuffers() { - return numberOfActiveBuffers; + private synchronized boolean tryEvict() { + ReadBuffer nodeToEvict = null; + if (getCompletedReadList().size() <= 0) { + return false; // there are no evict-able buffers + } + + long currentTimeInMs = currentTimeMillis(); + + // first, try buffers where all bytes have been consumed (approximated as first and last bytes consumed) + for (ReadBuffer buf : getCompletedReadList()) { + if (buf.isFullyConsumed()) { + nodeToEvict = buf; + break; + } + } + if (nodeToEvict != null) { + return manualEviction(nodeToEvict); + } + + // next, try buffers where any bytes have been consumed (maybe a bad idea? have to experiment and see) + for (ReadBuffer buf : getCompletedReadList()) { + if (buf.isAnyByteConsumed()) { + nodeToEvict = buf; + break; + } + } + + if (nodeToEvict != null) { + return manualEviction(nodeToEvict); + } + + // next, try any old nodes that have not been consumed + // Failed read buffers (with buffer index=-1) that are older than + // thresholdAge should be cleaned up, but at the same time should not + // report successful eviction. + // Queue logic expects that a buffer is freed up for read ahead when + // eviction is successful, whereas a failed ReadBuffer would have released + // its buffer when its status was set to READ_FAILED. + long earliestBirthday = Long.MAX_VALUE; + ArrayList oldFailedBuffers = new ArrayList<>(); + for (ReadBuffer buf : getCompletedReadList()) { + if ((buf.getBufferindex() != -1) + && (buf.getTimeStamp() < earliestBirthday)) { + nodeToEvict = buf; + earliestBirthday = buf.getTimeStamp(); + } else if ((buf.getBufferindex() == -1) + && (currentTimeInMs - buf.getTimeStamp()) > getThresholdAgeMilliseconds()) { + oldFailedBuffers.add(buf); + } + } + + for (ReadBuffer buf : oldFailedBuffers) { + manualEviction(buf); + } + + if ((currentTimeInMs - earliestBirthday > getThresholdAgeMilliseconds()) && (nodeToEvict != null)) { + return manualEviction(nodeToEvict); + } + + printTraceLog("No buffer eligible for eviction"); + // nothing can be evicted + return false; + } + + private boolean evict(final ReadBuffer buf) { + if (buf.getRefCount() > 0) { + // If the buffer is still being read, then we cannot evict it. + printTraceLog("Cannot evict buffer with index: {}, file: {}, with eTag: {}, offset: {} as it is still being read by some input stream", + buf.getBufferindex(), buf.getPath(), buf.getETag(), buf.getOffset()); + return false; + } + // As failed ReadBuffers (bufferIndx = -1) are saved in getCompletedReadList(), + // avoid adding it to availableBufferList. + if (buf.getBufferindex() != -1) { + getFreeList().push(buf.getBufferindex()); + } + getCompletedReadList().remove(buf); + buf.setTracingContext(null); + printTraceLog("Eviction of Buffer Completed for BufferIndex: {}, file: {}, with eTag: {}, offset: {}, is fully consumed: {}, is partially consumed: {}", + buf.getBufferindex(), buf.getPath(), buf.getETag(), buf.getOffset(), buf.isFullyConsumed(), buf.isAnyByteConsumed()); + return true; + } + + private void waitForProcess(final String eTag, final long position) { + ReadBuffer readBuf; + synchronized (this) { + clearFromReadAheadQueue(eTag, position); + readBuf = getFromList(getInProgressList(), eTag, position); + } + if (readBuf != null) { // if in in-progress queue, then block for it + try { + printTraceLog("A relevant read buffer for file: {}, with eTag: {}, offset: {}, queued by stream: {}, having buffer idx: {} is being prefetched, waiting for latch", + readBuf.getPath(), readBuf.getETag(), readBuf.getOffset(), readBuf.getStream().hashCode(), readBuf.getBufferindex()); + readBuf.getLatch().await(); // blocking wait on the caller stream's thread + // Note on correctness: readBuf gets out of getInProgressList() only in 1 place: after worker thread + // is done processing it (in doneReading). There, the latch is set after removing the buffer from + // getInProgressList(). So this latch is safe to be outside the synchronized block. + // Putting it in synchronized would result in a deadlock, since this thread would be holding the lock + // while waiting, so no one will be able to change any state. If this becomes more complex in the future, + // then the latch cane be removed and replaced with wait/notify whenever getInProgressList() is touched. + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + printTraceLog("Latch done for file: {}, with eTag: {}, for offset: {}, buffer index: {} queued by stream: {}", + readBuf.getPath(), readBuf.getETag(), readBuf.getOffset(), readBuf.getBufferindex(), readBuf.getStream().hashCode()); + } + } + + private void clearFromReadAheadQueue(final String eTag, final long requestedOffset) { + ReadBuffer buffer = getFromList(getReadAheadQueue(), eTag, requestedOffset); + if (buffer != null) { + getReadAheadQueue().remove(buffer); + notifyAll(); // lock is held in calling method + getFreeList().push(buffer.getBufferindex()); + } + } + + private int getBlockFromCompletedQueue(final String eTag, final long position, + final int length, final byte[] buffer) throws IOException { + ReadBuffer buf = getBufferFromCompletedQueue(eTag, position); + + if (buf == null) { + return 0; + } + + buf.startReading(); // atomic increment of refCount. + + if (buf.getStatus() == ReadBufferStatus.READ_FAILED) { + // To prevent new read requests to fail due to old read-ahead attempts, + // return exception only from buffers that failed within last getThresholdAgeMilliseconds() + if ((currentTimeMillis() - (buf.getTimeStamp()) < getThresholdAgeMilliseconds())) { + throw buf.getErrException(); + } else { + return 0; + } + } + + if ((buf.getStatus() != ReadBufferStatus.AVAILABLE) + || (position >= buf.getOffset() + buf.getLength())) { + return 0; + } + + int cursor = (int) (position - buf.getOffset()); + int availableLengthInBuffer = buf.getLength() - cursor; + int lengthToCopy = Math.min(length, availableLengthInBuffer); + System.arraycopy(buf.getBuffer(), cursor, buffer, 0, lengthToCopy); + if (cursor == 0) { + buf.setFirstByteConsumed(true); + } + if (cursor + lengthToCopy == buf.getLength()) { + buf.setLastByteConsumed(true); + } + buf.setAnyByteConsumed(true); + + buf.endReading(); // atomic decrement of refCount + return lengthToCopy; + } + + private ReadBuffer getBufferFromCompletedQueue(final String eTag, final long requestedOffset) { + for (ReadBuffer buffer : getCompletedReadList()) { + // Buffer is returned if the requestedOffset is at or above buffer's + // offset but less than buffer's length or the actual requestedLength + if (eTag.equals(buffer.getETag()) + && (requestedOffset >= buffer.getOffset()) + && ((requestedOffset < buffer.getOffset() + buffer.getLength()) + || (requestedOffset < buffer.getOffset() + buffer.getRequestedLength()))) { + return buffer; + } + } + return null; + } + + private boolean tryMemoryUpscale() { + if (!isDynamicScalingEnabled) { + return false; // Dynamic scaling is disabled, so no upscaling. + } + MemoryMXBean osBean = ManagementFactory.getMemoryMXBean(); + MemoryUsage memoryUsage = osBean.getHeapMemoryUsage(); + double memoryLoad = (double) memoryUsage.getUsed() / memoryUsage.getMax(); + if (memoryLoad < memoryThreshold && numberOfActiveBuffers < maxBufferPoolSize) { + // Create and Add more buffers in getFreeList(). + bufferPool[numberOfActiveBuffers] = new byte[getReadAheadBlockSize()]; + getFreeList().add(numberOfActiveBuffers); + numberOfActiveBuffers++; + printTraceLog("Current Memory Usage: {}. Incrementing buffer pool size to {}", memoryUsage, numberOfActiveBuffers); + return true; + } + printTraceLog("Could not Upscale memory. Total buffers: {} Memory Usage: {}", + numberOfActiveBuffers, memoryUsage); + return false; + } + + private void scheduledEviction() { + for (ReadBuffer buf : getCompletedReadList()) { + if (currentTimeMillis() - buf.getTimeStamp() > getThresholdAgeMilliseconds()) { + // If the buffer is older than thresholdAge, evict it. + printTraceLog("Scheduled Eviction of Buffer Triggered for BufferIndex: {}, file: {}, with eTag: {}, offset: {}, length: {}, queued by stream: {}", + buf.getBufferindex(), buf.getPath(), buf.getETag(), buf.getOffset(), buf.getLength(), buf.getStream().hashCode()); + evict(buf); + } + } + } + + private boolean manualEviction(final ReadBuffer buf) { + printTraceLog("Manual Eviction of Buffer Triggered for BufferIndex: {}, file: {}, with eTag: {}, offset: {}, queued by stream: {}", + buf.getBufferindex(), buf.getPath(), buf.getETag(), buf.getOffset(), buf.getStream().hashCode()); + return evict(buf); } + + private void adjustThreadPool() { + OperatingSystemMXBean osBean = ManagementFactory.getPlatformMXBean( + OperatingSystemMXBean.class); + double cpuLoad = osBean.getSystemCpuLoad(); + int currentPoolSize = workerRefs.size(); + int requiredPoolSize = (int) Math.ceil(1.2 * (getReadAheadQueue().size() + getInProgressList().size())); // 20% more for buffer + int newThreadPoolSize; + printTraceLog("Current CPU load: {}, Current worker pool size: {}, Current queue size: {}", cpuLoad, currentPoolSize, requiredPoolSize); + if (currentPoolSize < requiredPoolSize && cpuLoad < cpuThreshold) { + // Submit more background tasks. + newThreadPoolSize = Math.min(maxThreadPoolSize, + (int) Math.ceil((currentPoolSize * (ONE_HUNDRED + threadPoolUpscalePercentage))/ONE_HUNDRED)); + // Create new Worker Threads + for (int i = currentPoolSize; i < newThreadPoolSize; i++) { + ReadBufferWorker worker = new ReadBufferWorker(i, getBufferManager()); + workerRefs.add(worker); + workerPool.submit(worker); + } + printTraceLog("Increased worker pool size from {} to {}", currentPoolSize, newThreadPoolSize); + } else if (cpuLoad > cpuThreshold || currentPoolSize > requiredPoolSize) { + newThreadPoolSize = Math.max(minThreadPoolSize, + (int) Math.ceil((currentPoolSize * (ONE_HUNDRED - threadPoolDownscalePercentage))/ONE_HUNDRED)); + // Signal the extra workers to stop + while (workerRefs.size() > newThreadPoolSize) { + ReadBufferWorker worker = workerRefs.remove(workerRefs.size() - 1); + worker.stop(); + } + printTraceLog("Decreased worker pool size from {} to {}", currentPoolSize, newThreadPoolSize); + } else { + printTraceLog("No change in worker pool size. CPU load: {} Pool size: {}", cpuLoad, currentPoolSize); + } + } + /** - * {@inheritDoc} + * Similar to System.currentTimeMillis, except implemented with System.nanoTime(). + * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization), + * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing per CPU core. + * Note: it is not monotonic across Sockets, and even within a CPU, its only the + * more recent parts which share a clock across all cores. + * + * @return current time in milliseconds */ - @VisibleForTesting - @Override - public void callTryEvict() { - // TODO: To be implemented + private long currentTimeMillis() { + return System.nanoTime() / 1000 / 1000; + } + + private void purgeList(AbfsInputStream stream, LinkedList list) { + for (Iterator it = list.iterator(); it.hasNext();) { + ReadBuffer readBuffer = it.next(); + if (readBuffer.getStream() == stream) { + it.remove(); + // As failed ReadBuffers (bufferIndex = -1) are already pushed to free + // list in doneReading method, we will skip adding those here again. + if (readBuffer.getBufferindex() != -1) { + getFreeList().push(readBuffer.getBufferindex()); + } + } + } } /** - * {@inheritDoc} + * Test method that can clean up the current state of readAhead buffers and + * the lists. Will also trigger a fresh init. */ @VisibleForTesting @Override public void testResetReadBufferManager() { - // TODO: To be implemented + synchronized (this) { + ArrayList completedBuffers = new ArrayList<>(); + for (ReadBuffer buf : getCompletedReadList()) { + if (buf != null) { + completedBuffers.add(buf); + } + } + + for (ReadBuffer buf : completedBuffers) { + manualEviction(buf); + } + + getReadAheadQueue().clear(); + getInProgressList().clear(); + getCompletedReadList().clear(); + getFreeList().clear(); + for (int i = 0; i < maxBufferPoolSize; i++) { + bufferPool[i] = null; + } + bufferPool = null; + resetBufferManager(); + } } - /** - * {@inheritDoc} - */ @VisibleForTesting @Override - public void testResetReadBufferManager(final int readAheadBlockSize, - final int thresholdAgeMilliseconds) { - // TODO: To be implemented + public void testResetReadBufferManager(int readAheadBlockSize, int thresholdAgeMilliseconds) { + setReadAheadBlockSize(readAheadBlockSize); + setThresholdAgeMilliseconds(thresholdAgeMilliseconds); + testResetReadBufferManager(); + } + + @VisibleForTesting + public void callTryEvict() { + tryEvict(); + } + + + @VisibleForTesting + public int getNumBuffers() { + return numberOfActiveBuffers; } - /** - * {@inheritDoc} - */ @Override - public void testMimicFullUseAndAddFailedBuffer(final ReadBuffer buf) { - // TODO: To be implemented + void resetBufferManager() { + setBufferManager(null); // reset the singleton instance + } + + private static void setBufferManager(ReadBufferManagerV2 manager) { + bufferManager = manager; } private final ThreadFactory namedThreadFactory = new ThreadFactory() { @@ -217,12 +719,13 @@ public Thread newThread(Runnable r) { } }; - @Override - void resetBufferManager() { - setBufferManager(null); // reset the singleton instance + private void printTraceLog(String message, Object... args) { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace(message, args); + } } - private static void setBufferManager(ReadBufferManagerV2 manager) { - bufferManager = manager; + private void printDebugLog(String message, Object... args) { + LOGGER.debug(message, args); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java index 79d5eef955a4a..2c6efdc735aeb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java @@ -20,7 +20,9 @@ import java.io.IOException; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; @@ -29,6 +31,7 @@ class ReadBufferWorker implements Runnable { protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1); private int id; private ReadBufferManager bufferManager; + private AtomicBoolean isRunning = new AtomicBoolean(true); ReadBufferWorker(final int id, final ReadBufferManager bufferManager) { this.id = id; @@ -54,7 +57,7 @@ public void run() { Thread.currentThread().interrupt(); } ReadBuffer buffer; - while (true) { + while (isRunning()) { try { buffer = bufferManager.getNextBlockToRead(); // blocks, until a buffer is available for this thread } catch (InterruptedException ex) { @@ -72,7 +75,7 @@ public void run() { // read-ahead buffer size, make sure a valid length is passed // for remote read Math.min(buffer.getRequestedLength(), buffer.getBuffer().length), - buffer.getTracingContext()); + buffer.getTracingContext()); bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead); // post result back to ReadBufferManager } catch (IOException ex) { @@ -85,4 +88,13 @@ public void run() { } } } + + public void stop() { + isRunning.set(false); + } + + @VisibleForTesting + public boolean isRunning() { + return isRunning.get(); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java index b70f36de31867..69f4983a41fb1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java @@ -62,127 +62,138 @@ public class ITestReadBufferManager extends AbstractAbfsIntegrationTest { */ public static final int PROBE_INTERVAL_MILLIS = 1_000; - public ITestReadBufferManager() throws Exception { + public ITestReadBufferManager() throws Exception { + } + + @Test + public void testPurgeBufferManagerForParallelStreams() throws Exception { + describe("Testing purging of buffers from ReadBufferManagerV1 for " + + "parallel input streams"); + final int numBuffers = 16; + final LinkedList freeList = new LinkedList<>(); + for (int i=0; i < numBuffers; i++) { + freeList.add(i); } - - @Test - public void testPurgeBufferManagerForParallelStreams() throws Exception { - describe("Testing purging of buffers from ReadBufferManagerV1 for " - + "parallel input streams"); - final int numBuffers = 16; - final LinkedList freeList = new LinkedList<>(); - for (int i=0; i < numBuffers; i++) { - freeList.add(i); - } - ExecutorService executorService = Executors.newFixedThreadPool(4); - AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); - // verify that the fs has the capability to validate the fix - Assertions.assertThat(fs.hasPathCapability(new Path("/"), CAPABILITY_SAFE_READAHEAD)) - .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) - .isTrue(); - - try { - for (int i = 0; i < 4; i++) { - final String fileName = methodName.getMethodName() + i; - executorService.submit((Callable) () -> { - byte[] fileContent = getRandomBytesArray(ONE_MB); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - try (FSDataInputStream iStream = fs.open(testFilePath)) { - iStream.read(); - } - return null; - }); - } - } finally { - executorService.shutdown(); - // wait for all tasks to finish - executorService.awaitTermination(1, TimeUnit.MINUTES); - } - - ReadBufferManagerV1 bufferManager = ReadBufferManagerV1.getBufferManager(); - // readahead queue is empty - assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); - // verify the in progress list eventually empties out. - eventually(getTestTimeoutMillis() - TIMEOUT_OFFSET, PROBE_INTERVAL_MILLIS, () -> - assertListEmpty("InProgressList", bufferManager.getInProgressCopiedList())); + ExecutorService executorService = Executors.newFixedThreadPool(4); + AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); + // verify that the fs has the capability to validate the fix + Assertions.assertThat(fs.hasPathCapability(new Path("/"), CAPABILITY_SAFE_READAHEAD)) + .describedAs("path capability %s in %s", CAPABILITY_SAFE_READAHEAD, fs) + .isTrue(); + + try { + for (int i = 0; i < 4; i++) { + final String fileName = methodName.getMethodName() + i; + executorService.submit((Callable) () -> { + byte[] fileContent = getRandomBytesArray(ONE_MB); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); + try (FSDataInputStream iStream = fs.open(testFilePath)) { + iStream.read(); + } + return null; + }); + } + } finally { + executorService.shutdown(); + // wait for all tasks to finish + executorService.awaitTermination(1, TimeUnit.MINUTES); } - private void assertListEmpty(String listName, List list) { - Assertions.assertThat(list) - .describedAs("After closing all streams %s should be empty", listName) - .hasSize(0); + ReadBufferManager bufferManager = getBufferManager(fs); + // readahead queue is empty + assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); + // verify the in progress list eventually empties out. + eventually(getTestTimeoutMillis() - TIMEOUT_OFFSET, PROBE_INTERVAL_MILLIS, () -> + assertListEmpty("InProgressList", bufferManager.getInProgressListCopy())); + } + + private void assertListEmpty(String listName, List list) { + Assertions.assertThat(list) + .describedAs("After closing all streams %s should be empty", listName) + .hasSize(0); + } + + @Test + public void testPurgeBufferManagerForSequentialStream() throws Exception { + describe("Testing purging of buffers in ReadBufferManagerV1 for " + + "sequential input streams"); + AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); + final String fileName = methodName.getMethodName(); + byte[] fileContent = getRandomBytesArray(ONE_MB); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); + + AbfsInputStream iStream1 = null; + // stream1 will be closed right away. + try { + iStream1 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream(); + // Just reading one byte will trigger all read ahead calls. + iStream1.read(); + } finally { + IOUtils.closeStream(iStream1); } - - @Test - public void testPurgeBufferManagerForSequentialStream() throws Exception { - describe("Testing purging of buffers in ReadBufferManagerV1 for " - + "sequential input streams"); - AzureBlobFileSystem fs = getABFSWithReadAheadConfig(); - final String fileName = methodName.getMethodName(); - byte[] fileContent = getRandomBytesArray(ONE_MB); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - - AbfsInputStream iStream1 = null; - // stream1 will be closed right away. - try { - iStream1 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream(); - // Just reading one byte will trigger all read ahead calls. - iStream1.read(); - } finally { - IOUtils.closeStream(iStream1); - } - ReadBufferManagerV1 bufferManager = ReadBufferManagerV1.getBufferManager(); - AbfsInputStream iStream2 = null; - try { - iStream2 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream(); - iStream2.read(); - // After closing stream1, no queued buffers of stream1 should be present - // assertions can't be made about the state of the other lists as it is - // too prone to race conditions. - assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream1); - } finally { - // closing the stream later. - IOUtils.closeStream(iStream2); - } - // After closing stream2, no queued buffers of stream2 should be present. - assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream2); - - // After closing both the streams, read queue should be empty. - assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); - + ReadBufferManager bufferManager = getBufferManager(fs); + AbfsInputStream iStream2 = null; + try { + iStream2 = (AbfsInputStream) fs.open(testFilePath).getWrappedStream(); + iStream2.read(); + // After closing stream1, no queued buffers of stream1 should be present + // assertions can't be made about the state of the other lists as it is + // too prone to race conditions. + assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream1); + } finally { + // closing the stream later. + IOUtils.closeStream(iStream2); } + // After closing stream2, no queued buffers of stream2 should be present. + assertListDoesnotContainBuffersForIstream(bufferManager.getReadAheadQueueCopy(), iStream2); + // After closing both the streams, read queue should be empty. + assertListEmpty("ReadAheadQueue", bufferManager.getReadAheadQueueCopy()); - private void assertListDoesnotContainBuffersForIstream(List list, - AbfsInputStream inputStream) { - for (ReadBuffer buffer : list) { - Assertions.assertThat(buffer.getStream()) - .describedAs("Buffers associated with closed input streams shouldn't be present") - .isNotEqualTo(inputStream); - } - } + } - private AzureBlobFileSystem getABFSWithReadAheadConfig() throws Exception { - Configuration conf = getRawConfiguration(); - conf.setLong(FS_AZURE_READ_AHEAD_QUEUE_DEPTH, 8); - conf.setInt(AZURE_READ_BUFFER_SIZE, MIN_BUFFER_SIZE); - conf.setInt(FS_AZURE_READ_AHEAD_BLOCK_SIZE, MIN_BUFFER_SIZE); - return (AzureBlobFileSystem) FileSystem.newInstance(conf); - } - protected byte[] getRandomBytesArray(int length) { - final byte[] b = new byte[length]; - new Random().nextBytes(b); - return b; + private void assertListDoesnotContainBuffersForIstream(List list, + AbfsInputStream inputStream) { + for (ReadBuffer buffer : list) { + Assertions.assertThat(buffer.getStream()) + .describedAs("Buffers associated with closed input streams shouldn't be present") + .isNotEqualTo(inputStream); } - - protected Path createFileWithContent(FileSystem fs, String fileName, - byte[] fileContent) throws IOException { - Path testFilePath = path(fileName); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - oStream.flush(); - } - return testFilePath; + } + + private AzureBlobFileSystem getABFSWithReadAheadConfig() throws Exception { + Configuration conf = getRawConfiguration(); + conf.setLong(FS_AZURE_READ_AHEAD_QUEUE_DEPTH, 8); + conf.setInt(AZURE_READ_BUFFER_SIZE, MIN_BUFFER_SIZE); + conf.setInt(FS_AZURE_READ_AHEAD_BLOCK_SIZE, MIN_BUFFER_SIZE); + return (AzureBlobFileSystem) FileSystem.newInstance(conf); + } + + protected byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + protected Path createFileWithContent(FileSystem fs, String fileName, + byte[] fileContent) throws IOException { + Path testFilePath = path(fileName); + try (FSDataOutputStream oStream = fs.create(testFilePath)) { + oStream.write(fileContent); + oStream.flush(); + } + return testFilePath; + } + + private ReadBufferManager getBufferManager(AzureBlobFileSystem fs) { + int blockSize = fs.getAbfsStore().getAbfsConfiguration().getReadAheadBlockSize(); + if (getConfiguration().isReadAheadV2Enabled()) { + ReadBufferManagerV2.setReadBufferManagerConfigs(blockSize, + getConfiguration()); + return ReadBufferManagerV2.getBufferManager(); } + ReadBufferManagerV1.setReadBufferManagerConfigs(blockSize); + return ReadBufferManagerV1.getBufferManager(); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java new file mode 100644 index 0000000000000..636fc2ea02aea --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -0,0 +1,148 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import java.io.IOException; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; + +public class ITestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { + + private static final String TEST_FILE_NAME_PREFIX = "testFile"; + private static final int LESS_NUM_FILES = 5; + private static final int MORE_NUM_FILES = 10; + private static final int SMALL_FILE_SIZE = 30 * ONE_MB; + private static final int LARGE_FILE_SIZE = 200 * ONE_MB; + + public ITestReadBufferManagerV2() throws Exception { + super(); + getConfiguration().set(FS_AZURE_ENABLE_READAHEAD_V2, "true"); + } + + @Test + public void testReadBufferManagerV2() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + Path[] testPaths = createFilesWithContent(fs, TEST_FILE_NAME_PREFIX, + LESS_NUM_FILES, SMALL_FILE_SIZE); + ExecutorService executorService = Executors.newFixedThreadPool(LESS_NUM_FILES); + + int[] fileIdx = new int[1]; + try { + for (int i = 0; i < LESS_NUM_FILES; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = fs.open(testPaths[fileIdx[0]++])) { + int bytesRead = iStream.read(new byte[SMALL_FILE_SIZE], 0, SMALL_FILE_SIZE); + Assertions.assertEquals(SMALL_FILE_SIZE, bytesRead, + "Read size should match file size"); + } + return null; + }); + } + } catch(Exception e) { + System.out.println("Exception occurred during file read: " + e.getMessage()); + } finally { + executorService.shutdown(); + // wait for all tasks to finish + executorService.awaitTermination(1, TimeUnit.MINUTES); + } + } + + + /** + * Test to verify that multiple input streams can read the same file. + * With read ahead v2 enabled, multiple input stream read the same cached buffer + * based on file eTag. + * @throws Exception if any error occurs during the test + */ + @Test + public void testMultipleInputStreamReadingSameFile() throws Exception { + AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(spiedStore.getClient()); + Mockito.doReturn(spiedClient).when(spiedStore).getClient(); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + + int fileSize = SMALL_FILE_SIZE; + int numOfFile = LESS_NUM_FILES; + Path[] testPaths = createFilesWithContent(spiedFs, TEST_FILE_NAME_PREFIX, + 1, fileSize); + Path testPath = testPaths[0]; + ExecutorService executorService = Executors.newFixedThreadPool(numOfFile); + + try { + for (int i = 0; i < LESS_NUM_FILES; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = spiedFs.open(testPath)) { + int bytesRead = iStream.read(new byte[LARGE_FILE_SIZE], 0, LARGE_FILE_SIZE); + Assertions.assertEquals(LARGE_FILE_SIZE, bytesRead, + "Read size should match file size"); + } + return null; + }); + } + } catch(Exception e) { + System.out.println("Exception occurred during file read: " + e.getMessage()); + } finally { + executorService.shutdown(); + // wait for all tasks to finish + executorService.awaitTermination(1, TimeUnit.MINUTES); + } + } + + private byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + private Path createFileWithContent(FileSystem fs, String fileName, + byte[] fileContent) throws + IOException { + Path testFilePath = path(fileName); + try (FSDataOutputStream oStream = fs.create(testFilePath)) { + oStream.write(fileContent); + oStream.flush(); + } + return testFilePath; + } + + private Path[] createFilesWithContent(FileSystem fs, + String fileNamePrefix, + int numFiles, + int fileSize) throws Exception { + ExecutorService executorService = Executors.newFixedThreadPool(numFiles); + Path[] tesFilePaths = new Path[numFiles]; + int[] fileIdx = new int[1]; + try { + for (int i = 0; i < numFiles; i++) { + final String fileName = fileNamePrefix + i; + executorService.submit((Callable) () -> { + byte[] fileContent = getRandomBytesArray(fileSize); + tesFilePaths[fileIdx[0]++] = createFileWithContent(fs, fileName, fileContent); + return null; + }); + } + } finally { + executorService.shutdown(); + // wait for all tasks to finish + executorService.awaitTermination(1, TimeUnit.MINUTES); + } + return tesFilePaths; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index de49da5dc51d2..b90bf63975542 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -546,7 +546,7 @@ public void testStreamPurgeDuringReadAheadCallExecuting() throws Exception { //Sleeping to give ReadBufferWorker to pick the readBuffers for processing. Thread.sleep(readBufferTransferToInProgressProbableTime); - Assertions.assertThat(readBufferManager.getInProgressCopiedList()) + Assertions.assertThat(readBufferManager.getInProgressListCopy()) .describedAs(String.format("InProgressList should have %d elements", readBufferQueuedCount)) .hasSize(readBufferQueuedCount); @@ -559,7 +559,7 @@ public void testStreamPurgeDuringReadAheadCallExecuting() throws Exception { .hasSize(0); } - Assertions.assertThat(readBufferManager.getInProgressCopiedList()) + Assertions.assertThat(readBufferManager.getInProgressListCopy()) .describedAs(String.format("InProgressList should have %d elements", readBufferQueuedCount)) .hasSize(readBufferQueuedCount); From 28cb97fde1eda2fa096401499e9ee1dbccbdef2b Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Mon, 28 Jul 2025 21:59:53 -0700 Subject: [PATCH 02/13] Test Improvements --- .../services/ReadBufferManagerV2.java | 9 + .../services/ITestReadBufferManagerV2.java | 181 ++++++++++++------ 2 files changed, 128 insertions(+), 62 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 0f80beac32b87..063f14c9ab87b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -584,6 +584,15 @@ private void scheduledEviction() { evict(buf); } } + + /* + * Scheduled eviction does not clear up acquired memory, it only marks buffer + * to be available and reused for future read ahead requests. + + * To actually free up memory, we need to remove the extra buffer from all + * the queues and available list and nullify its reference in bufferPool. + */ + // TODO: Check Memory Utilization and adjust buffer pool size if needed. } private boolean manualEviction(final ReadBuffer buf) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java index 636fc2ea02aea..7b73e7630c8ca 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -1,6 +1,8 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -11,6 +13,7 @@ import org.junit.jupiter.api.Assertions; import org.mockito.Mockito; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -20,50 +23,62 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_BLOCK_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.verify; +/** + * Test class for ReadBufferManagerV2 functionality. + */ public class ITestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { private static final String TEST_FILE_NAME_PREFIX = "testFile"; private static final int LESS_NUM_FILES = 5; private static final int MORE_NUM_FILES = 10; private static final int SMALL_FILE_SIZE = 30 * ONE_MB; - private static final int LARGE_FILE_SIZE = 200 * ONE_MB; + private static final int LARGE_FILE_SIZE = 100 * ONE_MB; public ITestReadBufferManagerV2() throws Exception { super(); - getConfiguration().set(FS_AZURE_ENABLE_READAHEAD_V2, "true"); } + /** + * Test to verify that ReadBufferManagerV2 can read different files concurrently. + * @throws Exception if any error occurs during the test + */ @Test public void testReadBufferManagerV2() throws Exception { - AzureBlobFileSystem fs = getFileSystem(); - Path[] testPaths = createFilesWithContent(fs, TEST_FILE_NAME_PREFIX, - LESS_NUM_FILES, SMALL_FILE_SIZE); - ExecutorService executorService = Executors.newFixedThreadPool(LESS_NUM_FILES); - - int[] fileIdx = new int[1]; - try { - for (int i = 0; i < LESS_NUM_FILES; i++) { - executorService.submit((Callable) () -> { - try (FSDataInputStream iStream = fs.open(testPaths[fileIdx[0]++])) { - int bytesRead = iStream.read(new byte[SMALL_FILE_SIZE], 0, SMALL_FILE_SIZE); - Assertions.assertEquals(SMALL_FILE_SIZE, bytesRead, - "Read size should match file size"); - } - return null; - }); + Map configMap = new HashMap<>(); + configMap.put(FS_AZURE_READ_AHEAD_BLOCK_SIZE, String.valueOf(4 * ONE_MB)); + try (AzureBlobFileSystem fs = getConfiguredFileSystem(false, configMap)) { + int numOfFiles = LESS_NUM_FILES; + Path[] testPaths = createFilesWithContent(fs, numOfFiles); + ExecutorService executorService = Executors.newFixedThreadPool( + numOfFiles); + + int[] fileIdx = new int[1]; + try { + for (int i = 0; i < numOfFiles; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = fs.open(testPaths[fileIdx[0]++])) { + int bytesRead = iStream.read(new byte[SMALL_FILE_SIZE], 0, + SMALL_FILE_SIZE); + Assertions.assertEquals(SMALL_FILE_SIZE, bytesRead, + "Read size should match file size"); + } + return null; + }); + } + } catch (Exception e) { + System.out.println( + "Exception occurred during file read: " + e.getMessage()); + } finally { + shutdownExecutorService(executorService); } - } catch(Exception e) { - System.out.println("Exception occurred during file read: " + e.getMessage()); - } finally { - executorService.shutdown(); - // wait for all tasks to finish - executorService.awaitTermination(1, TimeUnit.MINUTES); } } - /** * Test to verify that multiple input streams can read the same file. * With read ahead v2 enabled, multiple input stream read the same cached buffer @@ -72,39 +87,71 @@ public void testReadBufferManagerV2() throws Exception { */ @Test public void testMultipleInputStreamReadingSameFile() throws Exception { - AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem()); - AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); - AbfsClient spiedClient = Mockito.spy(spiedStore.getClient()); - Mockito.doReturn(spiedClient).when(spiedStore).getClient(); - Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); - - int fileSize = SMALL_FILE_SIZE; - int numOfFile = LESS_NUM_FILES; - Path[] testPaths = createFilesWithContent(spiedFs, TEST_FILE_NAME_PREFIX, - 1, fileSize); - Path testPath = testPaths[0]; - ExecutorService executorService = Executors.newFixedThreadPool(numOfFile); - - try { - for (int i = 0; i < LESS_NUM_FILES; i++) { - executorService.submit((Callable) () -> { - try (FSDataInputStream iStream = spiedFs.open(testPath)) { - int bytesRead = iStream.read(new byte[LARGE_FILE_SIZE], 0, LARGE_FILE_SIZE); - Assertions.assertEquals(LARGE_FILE_SIZE, bytesRead, - "Read size should match file size"); - } - return null; - }); + Map configMap = new HashMap<>(); + try(AzureBlobFileSystem spiedFs = Mockito.spy(getConfiguredFileSystem(true, configMap))) { + AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(spiedStore.getClient()); + Mockito.doReturn(spiedClient).when(spiedStore).getClient(); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + int numOfFiles = MORE_NUM_FILES; + int fileSize = SMALL_FILE_SIZE; + int blockSize = 4 * ONE_MB; + + Path[] testPaths = createFilesWithContent(spiedFs, 1); + Path testPath = testPaths[0]; + ExecutorService executorService = Executors.newFixedThreadPool( + LESS_NUM_FILES); + + try { + for (int i = 0; i < LESS_NUM_FILES; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = spiedFs.open(testPath)) { + int bytesRead = iStream.read(new byte[LARGE_FILE_SIZE], 0, + LARGE_FILE_SIZE); + Assertions.assertEquals(LARGE_FILE_SIZE, bytesRead, + "Read size should match file size"); + } + return null; + }); + } + } finally { + shutdownExecutorService(executorService); } - } catch(Exception e) { - System.out.println("Exception occurred during file read: " + e.getMessage()); - } finally { - executorService.shutdown(); - // wait for all tasks to finish - executorService.awaitTermination(1, TimeUnit.MINUTES); + + int leastReadOpnCount = (numOfFiles * fileSize) / blockSize; + + verify(spiedClient, Mockito.atLeast(leastReadOpnCount)) + .read(eq(testPath.toString()), Mockito.anyLong(), Mockito.any(), + Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), + Mockito.anyString(), Mockito.any(), Mockito.any()); + verify(spiedClient, Mockito.atMost(2 * leastReadOpnCount)) + .read(eq(testPath.toString()), Mockito.anyLong(), Mockito.any(), + Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), + Mockito.anyString(), Mockito.any(), Mockito.any()); } } + /** + * Test to verify that scheduled eviction of completed buffers happens. + * This test will be implemented in the future. + * @throws Exception if any error occurs during the test + */ + @Test + public void testScheduledEviction() throws Exception { + + } + + private AzureBlobFileSystem getConfiguredFileSystem(boolean isRAV2Enabled, + Map configurations) throws IOException { + Configuration conf = getConfiguration().getRawConfiguration(); + conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2, isRAV2Enabled); + for (Map.Entry entry : configurations.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + + return (AzureBlobFileSystem) FileSystem.newInstance(conf); + } + private byte[] getRandomBytesArray(int length) { final byte[] b = new byte[length]; new Random().nextBytes(b); @@ -123,26 +170,36 @@ private Path createFileWithContent(FileSystem fs, String fileName, } private Path[] createFilesWithContent(FileSystem fs, - String fileNamePrefix, - int numFiles, - int fileSize) throws Exception { + int numFiles) throws Exception { ExecutorService executorService = Executors.newFixedThreadPool(numFiles); Path[] tesFilePaths = new Path[numFiles]; int[] fileIdx = new int[1]; try { for (int i = 0; i < numFiles; i++) { - final String fileName = fileNamePrefix + i; + final String fileName = ITestReadBufferManagerV2.TEST_FILE_NAME_PREFIX + i; executorService.submit((Callable) () -> { - byte[] fileContent = getRandomBytesArray(fileSize); + byte[] fileContent = getRandomBytesArray( + ITestReadBufferManagerV2.SMALL_FILE_SIZE); tesFilePaths[fileIdx[0]++] = createFileWithContent(fs, fileName, fileContent); return null; }); } } finally { - executorService.shutdown(); - // wait for all tasks to finish - executorService.awaitTermination(1, TimeUnit.MINUTES); + shutdownExecutorService(executorService); } return tesFilePaths; } + + private void shutdownExecutorService(ExecutorService executorService) { + if (executorService != null && !executorService.isShutdown()) { + executorService.shutdown(); + try { + if (!executorService.awaitTermination(1, TimeUnit.MINUTES)) { + executorService.shutdownNow(); + } + } catch (InterruptedException e) { + executorService.shutdownNow(); + } + } + } } From 3ee60df95f7b094bcb3b3b8aab26d090b6baaa1f Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Thu, 31 Jul 2025 05:54:17 -0700 Subject: [PATCH 03/13] Memory Downscale --- .../constants/FileSystemConfigurations.java | 2 +- .../fs/azurebfs/services/AbfsInputStream.java | 4 + .../services/ReadBufferManagerV1.java | 2 +- .../services/ReadBufferManagerV2.java | 77 +++++++++++++------ 4 files changed, 59 insertions(+), 26 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 82c58b41c9fb8..fab5491e4fe6c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -137,7 +137,7 @@ public final class FileSystemConfigurations { public static final int DEFAULT_READAHEAD_V2_THREAD_POOL_UPSCALE_PERCENTAGE = 20; public static final int DEFAULT_READAHEAD_V2_THREAD_POOL_DOWNSCALE_PERCENTAGE = 30; public static final int DEFAULT_READAHEAD_V2_MEMORY_MONITORING_INTERVAL_MILLIS = 6_000; - public static final int DEFAULT_READAHEAD_V2_EXECUTOR_SERVICE_TTL_MILLIS = 3_000; + public static final int DEFAULT_READAHEAD_V2_EXECUTOR_SERVICE_TTL_MILLIS = 6_000; public static final int DEFAULT_READAHEAD_V2_CACHED_BUFFER_TTL_MILLIS = 6_000; public static final int DEFAULT_READAHEAD_V2_CPU_USAGE_THRESHOLD_PERCENT = 50; public static final int DEFAULT_READAHEAD_V2_MEMORY_USAGE_THRESHOLD_PERCENT = 50; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 5adb350991b23..6c68ec0fd5250 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -914,6 +914,10 @@ long getLimit() { return this.limit; } + boolean isFirstRead() { + return this.firstRead; + } + @VisibleForTesting BackReference getFsBackRef() { return fsBackRef; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java index fe1ac3fa1f235..8d8e2244aa9df 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java @@ -88,7 +88,7 @@ static ReadBufferManagerV1 getBufferManager() { void init() { buffers = new byte[NUM_BUFFERS][]; for (int i = 0; i < NUM_BUFFERS; i++) { - buffers[i] = new byte[getReadAheadBlockSize()]; // same buffers are reused. These byte arrays are never garbage collected + buffers[i] = new byte[getReadAheadBlockSize()]; // same buffers are reused. The byte array never goes back to GC getFreeList().add(i); } for (int i = 0; i < NUM_THREADS; i++) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 063f14c9ab87b..268ef4332d437 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -79,6 +79,7 @@ final class ReadBufferManagerV2 extends ReadBufferManager { private static double memoryThreshold; private int numberOfActiveBuffers = 0; private byte[][] bufferPool; + private Stack removedBufferList = new Stack<>(); // indices in buffers[] array that are available // Buffer Manager Structures private static ReadBufferManagerV2 bufferManager; @@ -255,9 +256,10 @@ public int getBlock(final AbfsInputStream stream, final long position, final int stream.getPath(), stream.getETag(), position, length, stream.hashCode()); String requestedETag = stream.getETag(); + boolean isFirstRead = stream.isFirstRead(); // Wait for any in-progress read to complete. - waitForProcess(requestedETag, position); + waitForProcess(requestedETag, position, isFirstRead); int bytesRead = 0; synchronized (this) { @@ -457,7 +459,9 @@ private boolean evict(final ReadBuffer buf) { // As failed ReadBuffers (bufferIndx = -1) are saved in getCompletedReadList(), // avoid adding it to availableBufferList. if (buf.getBufferindex() != -1) { - getFreeList().push(buf.getBufferindex()); + synchronized (this) { + getFreeList().push(buf.getBufferindex()); + } } getCompletedReadList().remove(buf); buf.setTracingContext(null); @@ -466,10 +470,10 @@ private boolean evict(final ReadBuffer buf) { return true; } - private void waitForProcess(final String eTag, final long position) { + private void waitForProcess(final String eTag, final long position, boolean isFirstRead) { ReadBuffer readBuf; synchronized (this) { - clearFromReadAheadQueue(eTag, position); + clearFromReadAheadQueue(eTag, position, isFirstRead); readBuf = getFromList(getInProgressList(), eTag, position); } if (readBuf != null) { // if in in-progress queue, then block for it @@ -491,12 +495,18 @@ private void waitForProcess(final String eTag, final long position) { } } - private void clearFromReadAheadQueue(final String eTag, final long requestedOffset) { + private void clearFromReadAheadQueue(final String eTag, final long requestedOffset, boolean isFirstRead) { ReadBuffer buffer = getFromList(getReadAheadQueue(), eTag, requestedOffset); - if (buffer != null) { - getReadAheadQueue().remove(buffer); - notifyAll(); // lock is held in calling method - getFreeList().push(buffer.getBufferindex()); + /* + * If this prefetch was triggered by first read of this input stream, + * we should not remove it from queue and cache it for future purpose. + */ + if (buffer != null && !isFirstRead) { + synchronized (this) { + getReadAheadQueue().remove(buffer); + notifyAll(); // lock is held in calling method + getFreeList().push(buffer.getBufferindex()); + } } } @@ -555,8 +565,9 @@ private ReadBuffer getBufferFromCompletedQueue(final String eTag, final long req return null; } - private boolean tryMemoryUpscale() { + private synchronized boolean tryMemoryUpscale() { if (!isDynamicScalingEnabled) { + printTraceLog("Dynamic scaling is disabled, skipping memory upscale"); return false; // Dynamic scaling is disabled, so no upscaling. } MemoryMXBean osBean = ManagementFactory.getMemoryMXBean(); @@ -564,14 +575,26 @@ private boolean tryMemoryUpscale() { double memoryLoad = (double) memoryUsage.getUsed() / memoryUsage.getMax(); if (memoryLoad < memoryThreshold && numberOfActiveBuffers < maxBufferPoolSize) { // Create and Add more buffers in getFreeList(). - bufferPool[numberOfActiveBuffers] = new byte[getReadAheadBlockSize()]; - getFreeList().add(numberOfActiveBuffers); + if (removedBufferList.isEmpty()) { + bufferPool[numberOfActiveBuffers] = new byte[getReadAheadBlockSize()]; + getFreeList().add(numberOfActiveBuffers); + } else { + // Reuse a removed buffer index. + int freeIndex = removedBufferList.pop(); + if (freeIndex >= bufferPool.length) { + printTraceLog("Invalid free index: {}. Current buffer pool size: {}", + freeIndex, bufferPool.length); + return false; + } + bufferPool[freeIndex] = new byte[getReadAheadBlockSize()]; + getFreeList().add(freeIndex); + } numberOfActiveBuffers++; - printTraceLog("Current Memory Usage: {}. Incrementing buffer pool size to {}", memoryUsage, numberOfActiveBuffers); + printTraceLog("Current Memory Load: {}. Incrementing buffer pool size to {}", memoryLoad, numberOfActiveBuffers); return true; } - printTraceLog("Could not Upscale memory. Total buffers: {} Memory Usage: {}", - numberOfActiveBuffers, memoryUsage); + printTraceLog("Could not Upscale memory. Total buffers: {} Memory Load: {}", + numberOfActiveBuffers, memoryLoad); return false; } @@ -585,14 +608,18 @@ private void scheduledEviction() { } } - /* - * Scheduled eviction does not clear up acquired memory, it only marks buffer - * to be available and reused for future read ahead requests. - - * To actually free up memory, we need to remove the extra buffer from all - * the queues and available list and nullify its reference in bufferPool. - */ - // TODO: Check Memory Utilization and adjust buffer pool size if needed. + MemoryMXBean osBean = ManagementFactory.getMemoryMXBean(); + MemoryUsage memoryUsage = osBean.getHeapMemoryUsage(); + double memoryLoad = (double) memoryUsage.getUsed() / memoryUsage.getMax(); + if (memoryLoad > memoryThreshold) { + synchronized (this) { + int freeIndex = getFreeList().pop(); + bufferPool[freeIndex] = null; + removedBufferList.add(freeIndex); + numberOfActiveBuffers--; + } + printTraceLog("Current Memory Load: {}. Decrementing buffer pool size to {}", memoryLoad, numberOfActiveBuffers); + } } private boolean manualEviction(final ReadBuffer buf) { @@ -655,7 +682,9 @@ private void purgeList(AbfsInputStream stream, LinkedList list) { // As failed ReadBuffers (bufferIndex = -1) are already pushed to free // list in doneReading method, we will skip adding those here again. if (readBuffer.getBufferindex() != -1) { - getFreeList().push(readBuffer.getBufferindex()); + synchronized (this) { + getFreeList().push(readBuffer.getBufferindex()); + } } } } From 7670846f293f7f629cacdd968ccc1cbd9e70ff25 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Fri, 1 Aug 2025 08:06:02 -0700 Subject: [PATCH 04/13] Code Improv --- .../fs/azurebfs/services/ReadBufferManagerV2.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 268ef4332d437..fb4461b7e9bf8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -570,9 +570,7 @@ private synchronized boolean tryMemoryUpscale() { printTraceLog("Dynamic scaling is disabled, skipping memory upscale"); return false; // Dynamic scaling is disabled, so no upscaling. } - MemoryMXBean osBean = ManagementFactory.getMemoryMXBean(); - MemoryUsage memoryUsage = osBean.getHeapMemoryUsage(); - double memoryLoad = (double) memoryUsage.getUsed() / memoryUsage.getMax(); + double memoryLoad = getMemoryLoad(); if (memoryLoad < memoryThreshold && numberOfActiveBuffers < maxBufferPoolSize) { // Create and Add more buffers in getFreeList(). if (removedBufferList.isEmpty()) { @@ -608,9 +606,7 @@ private void scheduledEviction() { } } - MemoryMXBean osBean = ManagementFactory.getMemoryMXBean(); - MemoryUsage memoryUsage = osBean.getHeapMemoryUsage(); - double memoryLoad = (double) memoryUsage.getUsed() / memoryUsage.getMax(); + double memoryLoad = getMemoryLoad(); if (memoryLoad > memoryThreshold) { synchronized (this) { int freeIndex = getFreeList().pop(); @@ -766,4 +762,11 @@ private void printTraceLog(String message, Object... args) { private void printDebugLog(String message, Object... args) { LOGGER.debug(message, args); } + + @VisibleForTesting + double getMemoryLoad() { + MemoryMXBean osBean = ManagementFactory.getMemoryMXBean(); + MemoryUsage memoryUsage = osBean.getHeapMemoryUsage(); + return (double) memoryUsage.getUsed() / memoryUsage.getMax(); + } } From 6a686866e701665a2cb897afdf6470fe9b636ef5 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Sun, 3 Aug 2025 05:19:55 -0700 Subject: [PATCH 05/13] Close Executor Service --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 7 +++++++ .../azurebfs/services/ReadBufferManager.java | 6 ++++++ .../services/ReadBufferManagerV1.java | 11 +++++++++-- .../services/ReadBufferManagerV2.java | 19 ++++++++++++++++++- 4 files changed, 40 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 439b7626a86a7..7afbb7efd6557 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -110,6 +110,9 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.ListingSupport; +import org.apache.hadoop.fs.azurebfs.services.ReadBufferManager; +import org.apache.hadoop.fs.azurebfs.services.ReadBufferManagerV1; +import org.apache.hadoop.fs.azurebfs.services.ReadBufferManagerV2; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.azurebfs.services.StaticRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.VersionedFileStatus; @@ -324,6 +327,10 @@ public void close() throws IOException { HadoopExecutors.shutdown(boundedThreadPool, LOG, 30, TimeUnit.SECONDS); boundedThreadPool = null; + ReadBufferManagerV2 bufferManagerV2 = ReadBufferManagerV2.getBufferManager(); + ReadBufferManagerV1 bufferManagerV1 = ReadBufferManagerV1.getBufferManager(); + bufferManagerV1.close(); + bufferManagerV2.close(); } catch (InterruptedException e) { LOG.error("Interrupted freeing leases", e); Thread.currentThread().interrupt(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 9b1b108aac43f..51242b8f24972 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -119,6 +119,12 @@ abstract void doneReading(ReadBuffer buffer, */ abstract void purgeBuffersForStream(AbfsInputStream stream); + public void close() { + closeReadBufferManager(); + } + + abstract void closeReadBufferManager(); + // Following Methods are for testing purposes only and should not be used in production code. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java index 8d8e2244aa9df..f5462f05841e0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java @@ -33,7 +33,7 @@ * The Read Buffer Manager for Rest AbfsClient. * V1 implementation of ReadBufferManager. */ -final class ReadBufferManagerV1 extends ReadBufferManager { +public final class ReadBufferManagerV1 extends ReadBufferManager { private static final int NUM_BUFFERS = 16; private static final int NUM_THREADS = 8; @@ -66,7 +66,7 @@ static void setReadBufferManagerConfigs(int readAheadBlockSize) { * Returns the singleton instance of ReadBufferManagerV1. * @return the singleton instance of ReadBufferManagerV1 */ - static ReadBufferManagerV1 getBufferManager() { + public static ReadBufferManagerV1 getBufferManager() { if (bufferManager == null) { LOCK.lock(); try { @@ -542,6 +542,13 @@ private void purgeList(AbfsInputStream stream, LinkedList list) { } } + @Override + public void closeReadBufferManager() { + // no-op, as this is a singleton and should not be closed + // the buffers will be cleaned up when the JVM exits + LOGGER.debug("ReadBufferManagerV1 close called, but no action taken as it is a singleton."); + } + /** * {@inheritDoc} */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index fb4461b7e9bf8..e0215129281d9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -29,6 +29,7 @@ import java.lang.management.MemoryMXBean; import java.lang.management.MemoryUsage; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.LinkedList; @@ -47,6 +48,7 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.util.concurrent.HadoopExecutors; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_HUNDRED; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; @@ -54,7 +56,7 @@ /** * The Improved Read Buffer Manager for Rest AbfsClient. */ -final class ReadBufferManagerV2 extends ReadBufferManager { +public final class ReadBufferManagerV2 extends ReadBufferManager { // Internal constants private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManagerV2.class); private static final ReentrantLock LOCK = new ReentrantLock(); @@ -350,6 +352,21 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { purgeList(stream, getCompletedReadList()); } + @Override + public void closeReadBufferManager() { + printTraceLog("Closing ReadBufferManagerV2"); + HadoopExecutors.shutdown(workerPool, LOGGER, + 30, TimeUnit.SECONDS); + workerPool = null; + if (bufferPool != null) { + // help GC + Arrays.fill(bufferPool, null); + bufferPool = null; + } + setBufferManager(null); // reset the singleton instance + printTraceLog("ReadBufferManagerV2 closed"); + } + private boolean isAlreadyQueued(final String eTag, final long requestedOffset) { // returns true if any part of the buffer is already queued return (isInList(getReadAheadQueue(), eTag, requestedOffset) From 9552072e0e9e4addbb5f7ccb6f9109332fa47f2c Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Sun, 3 Aug 2025 06:42:31 -0700 Subject: [PATCH 06/13] Close Fix --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 7 +++---- .../hadoop/fs/azurebfs/services/ReadBufferManager.java | 7 ------- .../hadoop/fs/azurebfs/services/ReadBufferManagerV1.java | 7 ------- .../hadoop/fs/azurebfs/services/ReadBufferManagerV2.java | 4 +--- .../fs/azurebfs/services/ITestReadBufferManagerV2.java | 2 +- 5 files changed, 5 insertions(+), 22 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 7afbb7efd6557..a1241def37fb9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -327,10 +327,9 @@ public void close() throws IOException { HadoopExecutors.shutdown(boundedThreadPool, LOG, 30, TimeUnit.SECONDS); boundedThreadPool = null; - ReadBufferManagerV2 bufferManagerV2 = ReadBufferManagerV2.getBufferManager(); - ReadBufferManagerV1 bufferManagerV1 = ReadBufferManagerV1.getBufferManager(); - bufferManagerV1.close(); - bufferManagerV2.close(); + if (getAbfsConfiguration().isReadAheadV2Enabled()) { + ReadBufferManagerV2.getBufferManager().closeReadBufferManager(); + } } catch (InterruptedException e) { LOG.error("Interrupted freeing leases", e); Thread.currentThread().interrupt(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java index 51242b8f24972..d62f53cf83072 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java @@ -119,13 +119,6 @@ abstract void doneReading(ReadBuffer buffer, */ abstract void purgeBuffersForStream(AbfsInputStream stream); - public void close() { - closeReadBufferManager(); - } - - abstract void closeReadBufferManager(); - - // Following Methods are for testing purposes only and should not be used in production code. /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java index f5462f05841e0..389903df49700 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV1.java @@ -542,13 +542,6 @@ private void purgeList(AbfsInputStream stream, LinkedList list) { } } - @Override - public void closeReadBufferManager() { - // no-op, as this is a singleton and should not be closed - // the buffers will be cleaned up when the JVM exits - LOGGER.debug("ReadBufferManagerV1 close called, but no action taken as it is a singleton."); - } - /** * {@inheritDoc} */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index e0215129281d9..3d0c9363671cc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -352,11 +352,9 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { purgeList(stream, getCompletedReadList()); } - @Override public void closeReadBufferManager() { printTraceLog("Closing ReadBufferManagerV2"); - HadoopExecutors.shutdown(workerPool, LOGGER, - 30, TimeUnit.SECONDS); + workerPool.shutdownNow(); workerPool = null; if (bufferPool != null) { // help GC diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java index 7b73e7630c8ca..08933b0d97c66 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -51,7 +51,7 @@ public ITestReadBufferManagerV2() throws Exception { public void testReadBufferManagerV2() throws Exception { Map configMap = new HashMap<>(); configMap.put(FS_AZURE_READ_AHEAD_BLOCK_SIZE, String.valueOf(4 * ONE_MB)); - try (AzureBlobFileSystem fs = getConfiguredFileSystem(false, configMap)) { + try (AzureBlobFileSystem fs = getConfiguredFileSystem(true, configMap)) { int numOfFiles = LESS_NUM_FILES; Path[] testPaths = createFilesWithContent(fs, numOfFiles); ExecutorService executorService = Executors.newFixedThreadPool( From 0c368d836c420693f2467e8aeb52f25a5f53e1bd Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Mon, 4 Aug 2025 04:30:34 -0700 Subject: [PATCH 07/13] Close Fix --- hadoop-tools/hadoop-azure/pom.xml | 1 + .../fs/azurebfs/AzureBlobFileSystemStore.java | 5 +- .../services/ReadBufferManagerV2.java | 7 +- .../services/ITestReadBufferManagerV2.java | 77 +++++++++---------- 4 files changed, 48 insertions(+), 42 deletions(-) diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index 6c976f19f2428..c2fbace2775f0 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -29,6 +29,7 @@ Currently this consists of a filesystem client to read data from and write data to Azure Storage. + prefetchPR jar diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index a1241def37fb9..a25be97d96629 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -328,7 +328,10 @@ public void close() throws IOException { 30, TimeUnit.SECONDS); boundedThreadPool = null; if (getAbfsConfiguration().isReadAheadV2Enabled()) { - ReadBufferManagerV2.getBufferManager().closeReadBufferManager(); + ReadBufferManagerV2 bufferManagerV2 = ReadBufferManagerV2.getInstance(); + if (bufferManagerV2 != null) { + bufferManagerV2.close(); + } } } catch (InterruptedException e) { LOG.error("Interrupted freeing leases", e); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 3d0c9363671cc..537a8ba94afa5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -58,7 +58,6 @@ */ public final class ReadBufferManagerV2 extends ReadBufferManager { // Internal constants - private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManagerV2.class); private static final ReentrantLock LOCK = new ReentrantLock(); // Thread Pool Configurations @@ -109,6 +108,10 @@ public static ReadBufferManagerV2 getBufferManager() { return (ReadBufferManagerV2) bufferManager; } + public static ReadBufferManagerV2 getInstance() { + return bufferManager; + } + /** * Set the ReadBufferManagerV2 configurations based on the provided before singleton initialization. * @param abfsConfiguration the configuration to set for the ReadBufferManagerV2. @@ -352,7 +355,7 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { purgeList(stream, getCompletedReadList()); } - public void closeReadBufferManager() { + public void close() { printTraceLog("Closing ReadBufferManagerV2"); workerPool.shutdownNow(); workerPool = null; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java index 08933b0d97c66..da8ea3bc0f2f7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -41,6 +41,7 @@ public class ITestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { public ITestReadBufferManagerV2() throws Exception { super(); + getConfiguration().setBoolean(FS_AZURE_ENABLE_READAHEAD_V2, true); } /** @@ -87,48 +88,47 @@ public void testReadBufferManagerV2() throws Exception { */ @Test public void testMultipleInputStreamReadingSameFile() throws Exception { - Map configMap = new HashMap<>(); - try(AzureBlobFileSystem spiedFs = Mockito.spy(getConfiguredFileSystem(true, configMap))) { - AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); - AbfsClient spiedClient = Mockito.spy(spiedStore.getClient()); - Mockito.doReturn(spiedClient).when(spiedStore).getClient(); - Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); - int numOfFiles = MORE_NUM_FILES; - int fileSize = SMALL_FILE_SIZE; - int blockSize = 4 * ONE_MB; - - Path[] testPaths = createFilesWithContent(spiedFs, 1); - Path testPath = testPaths[0]; - ExecutorService executorService = Executors.newFixedThreadPool( - LESS_NUM_FILES); + AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem()); + AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); + AbfsClient spiedClient = Mockito.spy(spiedStore.getClient()); + Mockito.doReturn(spiedClient).when(spiedStore).getClient(); + Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); + int numOfFiles = MORE_NUM_FILES; + int fileSize = SMALL_FILE_SIZE; + int blockSize = 4 * ONE_MB; + + Path[] testPaths = createFilesWithContent(spiedFs, 1); + Path testPath = testPaths[0]; + ExecutorService executorService = Executors.newFixedThreadPool( + LESS_NUM_FILES); - try { - for (int i = 0; i < LESS_NUM_FILES; i++) { - executorService.submit((Callable) () -> { - try (FSDataInputStream iStream = spiedFs.open(testPath)) { - int bytesRead = iStream.read(new byte[LARGE_FILE_SIZE], 0, - LARGE_FILE_SIZE); - Assertions.assertEquals(LARGE_FILE_SIZE, bytesRead, - "Read size should match file size"); - } - return null; - }); - } - } finally { - shutdownExecutorService(executorService); + try { + for (int i = 0; i < LESS_NUM_FILES; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = spiedFs.open(testPath)) { + int bytesRead = iStream.read(new byte[LARGE_FILE_SIZE], 0, + LARGE_FILE_SIZE); + Assertions.assertEquals(LARGE_FILE_SIZE, bytesRead, + "Read size should match file size"); + } + return null; + }); } + } finally { + shutdownExecutorService(executorService); + } - int leastReadOpnCount = (numOfFiles * fileSize) / blockSize; + int leastReadOpnCount = fileSize/blockSize; + + verify(spiedClient, Mockito.atLeast(leastReadOpnCount)) + .read(Mockito.anyString(), Mockito.anyLong(), Mockito.any(), + Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), + Mockito.anyString(), Mockito.any(), Mockito.any()); + verify(spiedClient, Mockito.atMost(2 * leastReadOpnCount)) + .read(eq(testPath.toString()), Mockito.anyLong(), Mockito.any(), + Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), + Mockito.anyString(), Mockito.any(), Mockito.any()); - verify(spiedClient, Mockito.atLeast(leastReadOpnCount)) - .read(eq(testPath.toString()), Mockito.anyLong(), Mockito.any(), - Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), - Mockito.anyString(), Mockito.any(), Mockito.any()); - verify(spiedClient, Mockito.atMost(2 * leastReadOpnCount)) - .read(eq(testPath.toString()), Mockito.anyLong(), Mockito.any(), - Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), - Mockito.anyString(), Mockito.any(), Mockito.any()); - } } /** @@ -138,7 +138,6 @@ public void testMultipleInputStreamReadingSameFile() throws Exception { */ @Test public void testScheduledEviction() throws Exception { - } private AzureBlobFileSystem getConfiguredFileSystem(boolean isRAV2Enabled, From ad8190b141099ba77643cb50bc963496c0f2d71c Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Tue, 5 Aug 2025 05:31:18 -0700 Subject: [PATCH 08/13] Compilation Error --- .../fs/azurebfs/services/ITestReadBufferManagerV2.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java index da8ea3bc0f2f7..32003a2d4e5ac 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -151,12 +151,6 @@ private AzureBlobFileSystem getConfiguredFileSystem(boolean isRAV2Enabled, return (AzureBlobFileSystem) FileSystem.newInstance(conf); } - private byte[] getRandomBytesArray(int length) { - final byte[] b = new byte[length]; - new Random().nextBytes(b); - return b; - } - private Path createFileWithContent(FileSystem fs, String fileName, byte[] fileContent) throws IOException { From b4d81832a01395287563a3cba721b500dccfafa5 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Wed, 6 Aug 2025 03:12:59 -0700 Subject: [PATCH 09/13] Closed Fix --- hadoop-tools/hadoop-azure/pom.xml | 1 - .../hadoop/fs/azurebfs/services/ReadBufferManagerV2.java | 9 +++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index c2fbace2775f0..6c976f19f2428 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -29,7 +29,6 @@ Currently this consists of a filesystem client to read data from and write data to Azure Storage. - prefetchPR jar diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 537a8ba94afa5..458953b277606 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -84,6 +84,9 @@ public final class ReadBufferManagerV2 extends ReadBufferManager { // Buffer Manager Structures private static ReadBufferManagerV2 bufferManager; + private static ScheduledExecutorService memoryMonitorThread; + private static ScheduledExecutorService cpuMonitorThread; + /** * Private constructor to prevent instantiation as this needs to be singleton. @@ -150,7 +153,7 @@ void init() { getFreeList().add(i); numberOfActiveBuffers++; } - ScheduledExecutorService memoryMonitorThread + memoryMonitorThread = Executors.newSingleThreadScheduledExecutor(); memoryMonitorThread.scheduleAtFixedRate(this::scheduledEviction, memoryMonitoringIntervalInMilliSec, memoryMonitoringIntervalInMilliSec, TimeUnit.MILLISECONDS); @@ -172,7 +175,7 @@ void init() { ReadBufferWorker.UNLEASH_WORKERS.countDown(); if (isDynamicScalingEnabled) { - ScheduledExecutorService cpuMonitorThread + cpuMonitorThread = Executors.newSingleThreadScheduledExecutor(); cpuMonitorThread.scheduleAtFixedRate(this::adjustThreadPool, cpuMonitoringIntervalInMilliSec, cpuMonitoringIntervalInMilliSec, @@ -358,6 +361,8 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { public void close() { printTraceLog("Closing ReadBufferManagerV2"); workerPool.shutdownNow(); + memoryMonitorThread.shutdownNow(); + cpuMonitorThread.shutdownNow(); workerPool = null; if (bufferPool != null) { // help GC From f5883d0f5ae764ff7f39cb2dbdd1e2f677591721 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Thu, 7 Aug 2025 04:45:33 -0700 Subject: [PATCH 10/13] Code Improvements --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 - .../services/ReadBufferManagerV2.java | 30 +++--- .../azurebfs/AbstractAbfsIntegrationTest.java | 27 ++++++ .../ITestAzureBlobFileSystemFlush.java | 20 ++-- .../services/AbfsInputStreamTestUtils.java | 55 ----------- .../services/ITestAbfsInputStream.java | 8 +- .../ITestAbfsInputStreamReadFooter.java | 16 +-- .../ITestAbfsInputStreamSmallFileReads.java | 16 +-- .../services/ITestReadBufferManager.java | 19 ---- .../services/ITestReadBufferManagerV2.java | 97 ++++--------------- 10 files changed, 87 insertions(+), 203 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index ec8d2808ca06d..511f31ebb57e4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -110,8 +110,6 @@ import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy; import org.apache.hadoop.fs.azurebfs.services.ListingSupport; -import org.apache.hadoop.fs.azurebfs.services.ReadBufferManager; -import org.apache.hadoop.fs.azurebfs.services.ReadBufferManagerV1; import org.apache.hadoop.fs.azurebfs.services.ReadBufferManagerV2; import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials; import org.apache.hadoop.fs.azurebfs.services.StaticRetryPolicy; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 458953b277606..915b46593b9e4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -21,8 +21,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import com.sun.management.OperatingSystemMXBean; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.lang.management.ManagementFactory; @@ -34,11 +32,9 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; -import java.util.Queue; import java.util.Stack; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; @@ -48,10 +44,8 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.util.concurrent.HadoopExecutors; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_HUNDRED; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; /** * The Improved Read Buffer Manager for Rest AbfsClient. @@ -68,6 +62,7 @@ public final class ReadBufferManagerV2 extends ReadBufferManager { private static int threadPoolUpscalePercentage; private static int threadPoolDownscalePercentage; private static int executorServiceKeepAliveTimeInMilliSec; + private static final double threadPoolRequirementBuffer = 1.2; // 20% more threads than the queue size private static boolean isDynamicScalingEnabled; private ThreadPoolExecutor workerPool; @@ -108,10 +103,6 @@ public static ReadBufferManagerV2 getBufferManager() { LOCK.unlock(); } } - return (ReadBufferManagerV2) bufferManager; - } - - public static ReadBufferManagerV2 getInstance() { return bufferManager; } @@ -513,8 +504,9 @@ private void waitForProcess(final String eTag, final long position, boolean isFi } catch (InterruptedException ex) { Thread.currentThread().interrupt(); } - printTraceLog("Latch done for file: {}, with eTag: {}, for offset: {}, buffer index: {} queued by stream: {}", - readBuf.getPath(), readBuf.getETag(), readBuf.getOffset(), readBuf.getBufferindex(), readBuf.getStream().hashCode()); + printTraceLog("Latch done for file: {}, with eTag: {}, for offset: {}, " + + "buffer index: {} queued by stream: {}", readBuf.getPath(), readBuf.getETag(), + readBuf.getOffset(), readBuf.getBufferindex(), readBuf.getStream().hashCode()); } } @@ -636,8 +628,8 @@ private void scheduledEviction() { bufferPool[freeIndex] = null; removedBufferList.add(freeIndex); numberOfActiveBuffers--; + printTraceLog("Current Memory Load: {}. Decrementing buffer pool size to {}", memoryLoad, numberOfActiveBuffers); } - printTraceLog("Current Memory Load: {}. Decrementing buffer pool size to {}", memoryLoad, numberOfActiveBuffers); } } @@ -652,7 +644,8 @@ private void adjustThreadPool() { OperatingSystemMXBean.class); double cpuLoad = osBean.getSystemCpuLoad(); int currentPoolSize = workerRefs.size(); - int requiredPoolSize = (int) Math.ceil(1.2 * (getReadAheadQueue().size() + getInProgressList().size())); // 20% more for buffer + int requiredPoolSize = (int) Math.ceil(threadPoolRequirementBuffer + * (getReadAheadQueue().size() + getInProgressList().size())); // 20% more for buffer int newThreadPoolSize; printTraceLog("Current CPU load: {}, Current worker pool size: {}, Current queue size: {}", cpuLoad, currentPoolSize, requiredPoolSize); if (currentPoolSize < requiredPoolSize && cpuLoad < cpuThreshold) { @@ -756,7 +749,9 @@ public void callTryEvict() { @VisibleForTesting public int getNumBuffers() { - return numberOfActiveBuffers; + synchronized (this) { + return numberOfActiveBuffers; + } } @Override @@ -792,4 +787,9 @@ private void printDebugLog(String message, Object... args) { MemoryUsage memoryUsage = osBean.getHeapMemoryUsage(); return (double) memoryUsage.getUsed() / memoryUsage.getMax(); } + + @VisibleForTesting + public static ReadBufferManagerV2 getInstance() { + return bufferManager; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 9be4998cb8217..512285fb42e73 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsServiceType; @@ -708,12 +709,38 @@ protected void assertPathDns(Path path) { .contains(expectedDns); } + /** + * Return array of random bytes of the given length. + * + * @param length length of the byte array + * @return byte array + */ protected byte[] getRandomBytesArray(int length) { final byte[] b = new byte[length]; new Random().nextBytes(b); return b; } + /** + * Create a file on the file system with the given file name and content. + * + * @param fs fileSystem that stores the file + * @param fileName name of the file + * @param fileContent content of the file + * + * @return path of the file created + * @throws IOException exception in writing file on fileSystem + */ + protected Path createFileWithContent(FileSystem fs, String fileName, + byte[] fileContent) throws IOException { + Path testFilePath = path(fileName); + try (FSDataOutputStream oStream = fs.create(testFilePath)) { + oStream.write(fileContent); + oStream.flush(); + } + return testFilePath; + } + /** * Checks a list of futures for exceptions. * diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java index b55032c5132a5..13dd776f3498f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java @@ -234,7 +234,7 @@ private void testFlush(boolean disableOutputStreamFlush) throws Exception { .setDisableOutputStreamFlush(disableOutputStreamFlush); final Path testFilePath = path(methodName.getMethodName()); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); // The test case must write "fs.azure.write.request.size" bytes // to the stream in order for the data to be uploaded to storage. assertTrue(fs.getAbfsStore().getAbfsConfiguration().getWriteBufferSize() @@ -266,7 +266,7 @@ private void testFlush(boolean disableOutputStreamFlush) throws Exception { @Test public void testHflushWithFlushEnabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); String fileName = UUID.randomUUID().toString(); final Path testFilePath = path(fileName); @@ -279,7 +279,7 @@ public void testHflushWithFlushEnabled() throws Exception { @Test public void testHflushWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); final Path testFilePath = path(methodName.getMethodName()); boolean isAppendBlob = false; if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) { @@ -296,7 +296,7 @@ public void testHflushWithFlushDisabled() throws Exception { @Test public void testHsyncWithFlushEnabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); final Path testFilePath = path(methodName.getMethodName()); @@ -333,7 +333,7 @@ public void testTracingHeaderForAppendBlob() throws Exception { @Test public void testStreamCapabilitiesWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); final Path testFilePath = path(methodName.getMethodName()); @@ -350,7 +350,7 @@ public void testStreamCapabilitiesWithFlushDisabled() throws Exception { @Test public void testStreamCapabilitiesWithFlushEnabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); final Path testFilePath = path(methodName.getMethodName()); try (FSDataOutputStream stream = getStreamAfterWrite(fs, testFilePath, buffer, true)) { assertHasStreamCapabilities(stream, @@ -366,7 +366,7 @@ public void testStreamCapabilitiesWithFlushEnabled() throws Exception { @Test public void testHsyncWithFlushDisabled() throws Exception { final AzureBlobFileSystem fs = this.getFileSystem(); - byte[] buffer = getRandomBytesArray(); + byte[] buffer = getRandomBytesArray(TEST_FILE_LENGTH); final Path testFilePath = path(methodName.getMethodName()); boolean isAppendBlob = false; if (fs.getAbfsStore().isAppendBlobKey(fs.makeQualified(testFilePath).toString())) { @@ -379,12 +379,6 @@ public void testHsyncWithFlushDisabled() throws Exception { } } - private byte[] getRandomBytesArray() { - final byte[] b = new byte[TEST_FILE_LENGTH]; - new Random().nextBytes(b); - return b; - } - private FSDataOutputStream getStreamAfterWrite(AzureBlobFileSystem fs, Path path, byte[] buffer, boolean enableFlush) throws IOException { fs.getAbfsStore().getAbfsConfiguration().setEnableFlush(enableFlush); FSDataOutputStream stream = fs.create(path); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamTestUtils.java index 388e662115ed2..2d5d0a2d0b5a8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamTestUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamTestUtils.java @@ -49,29 +49,6 @@ public AbfsInputStreamTestUtils(AbstractAbfsIntegrationTest abstractAbfsIntegrat this.abstractAbfsIntegrationTest = abstractAbfsIntegrationTest; } - private Path path(String filepath) throws IOException { - return abstractAbfsIntegrationTest.getFileSystem().makeQualified( - new Path(getTestPath(), getUniquePath(filepath))); - } - - private Path getTestPath() { - Path path = new Path(UriUtils.generateUniqueTestPath()); - return path; - } - - /** - * Generate a unique path using the given filepath. - * @param filepath path string - * @return unique path created from filepath and a GUID - */ - private Path getUniquePath(String filepath) { - if (filepath.equals("/")) { - return new Path(filepath); - } - return new Path(filepath + StringUtils - .right(UUID.randomUUID().toString(), SHORTENED_GUID_LEN)); - } - /** * Returns AzureBlobFileSystem instance with the required * readFullFileOptimization configuration. @@ -90,38 +67,6 @@ public AzureBlobFileSystem getFileSystem(boolean readSmallFilesCompletely) return (AzureBlobFileSystem) FileSystem.newInstance(configuration); } - /** - * Return array of random bytes of the given length. - * - * @param length length of the byte array - * @return byte array - */ - public byte[] getRandomBytesArray(int length) { - final byte[] b = new byte[length]; - new Random().nextBytes(b); - return b; - } - - /** - * Create a file on the file system with the given file name and content. - * - * @param fs fileSystem that stores the file - * @param fileName name of the file - * @param fileContent content of the file - * - * @return path of the file created - * @throws IOException exception in writing file on fileSystem - */ - public Path createFileWithContent(FileSystem fs, String fileName, - byte[] fileContent) throws IOException { - Path testFilePath = path(fileName); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - oStream.flush(); - } - return testFilePath; - } - /** * Assert that the content read from the subsection of a file is correct. * diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java index d14ac05d5f5aa..a143599d0199d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java @@ -52,8 +52,8 @@ public void testWithNoOptimization() throws Exception { int fileSize = i * ONE_MB; final AzureBlobFileSystem fs = getFileSystem(false, false, fileSize); String fileName = methodName.getMethodName() + i; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(fs, fileName, fileContent); + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); testWithNoOptimization(fs, testFilePath, HUNDRED, fileContent); } } @@ -97,8 +97,8 @@ public void testExceptionInOptimization() throws Exception { int fileSize = i * ONE_MB; final AzureBlobFileSystem fs = getFileSystem(true, true, fileSize); String fileName = methodName.getMethodName() + i; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(fs, fileName, fileContent); + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); testExceptionInOptimization(fs, testFilePath, fileSize - HUNDRED, fileSize / 4, fileContent); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index c7c9da94ab2ed..d15eb180db87e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -260,8 +260,8 @@ private void validateSeekAndReadWithConf(boolean optimizeFooterRead, try (AzureBlobFileSystem spiedFs = createSpiedFs( getRawConfiguration())) { String fileName = methodName.getMethodName() + fileId; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(spiedFs, fileName, + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(spiedFs, fileName, fileContent); for (int readBufferSize : READ_BUFFER_SIZE) { validateSeekAndReadWithConf(spiedFs, optimizeFooterRead, seekTo, @@ -391,8 +391,8 @@ public void testPartialReadWithNoData() throws Exception { futureList.add(executorService.submit(() -> { try (AzureBlobFileSystem spiedFs = createSpiedFs( getRawConfiguration())) { - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(spiedFs, fileName, + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(spiedFs, fileName, fileContent); validatePartialReadWithNoData(spiedFs, fileSize, fileContent, testFilePath); @@ -463,8 +463,8 @@ public void testPartialReadWithSomeData() throws Exception { try (AzureBlobFileSystem spiedFs = createSpiedFs( getRawConfiguration())) { String fileName = methodName.getMethodName() + fileId; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(spiedFs, fileName, + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(spiedFs, fileName, fileContent); validatePartialReadWithSomeData(spiedFs, fileSize, testFilePath, fileContent); @@ -585,8 +585,8 @@ private void verifyConfigValueInStream(final FSDataInputStream inputStream, private Path createPathAndFileWithContent(final AzureBlobFileSystem fs, final int fileIdx, final int fileSize) throws Exception { String fileName = methodName.getMethodName() + fileIdx; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - return abfsInputStreamTestUtils.createFileWithContent(fs, fileName, fileContent); + byte[] fileContent = getRandomBytesArray(fileSize); + return createFileWithContent(fs, fileName, fileContent); } private FutureDataInputStreamBuilder getParameterizedBuilder(final Path path, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java index 64fac9ca94ed8..01a3387567a3c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamSmallFileReads.java @@ -74,8 +74,8 @@ private void validateNumBackendCalls(final boolean readSmallFilesCompletely, for (int i = 1; i <= 4; i++) { String fileName = methodName.getMethodName() + i; int fileSize = i * ONE_MB; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(fs, fileName, fileContent); + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); int length = ONE_KB; try (FSDataInputStream iStream = fs.open(testFilePath)) { byte[] buffer = new byte[length]; @@ -185,8 +185,8 @@ private void validateSeekAndReadWithConf(final SeekTo seekTo, for (int i = startFileSizeInMB; i <= endFileSizeInMB; i++) { String fileName = methodName.getMethodName() + i; int fileSize = i * ONE_MB; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray(fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(fs, fileName, fileContent); + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); int length = ONE_KB; int seekPos = seekPos(seekTo, fileSize, length); seekReadAndTest(fs, testFilePath, seekPos, length, fileContent); @@ -255,9 +255,9 @@ public void testPartialReadWithNoData() throws Exception { try (AzureBlobFileSystem fs = abfsInputStreamTestUtils.getFileSystem( true)) { String fileName = methodName.getMethodName() + i; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray( + byte[] fileContent = getRandomBytesArray( fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(fs, + Path testFilePath = createFileWithContent(fs, fileName, fileContent); partialReadWithNoData(fs, testFilePath, fileSize / 2, fileSize / 4, fileContent); @@ -305,9 +305,9 @@ public void testPartialReadWithSomeData() throws Exception { try (AzureBlobFileSystem fs = abfsInputStreamTestUtils.getFileSystem( true)) { String fileName = methodName.getMethodName() + i; - byte[] fileContent = abfsInputStreamTestUtils.getRandomBytesArray( + byte[] fileContent = getRandomBytesArray( fileSize); - Path testFilePath = abfsInputStreamTestUtils.createFileWithContent(fs, + Path testFilePath = createFileWithContent(fs, fileName, fileContent); partialReadWithSomeData(fs, testFilePath, fileSize / 2, fileSize / 4, fileContent); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java index 69f4983a41fb1..fd51fc7c420a1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManager.java @@ -18,10 +18,8 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.io.IOException; import java.util.LinkedList; import java.util.List; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -29,7 +27,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; @@ -170,22 +167,6 @@ private AzureBlobFileSystem getABFSWithReadAheadConfig() throws Exception { return (AzureBlobFileSystem) FileSystem.newInstance(conf); } - protected byte[] getRandomBytesArray(int length) { - final byte[] b = new byte[length]; - new Random().nextBytes(b); - return b; - } - - protected Path createFileWithContent(FileSystem fs, String fileName, - byte[] fileContent) throws IOException { - Path testFilePath = path(fileName); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - oStream.flush(); - } - return testFilePath; - } - private ReadBufferManager getBufferManager(AzureBlobFileSystem fs) { int blockSize = fs.getAbfsStore().getAbfsConfiguration().getReadAheadBlockSize(); if (getConfiguration().isReadAheadV2Enabled()) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java index 32003a2d4e5ac..a02cc5649b045 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -1,9 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -11,22 +28,17 @@ import org.junit.Test; import org.junit.jupiter.api.Assertions; -import org.mockito.Mockito; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_BLOCK_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.verify; /** * Test class for ReadBufferManagerV2 functionality. @@ -35,9 +47,7 @@ public class ITestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { private static final String TEST_FILE_NAME_PREFIX = "testFile"; private static final int LESS_NUM_FILES = 5; - private static final int MORE_NUM_FILES = 10; private static final int SMALL_FILE_SIZE = 30 * ONE_MB; - private static final int LARGE_FILE_SIZE = 100 * ONE_MB; public ITestReadBufferManagerV2() throws Exception { super(); @@ -80,66 +90,6 @@ public void testReadBufferManagerV2() throws Exception { } } - /** - * Test to verify that multiple input streams can read the same file. - * With read ahead v2 enabled, multiple input stream read the same cached buffer - * based on file eTag. - * @throws Exception if any error occurs during the test - */ - @Test - public void testMultipleInputStreamReadingSameFile() throws Exception { - AzureBlobFileSystem spiedFs = Mockito.spy(getFileSystem()); - AzureBlobFileSystemStore spiedStore = Mockito.spy(spiedFs.getAbfsStore()); - AbfsClient spiedClient = Mockito.spy(spiedStore.getClient()); - Mockito.doReturn(spiedClient).when(spiedStore).getClient(); - Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore(); - int numOfFiles = MORE_NUM_FILES; - int fileSize = SMALL_FILE_SIZE; - int blockSize = 4 * ONE_MB; - - Path[] testPaths = createFilesWithContent(spiedFs, 1); - Path testPath = testPaths[0]; - ExecutorService executorService = Executors.newFixedThreadPool( - LESS_NUM_FILES); - - try { - for (int i = 0; i < LESS_NUM_FILES; i++) { - executorService.submit((Callable) () -> { - try (FSDataInputStream iStream = spiedFs.open(testPath)) { - int bytesRead = iStream.read(new byte[LARGE_FILE_SIZE], 0, - LARGE_FILE_SIZE); - Assertions.assertEquals(LARGE_FILE_SIZE, bytesRead, - "Read size should match file size"); - } - return null; - }); - } - } finally { - shutdownExecutorService(executorService); - } - - int leastReadOpnCount = fileSize/blockSize; - - verify(spiedClient, Mockito.atLeast(leastReadOpnCount)) - .read(Mockito.anyString(), Mockito.anyLong(), Mockito.any(), - Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), - Mockito.anyString(), Mockito.any(), Mockito.any()); - verify(spiedClient, Mockito.atMost(2 * leastReadOpnCount)) - .read(eq(testPath.toString()), Mockito.anyLong(), Mockito.any(), - Mockito.anyInt(), Mockito.anyInt(), Mockito.anyString(), - Mockito.anyString(), Mockito.any(), Mockito.any()); - - } - - /** - * Test to verify that scheduled eviction of completed buffers happens. - * This test will be implemented in the future. - * @throws Exception if any error occurs during the test - */ - @Test - public void testScheduledEviction() throws Exception { - } - private AzureBlobFileSystem getConfiguredFileSystem(boolean isRAV2Enabled, Map configurations) throws IOException { Configuration conf = getConfiguration().getRawConfiguration(); @@ -151,17 +101,6 @@ private AzureBlobFileSystem getConfiguredFileSystem(boolean isRAV2Enabled, return (AzureBlobFileSystem) FileSystem.newInstance(conf); } - private Path createFileWithContent(FileSystem fs, String fileName, - byte[] fileContent) throws - IOException { - Path testFilePath = path(fileName); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - oStream.flush(); - } - return testFilePath; - } - private Path[] createFilesWithContent(FileSystem fs, int numFiles) throws Exception { ExecutorService executorService = Executors.newFixedThreadPool(numFiles); From d258d90f06489e5e4f2316a000b9d985cd572577 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Thu, 7 Aug 2025 04:48:07 -0700 Subject: [PATCH 11/13] Yetus Checks --- .../fs/azurebfs/services/ReadBufferManagerV2.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 915b46593b9e4..1d88271820d77 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -79,8 +79,8 @@ public final class ReadBufferManagerV2 extends ReadBufferManager { // Buffer Manager Structures private static ReadBufferManagerV2 bufferManager; - private static ScheduledExecutorService memoryMonitorThread; - private static ScheduledExecutorService cpuMonitorThread; + private ScheduledExecutorService memoryMonitorThread; + private ScheduledExecutorService cpuMonitorThread; /** @@ -108,6 +108,7 @@ public static ReadBufferManagerV2 getBufferManager() { /** * Set the ReadBufferManagerV2 configurations based on the provided before singleton initialization. + * @param readAheadBlockSize the block size for read-ahead operations. * @param abfsConfiguration the configuration to set for the ReadBufferManagerV2. */ public static void setReadBufferManagerConfigs(final int readAheadBlockSize, @@ -144,8 +145,7 @@ void init() { getFreeList().add(i); numberOfActiveBuffers++; } - memoryMonitorThread - = Executors.newSingleThreadScheduledExecutor(); + memoryMonitorThread = Executors.newSingleThreadScheduledExecutor(); memoryMonitorThread.scheduleAtFixedRate(this::scheduledEviction, memoryMonitoringIntervalInMilliSec, memoryMonitoringIntervalInMilliSec, TimeUnit.MILLISECONDS); @@ -166,8 +166,7 @@ void init() { ReadBufferWorker.UNLEASH_WORKERS.countDown(); if (isDynamicScalingEnabled) { - cpuMonitorThread - = Executors.newSingleThreadScheduledExecutor(); + cpuMonitorThread = Executors.newSingleThreadScheduledExecutor(); cpuMonitorThread.scheduleAtFixedRate(this::adjustThreadPool, cpuMonitoringIntervalInMilliSec, cpuMonitoringIntervalInMilliSec, TimeUnit.MILLISECONDS); From 2444f92a353dacf5cced826c51e181b696e424d7 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Fri, 8 Aug 2025 05:18:17 -0700 Subject: [PATCH 12/13] Added more tests --- .../services/ReadBufferManagerV2.java | 36 ++++++++++++ .../services/TestReadBufferManagerV2.java | 55 +++++++++++++++++++ 2 files changed, 91 insertions(+) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 1d88271820d77..3206ae0f1545f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -79,6 +79,7 @@ public final class ReadBufferManagerV2 extends ReadBufferManager { // Buffer Manager Structures private static ReadBufferManagerV2 bufferManager; + private static boolean isConfigured = false; private ScheduledExecutorService memoryMonitorThread; private ScheduledExecutorService cpuMonitorThread; @@ -91,6 +92,10 @@ private ReadBufferManagerV2() { } public static ReadBufferManagerV2 getBufferManager() { + if (!isConfigured) { + throw new IllegalStateException("ReadBufferManagerV2 is not configured. " + + "Please call setReadBufferManagerConfigs() before calling getBufferManager()."); + } if (bufferManager == null) { LOCK.lock(); try { @@ -130,6 +135,7 @@ public static void setReadBufferManagerConfigs(final int readAheadBlockSize, setThresholdAgeMilliseconds(abfsConfiguration.getReadAheadV2CachedBufferTTLMillis()); isDynamicScalingEnabled = abfsConfiguration.isReadAheadV2DynamicScalingEnabled(); setReadAheadBlockSize(readAheadBlockSize); + isConfigured = true; } } @@ -791,4 +797,34 @@ private void printDebugLog(String message, Object... args) { public static ReadBufferManagerV2 getInstance() { return bufferManager; } + + @VisibleForTesting + public int getMinThreadPoolSize() { + return minThreadPoolSize; + } + + @VisibleForTesting + public int getMaxThreadPoolSize() { + return maxThreadPoolSize; + } + + @VisibleForTesting + public int getMinBufferPoolSize() { + return minBufferPoolSize; + } + + @VisibleForTesting + public int getMaxBufferPoolSize() { + return maxBufferPoolSize; + } + + @VisibleForTesting + public int getCurrentThreadPoolSize() { + return workerRefs.size(); + } + + @VisibleForTesting + public int getCpuMonitoringIntervalInMilliSec() { + return cpuMonitoringIntervalInMilliSec; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java new file mode 100644 index 0000000000000..1509c1574a4a0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.azurebfs.services; + +import java.io.IOException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public class TestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { + + public TestReadBufferManagerV2() throws Exception { + super(); + } + + @Test + public void testReadBufferManagerV2Init() throws Exception { + Assertions.assertThat(ReadBufferManagerV2.getInstance()).isNull(); + intercept(IllegalStateException.class, "ReadBufferManagerV2 is not configured.", () -> { + ReadBufferManagerV2.getBufferManager(); + }); + ReadBufferManagerV2.setReadBufferManagerConfigs( + getConfiguration().getReadAheadBlockSize(), getConfiguration()); + ReadBufferManagerV2 bufferManager = ReadBufferManagerV2.getBufferManager(); + ReadBufferManagerV2 bufferManager2 = ReadBufferManagerV2.getBufferManager(); + Assertions.assertThat(bufferManager).isNotNull(); + Assertions.assertThat(bufferManager2).isNotNull(); + Assertions.assertThat(bufferManager).isSameAs(bufferManager2); + Assertions.assertThat(ReadBufferManagerV2.getInstance()).isNotNull(); + Assertions.assertThat(ReadBufferManagerV2.getInstance()).isSameAs(bufferManager); + + Assertions.assertThat(bufferManager.getMinBufferPoolSize()).isGreaterThan(0); + Assertions.assertThat(bufferManager.getMaxBufferPoolSize()).isGreaterThan(0); + Assertions.assertThat(bufferManager.getCurrentThreadPoolSize()).isEqualTo(bufferManager.getMinThreadPoolSize()); + } +} From a6064efe52a45fe05fe92bae1e97c5131085f74f Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Sun, 17 Aug 2025 22:14:28 -0700 Subject: [PATCH 13/13] Added some tests --- .../services/ReadBufferManagerV2.java | 31 +++- .../azurebfs/AbstractAbfsIntegrationTest.java | 2 +- .../services/ITestReadBufferManagerV2.java | 151 +++++++++++++++--- .../services/TestReadBufferManagerV2.java | 81 ++++++++-- 4 files changed, 224 insertions(+), 41 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java index 3206ae0f1545f..3e5c2c4c81c83 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManagerV2.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Stack; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.SynchronousQueue; @@ -356,9 +357,9 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) { public void close() { printTraceLog("Closing ReadBufferManagerV2"); - workerPool.shutdownNow(); - memoryMonitorThread.shutdownNow(); - cpuMonitorThread.shutdownNow(); + shutdown(workerPool); + shutdown(memoryMonitorThread); + shutdown(cpuMonitorThread); workerPool = null; if (bufferPool != null) { // help GC @@ -369,6 +370,23 @@ public void close() { printTraceLog("ReadBufferManagerV2 closed"); } + private void shutdown(ExecutorService service) { + if (service == null || service.isShutdown()) { + return; // already shut down or never initialized + } + + try { + service.shutdown(); // disable new tasks from being submitted + // wait for existing tasks to terminate + if (!service.awaitTermination(executorServiceKeepAliveTimeInMilliSec, TimeUnit.MILLISECONDS)) { + service.shutdownNow(); // cancel currently executing tasks + } + } catch (InterruptedException e) { + service.shutdownNow(); // force shutdown if interrupted + Thread.currentThread().interrupt(); // restore interrupted status + } + } + private boolean isAlreadyQueued(final String eTag, final long requestedOffset) { // returns true if any part of the buffer is already queued return (isInList(getReadAheadQueue(), eTag, requestedOffset) @@ -627,7 +645,7 @@ private void scheduledEviction() { } double memoryLoad = getMemoryLoad(); - if (memoryLoad > memoryThreshold) { + if (isDynamicScalingEnabled && memoryLoad > memoryThreshold) { synchronized (this) { int freeIndex = getFreeList().pop(); bufferPool[freeIndex] = null; @@ -827,4 +845,9 @@ public int getCurrentThreadPoolSize() { public int getCpuMonitoringIntervalInMilliSec() { return cpuMonitoringIntervalInMilliSec; } + + @VisibleForTesting + public ScheduledExecutorService getCpuMonitoringThread() { + return cpuMonitorThread; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index ab9ef639abe94..be1afcc4dfed6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -587,7 +587,7 @@ protected void assumeValidAuthConfigsPresent() { final AuthType currentAuthType = getAuthType(); assumeThat(currentAuthType). as("SAS Based Authentication Not Allowed For Integration Tests"). - isEqualTo(AuthType.SAS); + isNotEqualTo(AuthType.SAS); if (currentAuthType == AuthType.SharedKey) { assumeValidTestConfigPresent(getRawConfiguration(), FS_AZURE_ACCOUNT_KEY); } else { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java index a02cc5649b045..0c5fb85db6428 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadBufferManagerV2.java @@ -19,15 +19,13 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import org.junit.Test; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -35,10 +33,23 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_BLOCK_SIZE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; /** * Test class for ReadBufferManagerV2 functionality. @@ -46,12 +57,10 @@ public class ITestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { private static final String TEST_FILE_NAME_PREFIX = "testFile"; - private static final int LESS_NUM_FILES = 5; - private static final int SMALL_FILE_SIZE = 30 * ONE_MB; + private static final int LESS_NUM_FILES = 1; + private static final int SMALL_FILE_SIZE = 3 * ONE_MB; public ITestReadBufferManagerV2() throws Exception { - super(); - getConfiguration().setBoolean(FS_AZURE_ENABLE_READAHEAD_V2, true); } /** @@ -60,11 +69,15 @@ public ITestReadBufferManagerV2() throws Exception { */ @Test public void testReadBufferManagerV2() throws Exception { - Map configMap = new HashMap<>(); - configMap.put(FS_AZURE_READ_AHEAD_BLOCK_SIZE, String.valueOf(4 * ONE_MB)); - try (AzureBlobFileSystem fs = getConfiguredFileSystem(true, configMap)) { + try (AzureBlobFileSystem fs = getConfiguredFileSystem(true)) { + AzureBlobFileSystem spiedFs = spy(fs); + AzureBlobFileSystemStore spiedStore = spy(fs.getAbfsStore()); + AbfsClient spiedClient = spy(spiedStore.getClient()); + doReturn(spiedClient).when(spiedStore).getClient(); + doReturn(spiedStore).when(spiedFs).getAbfsStore(); int numOfFiles = LESS_NUM_FILES; - Path[] testPaths = createFilesWithContent(fs, numOfFiles); + int fileSize = SMALL_FILE_SIZE; + Path[] testPaths = createFilesWithContent(spiedFs, numOfFiles); ExecutorService executorService = Executors.newFixedThreadPool( numOfFiles); @@ -72,10 +85,11 @@ public void testReadBufferManagerV2() throws Exception { try { for (int i = 0; i < numOfFiles; i++) { executorService.submit((Callable) () -> { - try (FSDataInputStream iStream = fs.open(testPaths[fileIdx[0]++])) { - int bytesRead = iStream.read(new byte[SMALL_FILE_SIZE], 0, - SMALL_FILE_SIZE); - Assertions.assertEquals(SMALL_FILE_SIZE, bytesRead, + try (FSDataInputStream iStream = spiedFs.open( + testPaths[fileIdx[0]++])) { + int bytesRead = iStream.read(new byte[fileSize], 0, + fileSize); + Assertions.assertEquals(fileSize, bytesRead, "Read size should match file size"); } return null; @@ -87,17 +101,110 @@ public void testReadBufferManagerV2() throws Exception { } finally { shutdownExecutorService(executorService); } + + int expectedReadCalls = numOfFiles * (int)Math.ceil((double)fileSize/getConfiguration().getReadBufferSize()); + verify(spiedClient, atLeast(expectedReadCalls)).read(anyString(), + anyLong(), any(), anyInt(), anyInt(), anyString(), anyString(), + nullable(ContextEncryptionAdapter.class), any(TracingContext.class)); } } - private AzureBlobFileSystem getConfiguredFileSystem(boolean isRAV2Enabled, - Map configurations) throws IOException { - Configuration conf = getConfiguration().getRawConfiguration(); - conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2, isRAV2Enabled); - for (Map.Entry entry : configurations.entrySet()) { - conf.set(entry.getKey(), entry.getValue()); + /** + * Test to verify that ReadBufferManagerV2 can read different files concurrently. + * @throws Exception if any error occurs during the test + */ + @Test + public void testReadSameFileFromMultipleStreamsWithV2Disabled() throws Exception { + try (AzureBlobFileSystem fs = getConfiguredFileSystem(false)) { + AzureBlobFileSystem spiedFs = spy(fs); + AzureBlobFileSystemStore spiedStore = spy(fs.getAbfsStore()); + AbfsClient spiedClient = spy(spiedStore.getClient()); + doReturn(spiedClient).when(spiedStore).getClient(); + doReturn(spiedStore).when(spiedFs).getAbfsStore(); + int numOfFiles = LESS_NUM_FILES; + int fileSize = SMALL_FILE_SIZE; + Path[] testPaths = createFilesWithContent(spiedFs, 1); + ExecutorService executorService = Executors.newFixedThreadPool( + numOfFiles); + + try { + for (int i = 0; i < numOfFiles; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = spiedFs.open( + testPaths[0])) { + int bytesRead = iStream.read(new byte[fileSize], 0, + fileSize); + Assertions.assertEquals(fileSize, bytesRead, + "Read size should match file size"); + } + return null; + }); + } + } catch (Exception e) { + System.out.println( + "Exception occurred during file read: " + e.getMessage()); + } finally { + shutdownExecutorService(executorService); + } + + int expectedReadCalls = (numOfFiles * fileSize) + / getConfiguration().getReadBufferSize(); + verify(spiedClient, atLeast(expectedReadCalls)).read(eq(testPaths[0].getName()), + anyLong(), any(), anyInt(), anyInt(), anyString(), anyString(), + nullable(ContextEncryptionAdapter.class), any(TracingContext.class)); + } + } + + /** + * Test to verify that ReadBufferManagerV2 can read different files concurrently. + * @throws Exception if any error occurs during the test + */ + @Test + public void testReadSameFileFromMultipleStreamsWithV2Enabled() throws Exception { + try (AzureBlobFileSystem fs = getConfiguredFileSystem(true)) { + AzureBlobFileSystem spiedFs = spy(fs); + AzureBlobFileSystemStore spiedStore = spy(fs.getAbfsStore()); + AbfsClient spiedClient = spy(spiedStore.getClient()); + doReturn(spiedClient).when(spiedStore).getClient(); + doReturn(spiedStore).when(spiedFs).getAbfsStore(); + int numOfFiles = LESS_NUM_FILES; + int fileSize = SMALL_FILE_SIZE; + Path[] testPaths = createFilesWithContent(spiedFs, numOfFiles); + ExecutorService executorService = Executors.newFixedThreadPool( + numOfFiles); + + int[] fileIdx = new int[1]; + try { + for (int i = 0; i < numOfFiles; i++) { + executorService.submit((Callable) () -> { + try (FSDataInputStream iStream = spiedFs.open( + testPaths[fileIdx[0]++])) { + int bytesRead = iStream.read(new byte[fileSize], 0, + fileSize); + Assertions.assertEquals(fileSize, bytesRead, + "Read size should match file size"); + } + return null; + }); + } + } catch (Exception e) { + System.out.println( + "Exception occurred during file read: " + e.getMessage()); + } finally { + shutdownExecutorService(executorService); + } + + int expectedReadCalls = fileSize / getConfiguration().getReadBufferSize(); + verify(spiedClient, atLeast(expectedReadCalls)).read(eq(testPaths[0].getName()), + anyLong(), any(), anyInt(), anyInt(), anyString(), anyString(), + nullable(ContextEncryptionAdapter.class), any(TracingContext.class)); } + } + private AzureBlobFileSystem getConfiguredFileSystem(boolean isReadAheahdV2Enabled) throws IOException { + Configuration conf = getConfiguration().getRawConfiguration(); + conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2, isReadAheahdV2Enabled); + conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING, isReadAheahdV2Enabled); return (AzureBlobFileSystem) FileSystem.newInstance(conf); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java index 1509c1574a4a0..da9fdc2e294e5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestReadBufferManagerV2.java @@ -17,39 +17,92 @@ */ package org.apache.hadoop.fs.azurebfs.services; -import java.io.IOException; - -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; +/** + * Unit Tests around different components of Read Buffer Manager V2 + */ public class TestReadBufferManagerV2 extends AbstractAbfsIntegrationTest { public TestReadBufferManagerV2() throws Exception { super(); } + /** + * Test to verify init of ReadBufferManagerV2 + * @throws Exception if test fails + */ @Test public void testReadBufferManagerV2Init() throws Exception { - Assertions.assertThat(ReadBufferManagerV2.getInstance()).isNull(); + assertThat(ReadBufferManagerV2.getInstance()) + .as("ReadBufferManager should be uninitialized").isNull(); intercept(IllegalStateException.class, "ReadBufferManagerV2 is not configured.", () -> { ReadBufferManagerV2.getBufferManager(); }); ReadBufferManagerV2.setReadBufferManagerConfigs( getConfiguration().getReadAheadBlockSize(), getConfiguration()); + // verify that multiple invocations of getBufferManager returns same instance. ReadBufferManagerV2 bufferManager = ReadBufferManagerV2.getBufferManager(); ReadBufferManagerV2 bufferManager2 = ReadBufferManagerV2.getBufferManager(); - Assertions.assertThat(bufferManager).isNotNull(); - Assertions.assertThat(bufferManager2).isNotNull(); - Assertions.assertThat(bufferManager).isSameAs(bufferManager2); - Assertions.assertThat(ReadBufferManagerV2.getInstance()).isNotNull(); - Assertions.assertThat(ReadBufferManagerV2.getInstance()).isSameAs(bufferManager); - - Assertions.assertThat(bufferManager.getMinBufferPoolSize()).isGreaterThan(0); - Assertions.assertThat(bufferManager.getMaxBufferPoolSize()).isGreaterThan(0); - Assertions.assertThat(bufferManager.getCurrentThreadPoolSize()).isEqualTo(bufferManager.getMinThreadPoolSize()); + ReadBufferManagerV2 bufferManager3 = ReadBufferManagerV2.getInstance(); + assertThat(bufferManager).isNotNull(); + assertThat(bufferManager2).isNotNull(); + assertThat(bufferManager).isSameAs(bufferManager2); + assertThat(bufferManager3).isNotNull(); + assertThat(bufferManager3).isSameAs(bufferManager); + + // Verify default values are not invalid. + assertThat(bufferManager.getMinBufferPoolSize()).isGreaterThan(0); + assertThat(bufferManager.getMaxBufferPoolSize()).isGreaterThan(0); + } + + /** + * Test to verify that cpu monitor thread is not active if disabled. + * @throws Exception if test fails + */ + @Test + public void testDynamicScalingSwitchingOnAndOff() throws Exception { + Configuration conf = new Configuration(getRawConfiguration()); + conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2, true); + conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING, true); + try(AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(getFileSystem().getUri(), conf)) { + AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); + ReadBufferManagerV2.setReadBufferManagerConfigs(abfsConfiguration.getReadAheadBlockSize(), abfsConfiguration); + ReadBufferManagerV2 bufferManagerV2 = ReadBufferManagerV2.getBufferManager(); + assertThat(bufferManagerV2.getCpuMonitoringThread()) + .as("CPU Monitor thread should be initialized").isNotNull(); + bufferManagerV2.close(); + } + + conf.setBoolean(FS_AZURE_ENABLE_READAHEAD_V2_DYNAMIC_SCALING, false); + try(AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(getFileSystem().getUri(), conf)) { + AbfsConfiguration abfsConfiguration = fs.getAbfsStore().getAbfsConfiguration(); + ReadBufferManagerV2.setReadBufferManagerConfigs(abfsConfiguration.getReadAheadBlockSize(), abfsConfiguration); + ReadBufferManagerV2 bufferManagerV2 = ReadBufferManagerV2.getBufferManager(); + assertThat(bufferManagerV2.getCpuMonitoringThread()) + .as("CPU Monitor thread should not be initialized").isNull(); + bufferManagerV2.close(); + } + } + + /** + * Test to verify that prefetch for same file and same position is not queued + * even when attempted by different input streams instances. + * @throws Exception if test fails + */ + @Test + public void testPrefetchAlreadyQueued() throws Exception { + } }