From afd96204a40bfdeb52497fbc87cdc74eb4127851 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 22 Dec 2025 17:56:02 +0200 Subject: [PATCH 1/6] feat(inkless): rate-limited hot/cold path for fetch operations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Implements hot/cold path separation to protect Inkless from load spikes caused by lagging consumers while maintaining performance for recent consumers. Hot Path (Recent Data): - Uses object cache for fast repeated access - Dedicated executor pool - Prioritizes low-latency for up-to-date consumers Cold Path (Lagging Consumers): - Bypasses cache to avoid evicting hot data - Dedicated executor pool (default: half of hot-path pool) with bounded queue - Optional rate limiting (default 200 req/s) to control S3 costs and CPU usage - Separate storage client for resource isolation - AbortPolicy for backpressure: queue full → RejectedExecutionException → Kafka error handler → consumer backs off via fetch purgatory Path Selection: - Age threshold: Data older than cache TTL (default 60s) uses cold path - Configurable via fetch.lagging.consumer.threshold.ms (-1 = auto) - Based on batch timestamp (max across partitions in same object) - Handles future timestamps (clock skew) by treating as recent data Backpressure Mechanism: - Bounded queue prevents OOM (1600 tasks ≈ 3.2MB, ~8 seconds buffer) - AbortPolicy rejects tasks when full (no request handler blocking) - Only RejectedExecutionException tracked as rejections (capacity/backpressure) - InterruptedException treated as fetch operation failure, not rejection - Exceptions propagate to Kafka error handler (no custom empty responses) - Consumers receive error → fetch purgatory backoff → natural retry Fixes partial failure scenario: Reader#allOfFileExtents now handles partial failures gracefully and ensure that successful partitions are returned. Robustness Improvements: - Null safety: Validates laggingObjectFetcher is non-null when executor exists - Validation: FindBatchesJob validates response count matches request count - Edge cases: Handles future timestamps (clock skew) and empty batch coordinates - Resource management: Rate limiter only created when lagging consumer feature enabled Monitoring Metrics: - LaggingConsumerRequestRate: cold path request rate - LaggingConsumerRejectedRate: rejection events (queue full/shutdown) - LaggingRateLimitWaitTime: rate limiting latency histogram - RecentDataRequestRate: hot path request rate Configuration: - fetch.lagging.consumer.thread.pool.size (default: 16) Controls cold path concurrency - fetch.lagging.consumer.request.rate.limit (default: 200 req/s) Controls S3 request rate (set 0 to disable) - fetch.lagging.consumer.threshold.ms (default: -1 = cache TTL) Age threshold for hot/cold path selection -1 uses cache TTL automatically (validated to always be >= cache lifespan) Design Trade-offs: - Separate storage client doubles connection pools but prevents hot path degradation from cold path bursts - Cache bypass for cold path means no deduplication across lagging consumers, acceptable because objects are multi-partition blobs (future: split by partition may allow to consider enabling caching) --- build.gradle | 1 + docs/inkless/configs.rst | 26 +- gradle/dependencies.gradle | 4 +- .../aiven/inkless/config/InklessConfig.java | 166 +- .../aiven/inkless/consume/FetchCompleter.java | 50 +- .../aiven/inkless/consume/FetchException.java | 4 +- .../aiven/inkless/consume/FetchHandler.java | 16 + .../aiven/inkless/consume/FetchPlanner.java | 176 +- .../inkless/consume/FileExtentResult.java | 57 + .../aiven/inkless/consume/FindBatchesJob.java | 13 + .../inkless/consume/InklessFetchMetrics.java | 86 +- .../java/io/aiven/inkless/consume/Reader.java | 181 +- .../inkless/control_plane/ControlPlane.java | 9 + .../inkless/config/InklessConfigTest.java | 300 +++ .../inkless/consume/FetchCompleterTest.java | 23 +- .../inkless/consume/FetchPlannerTest.java | 1608 ++++++++++++----- .../io/aiven/inkless/consume/ReaderTest.java | 682 ++++++- 17 files changed, 2896 insertions(+), 506 deletions(-) create mode 100644 storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java diff --git a/build.gradle b/build.gradle index a4edf82b84..56acda39dc 100644 --- a/build.gradle +++ b/build.gradle @@ -2453,6 +2453,7 @@ project(':storage:inkless') { } implementation libs.metrics implementation libs.caffeine + implementation libs.bucket4jCore testImplementation project(':clients').sourceSets.test.output.classesDirs testImplementation project(':test-common') diff --git a/docs/inkless/configs.rst b/docs/inkless/configs.rst index e05cb53156..5c5e9b487c 100644 --- a/docs/inkless/configs.rst +++ b/docs/inkless/configs.rst @@ -39,6 +39,22 @@ Under ``inkless.`` * Default: io.aiven.inkless.storage_backend.in_memory.InMemoryStorage * Importance: high +``fetch.lagging.consumer.request.rate.limit`` + Maximum requests per second for lagging consumer data fetches. Set to 0 to disable rate limiting. The upper bound of 10000 req/s is a safety limit to prevent misconfiguration. For high-throughput systems, consider the relationship between this rate limit, thread pool size, and storage backend capacity. At the default rate of 200 req/s with ~50ms per request latency, this allows ~10 concurrent requests. + + * Type: int + * Default: 200 + * Valid Values: [0,...,10000] + * Importance: medium + +``fetch.lagging.consumer.threshold.ms`` + The time threshold in milliseconds to distinguish between recent and lagging consumers. Fetch requests for data strictly older than this threshold (dataAge > threshold, based on batch timestamp) will use the lagging consumer path. Set to -1 to use the default heuristic: the cache expiration lifespan. This provides a grace period ensuring data remains in cache before being considered 'lagging', accounting for cache warm-up and typical consumer lag variations. Must be >= cache expiration lifespan (see consume.cache.expiration.lifespan.sec). This is a startup-only configuration (no dynamic reconfiguration support). Both threshold and cache lifespan must be set together at startup to maintain the constraint. + + * Type: long + * Default: -1 + * Valid Values: [-1,...] + * Importance: medium + ``object.key.prefix`` The object storage key prefix. It cannot start of finish with a slash. @@ -125,8 +141,16 @@ Under ``inkless.`` * Valid Values: [0,...] * Importance: low +``fetch.lagging.consumer.thread.pool.size`` + Thread pool size for lagging consumer fetch requests (consumers reading old data). Set to 0 to disable the lagging consumer feature (all requests will use the recent data path). The default value of 16 is designed as approximately half of the default fetch.data.thread.pool.size (32), providing sufficient capacity for typical cold storage access patterns while leaving headroom for the hot path. The queue capacity is automatically set to thread.pool.size * 100, providing burst buffering (e.g., 16 threads = 1600 queue capacity ≈ 8 seconds buffer at 200 req/s). Tune based on lagging consumer SLA and expected load patterns. + + * Type: int + * Default: 16 + * Valid Values: [0,...] + * Importance: low + ``fetch.metadata.thread.pool.size`` - Thread pool size to concurrently fetch metadata from batch coordinator + Thread pool size to concurrently fetch metadata from batch coordinator. Note: This executor is shared between hot and cold path requests. The hot/cold path separation only applies to data fetching (after metadata is retrieved). A burst of lagging consumer requests can still compete with recent consumer requests at the metadata layer. For workloads with significant lagging consumer traffic, consider increasing this value proportionally to the combined fetch.data.thread.pool.size + fetch.lagging.consumer.thread.pool.size to prevent metadata fetching from becoming a bottleneck in mixed hot/cold workloads. * Type: int * Default: 8 diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 2d08e277dd..8267bc6873 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -58,8 +58,9 @@ versions += [ awsSdk: "2.29.6", azureSdk: "1.2.28", bcpkix: "1.80", - caffeine: "3.2.0", bndlib: "7.1.0", + bucket4j: "8.14.0", + caffeine: "3.2.0", checkstyle: project.hasProperty('checkstyleVersion') ? checkstyleVersion : "10.20.2", commonsBeanutils: "1.11.0", commonsValidator: "1.9.0", @@ -164,6 +165,7 @@ libs += [ azureSdkBom: "com.azure:azure-sdk-bom:$versions.azureSdk", bcpkix: "org.bouncycastle:bcpkix-jdk18on:$versions.bcpkix", bndlib:"biz.aQute.bnd:biz.aQute.bndlib:$versions.bndlib", + bucket4jCore: "com.bucket4j:bucket4j_jdk11-core:$versions.bucket4j", caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine", classgraph: "io.github.classgraph:classgraph:$versions.classgraph", commonsBeanutils: "commons-beanutils:commons-beanutils:$versions.commonsBeanutils", diff --git a/storage/inkless/src/main/java/io/aiven/inkless/config/InklessConfig.java b/storage/inkless/src/main/java/io/aiven/inkless/config/InklessConfig.java index 98896b797a..12f08fc7f8 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/config/InklessConfig.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/config/InklessConfig.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.metrics.Metrics; import java.lang.reflect.InvocationTargetException; @@ -136,9 +137,54 @@ public class InklessConfig extends AbstractConfig { private static final int FETCH_DATA_THREAD_POOL_SIZE_DEFAULT = 32; public static final String FETCH_METADATA_THREAD_POOL_SIZE_CONFIG = "fetch.metadata.thread.pool.size"; - public static final String FETCH_METADATA_THREAD_POOL_SIZE_DOC = "Thread pool size to concurrently fetch metadata from batch coordinator"; + public static final String FETCH_METADATA_THREAD_POOL_SIZE_DOC = "Thread pool size to concurrently fetch metadata from batch coordinator. " + + "Note: This executor is shared between hot and cold path requests. The hot/cold path separation " + + "only applies to data fetching (after metadata is retrieved). A burst of lagging consumer requests " + + "can still compete with recent consumer requests at the metadata layer. For workloads with significant " + + "lagging consumer traffic, consider increasing this value proportionally to the combined " + + "fetch.data.thread.pool.size + fetch.lagging.consumer.thread.pool.size to prevent metadata fetching " + + "from becoming a bottleneck in mixed hot/cold workloads."; private static final int FETCH_METADATA_THREAD_POOL_SIZE_DEFAULT = 8; + public static final String FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_CONFIG = "fetch.lagging.consumer.thread.pool.size"; + public static final String FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_DOC = "Thread pool size for lagging consumer fetch requests (consumers reading old data). " + + "Set to 0 to disable the lagging consumer feature (all requests will use the recent data path). " + + "The default value of 16 is designed as approximately half of the default fetch.data.thread.pool.size (32), " + + "providing sufficient capacity for typical cold storage access patterns while leaving headroom for the hot path. " + + "The queue capacity is automatically set to thread.pool.size * 100, providing burst buffering " + + "(e.g., 16 threads = 1600 queue capacity ≈ 8 seconds buffer at 200 req/s). " + + "Tune based on lagging consumer SLA and expected load patterns."; + // Default 16: Designed as half of default fetch.data.thread.pool.size (32), sufficient for typical + // cold storage access patterns while leaving headroom for hot path. Tune based on lagging consumer SLA. + private static final int FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_DEFAULT = 16; + + public static final String FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG = "fetch.lagging.consumer.threshold.ms"; + public static final String FETCH_LAGGING_CONSUMER_THRESHOLD_MS_DOC = "The time threshold in milliseconds to distinguish between recent and lagging consumers. " + + "Fetch requests for data strictly older than this threshold (dataAge > threshold, based on batch timestamp) will use the lagging consumer path. " + + "Set to -1 to use the default heuristic: the cache expiration lifespan. " + + "This provides a grace period ensuring data remains in cache before being considered 'lagging', " + + "accounting for cache warm-up and typical consumer lag variations. " + + "Must be >= cache expiration lifespan (see " + CONSUME_CACHE_EXPIRATION_LIFESPAN_SEC_CONFIG + "). " + + "This is a startup-only configuration (no dynamic reconfiguration support). " + + "Both threshold and cache lifespan must be set together at startup to maintain the constraint."; + /** + * Default value for {@link #FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG}. + * A value of -1 means "auto-detect from cache TTL" - the {@link #fetchLaggingConsumerThresholdMs()} method + * will automatically use the cache expiration lifespan as the effective threshold. + */ + private static final int FETCH_LAGGING_CONSUMER_THRESHOLD_MS_DEFAULT = -1; + + public static final String FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_CONFIG = "fetch.lagging.consumer.request.rate.limit"; + public static final String FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_DOC = "Maximum requests per second for lagging consumer data fetches. " + + "Set to 0 to disable rate limiting. " + + "The upper bound of 10000 req/s is a safety limit to prevent misconfiguration. For high-throughput systems, " + + "consider the relationship between this rate limit, thread pool size, and storage backend capacity. " + + "At the default rate of 200 req/s with ~50ms per request latency, this allows ~10 concurrent requests."; + // Default 200 req/s: Conservative limit based on typical object storage GET request costs and latency. + // At ~50ms per request, 200 req/s = ~10 concurrent requests, balancing throughput with cost control. + // Tune based on storage backend capacity and budget constraints. + private static final int FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_DEFAULT = 200; + public static final String FETCH_FIND_BATCHES_MAX_BATCHES_PER_PARTITION_CONFIG = "fetch.find.batches.max.per.partition"; public static final String FETCH_FIND_BATCHES_MAX_BATCHES_PER_PARTITION_DOC = "The maximum number of batches to find per partition when processing a fetch request. " + "A value of 0 means all available batches are fetched. " @@ -322,6 +368,32 @@ public static ConfigDef configDef() { ConfigDef.Importance.LOW, FETCH_METADATA_THREAD_POOL_SIZE_DOC ); + configDef.define( + FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_CONFIG, + ConfigDef.Type.INT, + FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_DEFAULT, + ConfigDef.Range.atLeast(0), + ConfigDef.Importance.LOW, + FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_DOC + ); + configDef.define( + FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG, + ConfigDef.Type.LONG, + FETCH_LAGGING_CONSUMER_THRESHOLD_MS_DEFAULT, + ConfigDef.Range.atLeast(-1), + ConfigDef.Importance.MEDIUM, + FETCH_LAGGING_CONSUMER_THRESHOLD_MS_DOC + ); + configDef.define( + FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_CONFIG, + ConfigDef.Type.INT, + FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_DEFAULT, + ConfigDef.Range.between(0, 10000), + // Safety limit to prevent misconfiguration. For high-throughput systems, + // consider the relationship between this rate limit, thread pool size, and storage backend capacity. + ConfigDef.Importance.MEDIUM, + FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_DOC + ); configDef.define( FETCH_FIND_BATCHES_MAX_BATCHES_PER_PARTITION_CONFIG, ConfigDef.Type.INT, @@ -362,7 +434,69 @@ public InklessConfig(final AbstractConfig config) { } public InklessConfig(final Map props) { - super(configDef(), props); + super(validate(props), props); + } + + private static ConfigDef validate(final Map props) { + final ConfigDef configDef = configDef(); + // Parse the properties using ConfigDef directly for validation. This avoids creating a + // temporary AbstractConfig instance while still leveraging the same parsing and defaulting + // logic that AbstractConfig would use. Note: We still parse twice (once here, once in super()), + // but this avoids the overhead of creating an AbstractConfig instance. This is necessary to + // avoid 'this-escape' warnings in JDK 23+ and ensure super() is the first statement for JDK 17. + // The performance impact is minimal since config parsing only happens at startup. + final Map parsedProps = configDef.parse(props); + + final long thresholdMs = + ((Number) parsedProps.get(FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG)).longValue(); + final int cacheLifespanSec = + ((Number) parsedProps.get(CONSUME_CACHE_EXPIRATION_LIFESPAN_SEC_CONFIG)).intValue(); + final long lifespanMs = Duration.ofSeconds(cacheLifespanSec).toMillis(); + + // Validate threshold is not less than cache lifespan (unless using default heuristic). + // If threshold < cache lifespan, we'd route requests for potentially cached data to the + // cold path, defeating the cache and unnecessarily loading the cold path/storage backend. + // + // Note: This validation occurs at construction time. These configurations are startup-only + // and do not support dynamic reconfiguration. Both threshold and cache lifespan must be set + // together at startup to maintain the constraint that threshold >= cache lifespan. + // + // Explicitly reject threshold=0 with a clear error message: While threshold=0 would always fail + // the cache lifespan validation below (since minimum cache lifespan is 10 seconds = 10000ms), + // we check it explicitly here to provide a more specific error message that explains why 0 is + // invalid. With threshold=0, the runtime check (dataAge > threshold) would route almost all cached + // data (anything with dataAge > 0) to the cold path, defeating the cache. Only data with + // dataAge == 0 would use the hot path, which is negligible. + if (thresholdMs == 0) { + throw new ConfigException( + FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG, + thresholdMs, + "Lagging consumer threshold cannot be 0. Use -1 to auto-detect from cache TTL, or set a value >= cache lifespan (" + + lifespanMs + "ms). Threshold=0 would route almost all cached data to the cold path, defeating the cache." + ); + } + // + // Minimum allowed value: threshold == cache lifespan (>=, not >) is valid because: + // - The runtime check uses dataAge > threshold (strictly greater), so dataAge == threshold uses hot path + // - Data can still be in cache at exactly TTL seconds old (cache expiration runs periodically) + // - With threshold == cache lifespan, when dataAge == cache lifespan, data might still be cached + // and correctly uses hot path. When dataAge > cache lifespan, data is expired and uses cold path. + // This ensures we only route data to cold path after it's guaranteed to be expired from cache. + // + // Special case: thresholdMs == -1 is explicitly excluded from validation (condition checks != -1) + // because fetchLaggingConsumerThresholdMs() will automatically use cache lifespan as the effective + // runtime value, which is always >= cache lifespan by definition. This design allows operators + // to use -1 as a "use cache TTL" heuristic without needing to know the exact cache lifespan value. + if (thresholdMs != -1 && thresholdMs < lifespanMs) { + throw new ConfigException( + FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG, + thresholdMs, + "Lagging consumer threshold (" + thresholdMs + "ms) must be >= cache lifespan (" + + lifespanMs + "ms) to avoid routing requests for cached data to the lagging path." + ); + } + + return configDef; } @SuppressWarnings("unchecked") @@ -459,6 +593,34 @@ public int fetchMetadataThreadPoolSize() { return getInt(FETCH_METADATA_THREAD_POOL_SIZE_CONFIG); } + public int fetchLaggingConsumerThreadPoolSize() { + return getInt(FETCH_LAGGING_CONSUMER_THREAD_POOL_SIZE_CONFIG); + } + + /** + * Returns the effective lagging consumer threshold in milliseconds. + *

+ * If the configured value is -1 (auto), this method returns the cache expiration lifespan, + * which serves as the default heuristic. This ensures the effective threshold is always >= cache + * lifespan, which is why validation in the constructor skips threshold=-1 (it will automatically + * use cache lifespan at runtime). + *

+ * + * @return the effective threshold in milliseconds (cache lifespan if configured as -1, otherwise the configured value) + */ + public long fetchLaggingConsumerThresholdMs() { + final long configuredValue = getLong(FETCH_LAGGING_CONSUMER_THRESHOLD_MS_CONFIG); + if (configuredValue == -1) { + // Use heuristic: cache TTL (provides grace period for recent data) + return Duration.ofSeconds(getInt(CONSUME_CACHE_EXPIRATION_LIFESPAN_SEC_CONFIG)).toMillis(); + } + return configuredValue; + } + + public int fetchLaggingConsumerRequestRateLimit() { + return getInt(FETCH_LAGGING_CONSUMER_REQUEST_RATE_LIMIT_CONFIG); + } + public int maxBatchesPerPartitionToFind() { return getInt(FETCH_FIND_BATCHES_MAX_BATCHES_PER_PARTITION_CONFIG); } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java index ebd52d1c93..cab0762355 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java @@ -52,14 +52,14 @@ public class FetchCompleter implements Supplier fetchInfos; private final Map coordinates; - private final List backingData; + private final List backingData; private final Consumer durationCallback; public FetchCompleter(Time time, ObjectKeyCreator objectKeyCreator, Map fetchInfos, Map coordinates, - List backingData, + List backingData, Consumer durationCallback) { this.time = time; this.objectKeyCreator = objectKeyCreator; @@ -75,23 +75,29 @@ public Map get() { final Map> files = groupFileData(); return TimeUtils.measureDurationMs(time, () -> serveFetch(coordinates, files), durationCallback); } catch (Exception e) { - throw new FetchException(e); + throw new FetchException("Failed to complete fetch for partitions " + fetchInfos.keySet(), e); } } private Map> groupFileData() { Map> files = new HashMap<>(); - for (FileExtent fileExtent : backingData) { - files.compute(fileExtent.object(), (k, v) -> { - if (v == null) { - List out = new ArrayList<>(1); - out.add(fileExtent); - return out; - } else { - v.add(fileExtent); - return v; - } - }); + for (FileExtentResult result : backingData) { + // Only process successful fetches - failures are handled as missing data + // which results in KAFKA_STORAGE_ERROR in extractRecords/servePartition + if (result instanceof FileExtentResult.Success success) { + final FileExtent fileExtent = success.extent(); + files.compute(fileExtent.object(), (k, v) -> { + if (v == null) { + List out = new ArrayList<>(1); + out.add(fileExtent); + return out; + } else { + v.add(fileExtent); + return v; + } + }); + } + // Failure results are intentionally skipped - they don't contribute to files map } return files; } @@ -162,16 +168,30 @@ private FetchPartitionData servePartition(TopicIdPartition key, MapPartial Failure Handling: + * This method returns partial results if some batches fail to extract (missing files or null records). + * This is intentional to support partial failure scenarios where some batches succeed while others fail. + * The calling code (servePartition) will check if foundRecords is empty and return KAFKA_STORAGE_ERROR + * if no records were found, allowing successful batches to be returned when possible. + * + * @param metadata the batch metadata for the partition + * @param allFiles the map of object keys to fetched file extents + * @return list of memory records (may be partial if some batches failed) + */ private List extractRecords(FindBatchResponse metadata, Map> allFiles) { List foundRecords = new ArrayList<>(); for (BatchInfo batch : metadata.batches()) { List files = allFiles.get(objectKeyCreator.from(batch.objectKey()).value()); if (files == null || files.isEmpty()) { - // as soon as we encounter an error + // Missing file extent for this batch - return partial results (successful batches so far) return foundRecords; } MemoryRecords fileRecords = constructRecordsFromFile(batch, files); if (fileRecords == null) { + // Failed to construct records from file - return partial results (successful batches so far) return foundRecords; } foundRecords.add(fileRecords); diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchException.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchException.java index d8fa059249..e42a424172 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchException.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchException.java @@ -18,7 +18,7 @@ package io.aiven.inkless.consume; public class FetchException extends RuntimeException { - public FetchException(Throwable cause) { - super(cause); + public FetchException(String message, Throwable cause) { + super(message, cause); } } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchHandler.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchHandler.java index fb10490dd5..ee1c80c5ce 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchHandler.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchHandler.java @@ -55,6 +55,22 @@ public FetchHandler(final SharedState state) { state.brokerTopicStats(), state.config().fetchMetadataThreadPoolSize(), state.config().fetchDataThreadPoolSize(), + // Separate storage client for lagging consumers to: + // 1. Isolate connection pool usage (lagging consumers shouldn't exhaust connections for hot path) + // 2. Allow independent tuning of timeouts/retries for cold storage access patterns + // (This requires some refactoring on how the storage client is built/configured) + // If thread pool size is 0, disabling lagging consumer support, don't create a separate client + // + // NOTE: The client for lagging consumers is created only when this FetchHandler (and Reader) + // is constructed. If fetchLaggingConsumerThreadPoolSize() is 0 at this time, no separate + // client is created and lagging consumer support is effectively disabled for the lifetime + // of this instance, even if the configuration is later reloaded with a non-zero value. + // Enabling lagging consumer support therefore requires a broker restart (or reconstruction + // of the SharedState/FetchHandler) so that a new storage client can be created. + state.config().fetchLaggingConsumerThreadPoolSize() > 0 ? state.buildStorage() : null, + state.config().fetchLaggingConsumerThresholdMs(), + state.config().fetchLaggingConsumerRequestRateLimit(), + state.config().fetchLaggingConsumerThreadPoolSize(), state.config().maxBatchesPerPartitionToFind() ) ); diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java index 3c38ffd5e2..17104e022c 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java @@ -26,6 +26,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -41,6 +42,7 @@ import io.aiven.inkless.generated.CacheKey; import io.aiven.inkless.generated.FileExtent; import io.aiven.inkless.storage_backend.common.ObjectFetcher; +import io.github.bucket4j.Bucket; /** * Plans and executes fetch operations for diskless topics. @@ -64,7 +66,13 @@ public class FetchPlanner implements Supplier private final KeyAlignmentStrategy keyAlignment; private final ObjectCache cache; private final ObjectFetcher objectFetcher; - private final ExecutorService dataExecutor; + // Executor for fetching data from remote storage. + // Will only be used for recent data (hot path) if lagging consumer support feature is enabled. + private final ExecutorService fetchDataExecutor; + private final ExecutorService laggingFetchDataExecutor; + private final ObjectFetcher laggingObjectFetcher; + private final long laggingConsumerThresholdMs; + private final Bucket laggingRateLimiter; private final Map batchCoordinates; private final InklessFetchMetrics metrics; @@ -74,7 +82,11 @@ public FetchPlanner( KeyAlignmentStrategy keyAlignment, ObjectCache cache, ObjectFetcher objectFetcher, - ExecutorService dataExecutor, + ExecutorService fetchDataExecutor, + ObjectFetcher laggingObjectFetcher, + long laggingConsumerThresholdMs, + Bucket laggingRateLimiter, + ExecutorService laggingFetchDataExecutor, Map batchCoordinates, InklessFetchMetrics metrics ) { @@ -83,7 +95,11 @@ public FetchPlanner( this.keyAlignment = keyAlignment; this.cache = cache; this.objectFetcher = objectFetcher; - this.dataExecutor = dataExecutor; + this.fetchDataExecutor = fetchDataExecutor; + this.laggingObjectFetcher = laggingObjectFetcher; + this.laggingFetchDataExecutor = laggingFetchDataExecutor; + this.laggingConsumerThresholdMs = laggingConsumerThresholdMs; + this.laggingRateLimiter = laggingRateLimiter; this.batchCoordinates = batchCoordinates; this.metrics = metrics; } @@ -108,12 +124,16 @@ public List> get() { /** * Plans fetch jobs from batch coordinates. * + *

This method filters out responses with errors and groups batches by object key. + * If all responses have errors or batchCoordinates is empty, this returns an empty list, + * which is handled gracefully by the caller (no fetch requests are submitted). + * * @param batchCoordinates map of partition to batch information - * @return list of planned fetch requests + * @return list of planned fetch requests (empty if no valid batches found) */ List planJobs(final Map batchCoordinates) { - // Group batches by object key - final Set>> objectKeysToRanges = + // Group batches by object key and collect full BatchInfo for metadata aggregation + final Set>> objectKeysToBatches = batchCoordinates.values().stream() // Filter out responses with errors .filter(findBatch -> findBatch.errors() == Errors.NONE) @@ -122,34 +142,52 @@ List planJobs(final Map .flatMap(List::stream) .collect(Collectors.groupingBy( BatchInfo::objectKey, - Collectors.mapping(batch -> batch.metadata().range(), Collectors.toList()) + Collectors.toList() )) .entrySet(); - metrics.recordFetchObjectsSize(objectKeysToRanges.size()); + metrics.recordFetchObjectsSize(objectKeysToBatches.size()); - // Create fetch requests with aligned byte ranges - return objectKeysToRanges.stream() + // Create fetch requests with aligned byte ranges and aggregated metadata + return objectKeysToBatches.stream() .flatMap(e -> createFetchRequests(e.getKey(), e.getValue())) .collect(Collectors.toList()); } /** * Creates fetch requests for a single object with multiple batches. - * Aligns byte ranges for efficient fetching. + * Aligns byte ranges and aggregates metadata (timestamp for hot/cold path decision). */ private Stream createFetchRequests( final String objectKey, - final List byteRanges + final List batches ) { + if (batches.isEmpty()) { + return Stream.empty(); // Defensive: shouldn't happen, but handle gracefully + } + + // Use max timestamp for hot/cold path decision. + // Objects contain multi-partition data, so if ANY batch is recent, use hot path (cache + priority). + // Since we check for empty batches above, max() is guaranteed to have a value here. + final long timestamp = batches.stream() + .mapToLong(b -> b.metadata().batchMaxTimestamp()) + .max() + .getAsLong(); + + // Extract byte ranges + final List byteRanges = batches.stream() + .map(b -> b.metadata().range()) + .collect(Collectors.toList()); + // Align byte ranges for efficient fetching final Set alignedRanges = keyAlignment.align(byteRanges); - // Create a fetch request for each aligned range + // Create a fetch request for each aligned range with aggregated metadata return alignedRanges.stream() .map(byteRange -> new ObjectFetchRequest( objectKeyCreator.from(objectKey), - byteRange + byteRange, + timestamp )); } @@ -160,28 +198,102 @@ private List> submitAllRequests(final List submitSingleRequest(final ObjectFetchRequest request) { - // Cache API performs lookup on the calling thread and returns a CompletableFuture immediately (non-blocking). - // On cache miss, fetchFileExtent executes on dataExecutor; on cache hit, a completed future is returned immediately. - return cache.computeIfAbsent( - request.toCacheKey(), - k -> fetchFileExtent(request), - dataExecutor - ); + final long currentTime = time.milliseconds(); + // If timestamp is in the future (clock skew), treat as recent data (hot path) + // Math.max ensures dataAge is never negative, which would incorrectly route future timestamps + final long dataAge = Math.max(0, currentTime - request.timestamp()); + + // Lagging consumer feature is enabled only when BOTH laggingFetchDataExecutor AND laggingObjectFetcher are non-null. + // If either is null, the feature is disabled and all requests are treated as recent (hot path). + final boolean laggingFeatureEnabled = laggingFetchDataExecutor != null && laggingObjectFetcher != null; + final boolean isLagging = laggingFeatureEnabled && (dataAge > laggingConsumerThresholdMs); + + if (!isLagging) { + // Hot path: up-to-date consumers use cache + recentDataExecutor + metrics.recordRecentDataRequest(); + return cache.computeIfAbsent( + request.toCacheKey(), + k -> fetchFileExtent(objectFetcher, request), + fetchDataExecutor + ); + } else { + // Cold path: lagging consumers bypass cache, use dedicated executor with rate limiting. + // Cache bypass rationale: Objects are multi-partition blobs, caching them would evict hot data + // and provide little benefit to the lagging consumer. Backpressure via AbortPolicy: queue full + // → RejectedExecutionException → Kafka error handler → consumer backs off (fetch purgatory). + metrics.recordLaggingConsumerRequest(); + try { + return CompletableFuture.supplyAsync(() -> { + // Apply rate limiting if configured (rate limit > 0) + if (laggingRateLimiter != null) { + applyRateLimit(); // InterruptedException here is wrapped in FetchException + } + return fetchFileExtent(laggingObjectFetcher, request); + }, + laggingFetchDataExecutor + ).whenComplete((result, throwable) -> { + // Track async rejections that occur during task execution: + // - RejectedExecutionException: rejection when executor is shut down or queue is full. + // This may surface either as the throwable itself or as its cause (e.g., via CompletionException). + // Note: InterruptedException and FetchException are treated as fetch operation failures, + // not as backpressure/capacity rejections. + if (throwable != null) { + final Throwable cause = throwable.getCause(); + final boolean isRejected = + throwable instanceof RejectedExecutionException + || cause instanceof RejectedExecutionException; + if (isRejected) { + metrics.recordLaggingConsumerRejection(); + } + } + }); + } catch (final RejectedExecutionException e) { + // Sync rejection (executor shut down or queue full at submission) - return failed future + // instead of propagating exception. This allows allOfFileExtents to handle the failure + // gracefully, returning empty FileExtent for this partition while other partitions + // (hot path) can still succeed. + // Metrics recorded here since the task never executes (vs async rejection tracked in whenComplete). + metrics.recordLaggingConsumerRejection(); + return CompletableFuture.failedFuture(e); + } + } + } + + // Applies request-based rate limiting by blocking executor thread until token available. + // Always records wait time (including zero-wait) for accurate latency histogram. + // Note: If interrupted, the duration is still recorded before the exception is thrown. + private void applyRateLimit() { + TimeUtils.measureDurationMs(time, () -> { + try { + laggingRateLimiter.asBlocking().consume(1); + } catch (final InterruptedException e) { + // Rate limit wait was interrupted (typically during shutdown). + // Preserve interrupt status for executor framework, but wrap in FetchException + // to indicate this is a fetch failure, not a rejection (executor capacity issue). + // + // Note: This is distinct from executor shutdown interruption caught in whenComplete, + // which occurs during fetchFileExtent and IS tracked as rejection. Rate limit + // interruption is a fetch operation failure, not a capacity/backpressure issue. + // + // The duration is still recorded by measureDurationMs even when an exception is thrown. + Thread.currentThread().interrupt(); + throw new FetchException("Rate limit wait interrupted for lagging consumer", e); + } + }, metrics::recordRateLimitWaitTime); } /** * Fetches a file extent from remote storage. - * This method is called by the cache (on dataExecutor) when a cache miss occurs. * * @param request the fetch request with object key and byte range * @return the fetched file extent * @throws FileFetchException if remote fetch fails */ - private FileExtent fetchFileExtent(final ObjectFetchRequest request) { + private FileExtent fetchFileExtent(final ObjectFetcher fetcher, final ObjectFetchRequest request) { try { final FileFetchJob job = new FileFetchJob( time, - objectFetcher, + fetcher, request.objectKey(), request.byteRange(), metrics::fetchFileFinished @@ -203,11 +315,25 @@ private FileExtent fetchFileExtent(final ObjectFetchRequest request) { * * @param objectKey the storage object key * @param byteRange the range of bytes to fetch + * @param timestamp the maximum timestamp from batches (for hot/cold path decision). + * Using max instead of min because if ANY batch in the object is recent, + * we treat the entire fetch as hot path to prioritize recent data access. */ record ObjectFetchRequest( ObjectKey objectKey, - ByteRange byteRange + ByteRange byteRange, + long timestamp ) { + /** + * Converts to cache key for deduplication. + * + *

Note: timestamp is intentionally excluded from cache key. + * Timestamp determines hot/cold path routing but doesn't affect cache identity: + * only the hot path uses the object cache, while the cold path bypasses the cache + * entirely. For a given object+range, the returned data is identical regardless of + * timestamp, so using just object+range as the cache key is safe and ensures both + * hot-path fetches for the same region share a single cache entry. + */ CacheKey toCacheKey() { return new CacheKey() .setObject(objectKey.value()) diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java new file mode 100644 index 0000000000..9d836b5feb --- /dev/null +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java @@ -0,0 +1,57 @@ +/* + * Inkless + * Copyright (C) 2024 - 2025 Aiven OY + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package io.aiven.inkless.consume; + +import io.aiven.inkless.generated.FileExtent; + +/** + * Result type for file extent fetch operations that can either succeed with data or fail with an error. + * + *

This type explicitly models the success/failure states of fetching file extents, avoiding + * the use of sentinel values (like empty FileExtent) to signal failures. This makes the code + * more type-safe and self-documenting. + * + *

Usage in fetch pipeline: + *

    + *
  • FetchPlanner returns CompletableFuture<FileExtentResult> for each object fetch
  • + *
  • Reader.allOfFileExtents converts exceptions to Failure instances
  • + *
  • FetchCompleter pattern-matches on Success/Failure to build responses
  • + *
+ */ +public sealed interface FileExtentResult { + /** + * Successful file extent fetch with data. + * + * @param extent the fetched file extent containing object data + */ + record Success(FileExtent extent) implements FileExtentResult {} + + /** + * Failed file extent fetch with error information. + * + *

Common failure scenarios: + *

    + *
  • RejectedExecutionException: lagging consumer executor saturated
  • + *
  • StorageBackendException: remote storage fetch failed
  • + *
  • FileFetchException: file fetch or processing failed
  • + *
+ * + * @param error the exception that caused the fetch to fail + */ + record Failure(Throwable error) implements FileExtentResult {} +} diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FindBatchesJob.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FindBatchesJob.java index 5791d0f6a4..1640ef8eb6 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FindBatchesJob.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FindBatchesJob.java @@ -72,6 +72,19 @@ private Map doWork() { List responses = controlPlane.findBatches(requests, params.maxBytes, maxBatchesPerPartition); + // Validate that control plane returned the expected number of responses. + // Control plane contract requires responses to be in the same order as requests: + // responses[i] must correspond to requests[i] for all i. + // This ensures index-based mapping is safe and partitions are mapped correctly. + if (responses.size() != requests.size()) { + throw new IllegalStateException( + "Control plane returned " + responses.size() + " responses but " + requests.size() + + " were requested. Responses must match requests in count and order." + ); + } + + // Map responses to partitions by index, relying on control plane contract that + // responses[i] corresponds to requests[i] for all i. Map out = new HashMap<>(); for (int i = 0; i < requests.size(); i++) { FindBatchRequest request = requests.get(i); diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java index 0996bc6ad1..107f7efeab 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java @@ -55,6 +55,14 @@ public class InklessFetchMetrics { private static final String FETCH_PARTITIONS_PER_FETCH_COUNT = "FetchPartitionsPerFetchCount"; private static final String FETCH_BATCHES_PER_FETCH_COUNT = "FetchBatchesPerPartitionCount"; private static final String FETCH_OBJECTS_PER_FETCH_COUNT = "FetchObjectsPerFetchCount"; + private static final String RECENT_DATA_REQUEST_RATE = "RecentDataRequestRate"; + private static final String LAGGING_CONSUMER_REQUEST_RATE = "LaggingConsumerRequestRate"; + private static final String LAGGING_CONSUMER_REJECTED_RATE = "LaggingConsumerRejectedRate"; + // Tracks wait time (including zero-wait) for ALL lagging consumer requests when rate limiting is enabled. + // When rate limiter is disabled (config = 0), LaggingConsumerRequestRate > 0 but this metric rate = 0. + // Always records wait time to avoid histogram bias - zero-wait cases show when rate limiting is NOT a bottleneck. + // Use to monitor: rate limiting latency distribution, actual throttling pressure, and limiter effectiveness. + private static final String LAGGING_RATE_LIMIT_WAIT_TIME = "LaggingRateLimitWaitTime"; private final Time time; @@ -78,6 +86,10 @@ public class InklessFetchMetrics { private final Histogram fetchPartitionSizeHistogram; private final Histogram fetchBatchesSizeHistogram; private final Histogram fetchObjectsSizeHistogram; + private final Meter recentDataRequestRate; + private final Meter laggingConsumerRequestRate; + private final Meter laggingConsumerRejectedRate; + private final Histogram laggingRateLimitWaitTime; public InklessFetchMetrics(final Time time, final ObjectCache cache) { this.time = Objects.requireNonNull(time, "time cannot be null"); @@ -100,7 +112,10 @@ public InklessFetchMetrics(final Time time, final ObjectCache cache) { fetchObjectsSizeHistogram = metricsGroup.newHistogram(FETCH_OBJECTS_PER_FETCH_COUNT, true, Map.of()); cacheEntrySize = metricsGroup.newHistogram(CACHE_ENTRY_SIZE, true, Map.of()); cacheSize = metricsGroup.newGauge(CACHE_SIZE, () -> cache.size()); - + recentDataRequestRate = metricsGroup.newMeter(RECENT_DATA_REQUEST_RATE, "requests", TimeUnit.SECONDS, Map.of()); + laggingConsumerRequestRate = metricsGroup.newMeter(LAGGING_CONSUMER_REQUEST_RATE, "requests", TimeUnit.SECONDS, Map.of()); + laggingConsumerRejectedRate = metricsGroup.newMeter(LAGGING_CONSUMER_REJECTED_RATE, "rejections", TimeUnit.SECONDS, Map.of()); + laggingRateLimitWaitTime = metricsGroup.newHistogram(LAGGING_RATE_LIMIT_WAIT_TIME, true, Map.of()); } public void fetchCompleted(Instant startAt) { @@ -180,6 +195,10 @@ public void close() { metricsGroup.removeMetric(FETCH_PARTITIONS_PER_FETCH_COUNT); metricsGroup.removeMetric(FETCH_BATCHES_PER_FETCH_COUNT); metricsGroup.removeMetric(FETCH_OBJECTS_PER_FETCH_COUNT); + metricsGroup.removeMetric(RECENT_DATA_REQUEST_RATE); + metricsGroup.removeMetric(LAGGING_CONSUMER_REQUEST_RATE); + metricsGroup.removeMetric(LAGGING_CONSUMER_REJECTED_RATE); + metricsGroup.removeMetric(LAGGING_RATE_LIMIT_WAIT_TIME); } public void fetchStarted(int partitionSize) { @@ -194,4 +213,69 @@ public void recordFetchBatchSize(int size) { public void recordFetchObjectsSize(int size) { fetchObjectsSizeHistogram.update(size); } + + /** + * Records a request that used the hot path (recent data with cache). + * Metric: RecentDataRequestRate + */ + public void recordRecentDataRequest() { + recentDataRequestRate.mark(); + } + + /** + * Records a request that used the cold path (lagging consumer, bypasses cache). + * This is recorded for ALL cold path requests, regardless of rate limiting. + * Metric: LaggingConsumerRequestRate + * + * @see #recordRateLimitWaitTime(long) for requests that were actually rate limited + */ + public void recordLaggingConsumerRequest() { + laggingConsumerRequestRate.mark(); + } + + /** + * Records a lagging consumer request that was rejected due to executor unavailability. + * This typically corresponds to: + * - RejectedExecutionException: Queue full (AbortPolicy triggered) + * + * In this case, backpressure is applied: the consumer receives an error response + * and backs off via fetch purgatory. + * Metric: LaggingConsumerRejectedRate + * + * High rejection rate indicates: + * - Sustained lagging consumer load exceeding capacity + * - May need to increase fetch.lagging.consumer.thread.pool.size + * - Or increase fetch.lagging.consumer.request.rate.limit + * - Or consumers are genuinely too far behind (acceptable backpressure) + * - Or executor is shutting down (transient) + */ + public void recordLaggingConsumerRejection() { + laggingConsumerRejectedRate.mark(); + } + + /** + * Records wait time for lagging consumer requests when rate limiting is enabled. + * This is recorded for ALL lagging consumer requests processed through the rate limiter, + * including zero-wait cases (when tokens are immediately available). + * + *

Recording zero-wait cases is intentional to avoid histogram bias. Zero-wait entries + * show when rate limiting is NOT a bottleneck, which is valuable monitoring data. + * + *

Metric: LaggingRateLimitWaitTime (Histogram) + * + *

Relationship: + * - When rate limiting is ENABLED: LaggingRateLimitWaitTime.Rate ≈ LaggingConsumerRequestRate + * - When rate limiting is DISABLED: LaggingRateLimitWaitTime.Rate = 0, LaggingConsumerRequestRate > 0 + * + *

Use this metric to: + * - Monitor rate limiting latency distribution (including p50, p99, p999) + * - Identify when rate limiting becomes a bottleneck (high percentiles) + * - Verify rate limiting is working (rate should match LaggingConsumerRequestRate when enabled) + * - Detect configuration issues (if rate is 0 but LaggingConsumerRequestRate > 0, rate limiting is disabled) + * + * @param waitMs Wait time in milliseconds (can be 0 if token was immediately available) + */ + public void recordRateLimitWaitTime(long waitMs) { + laggingRateLimitWaitTime.update(waitMs); + } } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java index 548ae0d4ce..8b9719ea38 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java @@ -33,10 +33,12 @@ import java.time.Instant; import java.util.List; import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import io.aiven.inkless.TimeUtils; @@ -48,10 +50,25 @@ import io.aiven.inkless.control_plane.ControlPlane; import io.aiven.inkless.generated.FileExtent; import io.aiven.inkless.storage_backend.common.ObjectFetcher; +import io.github.bucket4j.Bandwidth; +import io.github.bucket4j.Bucket; public class Reader implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(Reader.class); private static final long EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS = 5; + + /** + * Queue capacity multiplier for lagging consumer executor. + * + *

The queue capacity is automatically calculated as {@code thread.pool.size * LAGGING_CONSUMER_QUEUE_MULTIPLIER}. + * This provides burst buffering while preventing unbounded growth. When full, AbortPolicy + * rejects tasks and exceptions propagate to Kafka's error handler. + * + *

Example: With default 16 threads and multiplier of 100, capacity = 1600 tasks. + * At 200 req/s rate limit, this provides ~8 seconds of buffer before rejections occur. + * Note: These values are derived from defaults and may differ if configuration is changed. + */ + private static final int LAGGING_CONSUMER_QUEUE_MULTIPLIER = 100; private final Time time; private final ObjectKeyCreator objectKeyCreator; private final KeyAlignmentStrategy keyAlignmentStrategy; @@ -60,11 +77,24 @@ public class Reader implements AutoCloseable { private final ObjectFetcher objectFetcher; private final int maxBatchesPerPartitionToFind; private final ExecutorService metadataExecutor; - private final ExecutorService dataExecutor; + private final ExecutorService fetchDataExecutor; + private final long laggingConsumerThresholdMs; + private final ExecutorService laggingFetchDataExecutor; + /** + * Separate ObjectFetcher for lagging consumer requests to provide resource isolation. + * + *

Prevents lagging consumer bursts from exhausting hot path resources (connection pools, + * HTTP/2 streams, rate limits). Cold path failures don't propagate to hot path. + * + *

Trade-off: Doubles connection pools (~500KB) but prevents hot path degradation. + */ + private final ObjectFetcher laggingConsumerObjectFetcher; private final InklessFetchMetrics fetchMetrics; private final BrokerTopicStats brokerTopicStats; + private final Bucket rateLimiter; private ThreadPoolMonitor metadataThreadPoolMonitor; private ThreadPoolMonitor dataThreadPoolMonitor; + private ThreadPoolMonitor laggingConsumerThreadPoolMonitor; public Reader( Time time, @@ -76,6 +106,10 @@ public Reader( BrokerTopicStats brokerTopicStats, int fetchMetadataThreadPoolSize, int fetchDataThreadPoolSize, + ObjectFetcher laggingObjectFetcher, + long laggingConsumerThresholdMs, + int laggingConsumerRequestRateLimit, + int laggingConsumerThreadPoolSize, int maxBatchesPerPartitionToFind ) { this( @@ -88,11 +122,38 @@ public Reader( maxBatchesPerPartitionToFind, Executors.newFixedThreadPool(fetchMetadataThreadPoolSize, new InklessThreadFactory("inkless-fetch-metadata-", false)), Executors.newFixedThreadPool(fetchDataThreadPoolSize, new InklessThreadFactory("inkless-fetch-data-", false)), + // Only create lagging consumer fetcher when feature is enabled (pool size > 0). + // A pool size of 0 is a valid configuration that explicitly disables the feature (null fetcher and executor). + laggingConsumerThreadPoolSize > 0 ? laggingObjectFetcher : null, + laggingConsumerThresholdMs, + laggingConsumerRequestRateLimit, + // Only create lagging consumer resources when feature is enabled (pool size > 0). + // A pool size of 0 is a valid configuration that explicitly disables the feature + // by passing both a null executor and a null laggingObjectFetcher. + laggingConsumerThreadPoolSize > 0 + ? createBoundedThreadPool(laggingConsumerThreadPoolSize) + : null, new InklessFetchMetrics(time, cache), brokerTopicStats ); } + private static ExecutorService createBoundedThreadPool(int poolSize) { + // Creates a bounded thread pool for lagging consumer fetch requests. + // Fixed pool design: all threads persist for executor lifetime (never removed when idle). + final int queueCapacity = poolSize * LAGGING_CONSUMER_QUEUE_MULTIPLIER; + return new ThreadPoolExecutor( + poolSize, // corePoolSize: fixed pool, always this many threads + poolSize, // maximumPoolSize: no dynamic scaling (core == max) + 0L, // keepAliveTime: unused for fixed pools (core threads don't time out) + TimeUnit.MILLISECONDS, + new ArrayBlockingQueue<>(queueCapacity), // Bounded queue prevents OOM + new InklessThreadFactory("inkless-fetch-lagging-consumer-", false), + // Why AbortPolicy: CallerRunsPolicy would block request handler threads causing broker-wide degradation + new ThreadPoolExecutor.AbortPolicy() // Reject when full, don't block callers + ); + } + // visible for testing Reader( Time time, @@ -103,7 +164,11 @@ public Reader( ObjectFetcher objectFetcher, int maxBatchesPerPartitionToFind, ExecutorService metadataExecutor, - ExecutorService dataExecutor, + ExecutorService fetchDataExecutor, + ObjectFetcher laggingConsumerObjectFetcher, + long laggingConsumerThresholdMs, + int laggingConsumerRequestRateLimit, + ExecutorService laggingFetchDataExecutor, InklessFetchMetrics fetchMetrics, BrokerTopicStats brokerTopicStats ) { @@ -115,12 +180,59 @@ public Reader( this.objectFetcher = objectFetcher; this.maxBatchesPerPartitionToFind = maxBatchesPerPartitionToFind; this.metadataExecutor = metadataExecutor; - this.dataExecutor = dataExecutor; + this.fetchDataExecutor = fetchDataExecutor; + this.laggingFetchDataExecutor = laggingFetchDataExecutor; + this.laggingConsumerThresholdMs = laggingConsumerThresholdMs; + this.laggingConsumerObjectFetcher = laggingConsumerObjectFetcher; + + // Validate that lagging consumer resources are consistently configured: + // both executor and fetcher must be null (feature disabled) or both must be non-null (feature enabled). + // This ensures fail-fast behavior rather than silent runtime failure. + if ((laggingFetchDataExecutor == null) != (laggingConsumerObjectFetcher == null)) { + throw new IllegalArgumentException( + "Lagging consumer feature requires both laggingFetchDataExecutor and laggingConsumerObjectFetcher " + + "to be non-null (feature enabled) or both to be null (feature disabled). " + + "Found: executor=" + (laggingFetchDataExecutor != null ? "non-null" : "null") + + ", fetcher=" + (laggingConsumerObjectFetcher != null ? "non-null" : "null") + ); + } + + // Initialize rate limiter only if lagging consumer feature is enabled (executor exists) and rate limit > 0 + // This avoids creating unused objects when the feature is disabled + if (laggingFetchDataExecutor != null && laggingConsumerRequestRateLimit > 0) { + // Rate limiter configuration: + // - capacity = rateLimit: Allows initial burst up to full rate limit (e.g., 200 tokens for 200 req/s) + // - refillGreedy: Refills at rateLimit tokens per second + // Note: While the rate limiter allows bursts up to capacity, the effective burst is limited by + // the thread pool size (e.g., 16 threads). Even if 200 tokens are available, only 16 requests + // can execute concurrently. The rate limiter controls request rate over time, while the thread + // pool limits concurrent execution, providing both rate limiting and concurrency control. + final Bandwidth limit = Bandwidth.builder() + .capacity(laggingConsumerRequestRateLimit) + .refillGreedy(laggingConsumerRequestRateLimit, java.time.Duration.ofSeconds(1)) + .build(); + this.rateLimiter = Bucket.builder() + .addLimit(limit) + .build(); + } else { + this.rateLimiter = null; + } + this.fetchMetrics = fetchMetrics; this.brokerTopicStats = brokerTopicStats; try { - this.metadataThreadPoolMonitor = new ThreadPoolMonitor("inkless-fetch-metadata", metadataExecutor); - this.dataThreadPoolMonitor = new ThreadPoolMonitor("inkless-fetch-data", dataExecutor); + // Initialize all monitors first, then assign to fields to ensure all-or-nothing semantics. + // If any monitor creation fails, none are assigned, preventing inconsistent monitoring state. + final ThreadPoolMonitor metadataMonitor = new ThreadPoolMonitor("inkless-fetch-metadata", metadataExecutor); + final ThreadPoolMonitor dataMonitor = new ThreadPoolMonitor("inkless-fetch-data", fetchDataExecutor); + // Only create monitor if lagging consumer executor exists (feature enabled) + final ThreadPoolMonitor laggingMonitor = laggingFetchDataExecutor != null + ? new ThreadPoolMonitor("inkless-fetch-lagging-consumer", laggingFetchDataExecutor) + : null; + // All monitors created successfully, assign to fields + this.metadataThreadPoolMonitor = metadataMonitor; + this.dataThreadPoolMonitor = dataMonitor; + this.laggingConsumerThreadPoolMonitor = laggingMonitor; } catch (final Exception e) { // only expected to happen on tests passing other types of pools LOGGER.warn("Failed to create thread pool monitors", e); @@ -132,7 +244,7 @@ public Reader( * *

1. Finds batches (metadataExecutor); *

2. Plans and submits fetches (completing thread); - *

3. Fetches data (dataExecutor); + *

3. Fetches data (dataExecutor or laggingFetchDataExecutor); *

4. Assembles final response (completing thread) * *

Stages 2 and 4 use non-async methods (thenApply, thenCombine), so they run on @@ -140,6 +252,13 @@ public Reader( * are lightweight CPU-bound tasks that complete quickly (1-5ms), and the I/O operations * are already fully non-blocking (handled by dataExecutor). * + *

Note on metadata executor sharing: The metadataExecutor (stage 1) is shared + * between hot and cold path requests. The hot/cold path separation only applies to data + * fetching (stage 3), which occurs after metadata is retrieved. A burst of lagging consumer + * requests can still compete with recent consumer requests at the metadata layer. Consider + * increasing {@code fetch.metadata.thread.pool.size} if metadata fetching becomes a + * bottleneck in mixed hot/cold workloads. + * * @param params fetch parameters * @param fetchInfos partitions and offsets to fetch * @return CompletableFuture with fetch results per partition @@ -168,14 +287,19 @@ public CompletableFuture> fetch( // Runs on the thread that completed batchCoordinates (metadataExecutor thread) return batchCoordinates.thenApply(coordinates -> { // FetchPlanner creates a plan and submits requests to cache - // Returns List> immediately + // Recent vs lagging consumer path separation happens here based on data age + // Rate limiter is reused across all requests for consistent rate limiting return new FetchPlanner( time, objectKeyCreator, keyAlignmentStrategy, cache, objectFetcher, - dataExecutor, // Will execute remote fetches on this pool + fetchDataExecutor, + laggingConsumerObjectFetcher, + laggingConsumerThresholdMs, + rateLimiter, + laggingFetchDataExecutor, coordinates, fetchMetrics ).get(); @@ -234,19 +358,47 @@ public CompletableFuture> fetch( /** * Waits for all file extent futures to complete and collects results in order. * + *

Partial Failure Handling: + * If some futures fail (e.g., lagging consumer rejection) while others succeed (hot path), + * this method converts failed fetches to Failure results instead of failing the entire request. + * This allows successful partitions to return data while failed partitions are marked as failures, + * enabling consumers to retry only the failed partitions. + * * @param fileExtentFutures the list of futures to wait for - * @return a future that completes with a list of file extents in the same order as the input + * @return a future that completes with a list of file extent results in the same order as the input */ - static CompletableFuture> allOfFileExtents( + static CompletableFuture> allOfFileExtents( List> fileExtentFutures ) { - final CompletableFuture[] futuresArray = fileExtentFutures.toArray(CompletableFuture[]::new); + // Handle each future individually to support partial failures. + // Convert exceptions to Failure results so successful partitions still get their data. + final List> handledFutures = fileExtentFutures.stream() + .map(future -> future + .handle((extent, throwable) -> { + if (throwable != null) { + // Restore interrupt status if the exception is InterruptedException. + // This callback may execute on various threads (executor threads, completing thread, etc.), + // but restoring interrupt status is safe and correct: it only sets a flag and doesn't stop execution. + // The interrupt flag is informational and allows code that checks Thread.interrupted() to see it. + if (throwable instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + // Log at debug level - metrics are recorded in FetchPlanner + LOGGER.debug("File extent fetch failed, returning failure result", throwable); + return new FileExtentResult.Failure(throwable); + } else { + return (FileExtentResult) new FileExtentResult.Success(extent); + } + })) + .toList(); + + final CompletableFuture[] futuresArray = handledFutures.toArray(CompletableFuture[]::new); return CompletableFuture.allOf(futuresArray) // The thenApply callback runs on whichever thread completes the last file // extent future (typically a dataExecutor thread or metadataExecutor when data is cached). // The join() calls are safe because all futures are already completed when this callback executes. .thenApply(v -> - fileExtentFutures.stream() + handledFutures.stream() .map(CompletableFuture::join) .toList()); } @@ -254,10 +406,13 @@ static CompletableFuture> allOfFileExtents( @Override public void close() throws IOException { ThreadUtils.shutdownExecutorServiceQuietly(metadataExecutor, EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS); - ThreadUtils.shutdownExecutorServiceQuietly(dataExecutor, EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS); + ThreadUtils.shutdownExecutorServiceQuietly(fetchDataExecutor, EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS); + ThreadUtils.shutdownExecutorServiceQuietly(laggingFetchDataExecutor, EXECUTOR_SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS); if (metadataThreadPoolMonitor != null) metadataThreadPoolMonitor.close(); if (dataThreadPoolMonitor != null) dataThreadPoolMonitor.close(); + if (laggingConsumerThreadPoolMonitor != null) laggingConsumerThreadPoolMonitor.close(); objectFetcher.close(); + if (laggingConsumerObjectFetcher != null) laggingConsumerObjectFetcher.close(); fetchMetrics.close(); } } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/control_plane/ControlPlane.java b/storage/inkless/src/main/java/io/aiven/inkless/control_plane/ControlPlane.java index cde4dad7df..a008b02154 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/control_plane/ControlPlane.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/control_plane/ControlPlane.java @@ -38,6 +38,15 @@ List commitFile( long fileSize, List batches); + /** + * Find batches for the given partition requests. + * + * @param findBatchRequests the list of partition requests + * @param fetchMaxBytes maximum bytes to fetch across all partitions + * @param maxBatchesPerPartition maximum batches per partition to return + * @return list of responses, one per request, in the same order as the requests. + * The contract requires that responses[i] corresponds to findBatchRequests[i] for all i. + */ List findBatches( List findBatchRequests, int fetchMaxBytes, diff --git a/storage/inkless/src/test/java/io/aiven/inkless/config/InklessConfigTest.java b/storage/inkless/src/test/java/io/aiven/inkless/config/InklessConfigTest.java index 6c3b22f12f..1db9ad002a 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/config/InklessConfigTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/config/InklessConfigTest.java @@ -130,6 +130,9 @@ void fullConfig() { configs.put("produce.upload.thread.pool.size", "16"); configs.put("fetch.data.thread.pool.size", "12"); configs.put("fetch.metadata.thread.pool.size", "14"); + configs.put("fetch.lagging.consumer.thread.pool.size", "20"); + configs.put("fetch.lagging.consumer.threshold.ms", "240000"); // 4 minutes + configs.put("fetch.lagging.consumer.request.rate.limit", "250"); configs.put("retention.enforcement.max.batches.per.request", "10"); final var config = new InklessConfig( configs @@ -151,6 +154,9 @@ void fullConfig() { assertThat(config.produceUploadThreadPoolSize()).isEqualTo(16); assertThat(config.fetchDataThreadPoolSize()).isEqualTo(12); assertThat(config.fetchMetadataThreadPoolSize()).isEqualTo(14); + assertThat(config.fetchLaggingConsumerThreadPoolSize()).isEqualTo(20); + assertThat(config.fetchLaggingConsumerThresholdMs()).isEqualTo(240_000L); + assertThat(config.fetchLaggingConsumerRequestRateLimit()).isEqualTo(250); assertThat(config.maxBatchesPerEnforcementRequest()).isEqualTo(10); } @@ -264,4 +270,298 @@ void consumeCacheSizeLessThanOne() { .isInstanceOf(ConfigException.class) .hasMessage("Invalid value 0 for configuration consume.cache.max.count: Value must be at least 1"); } + + @Test + void laggingConsumerConfigDefaults() { + // Test that lagging consumer configs have correct default values + final var config = new InklessConfig( + Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName() + ) + ); + + // Default thread pool size + assertThat(config.fetchLaggingConsumerThreadPoolSize()).isEqualTo(16); + + // Default rate limit + assertThat(config.fetchLaggingConsumerRequestRateLimit()).isEqualTo(200); + + // Default threshold: -1 (auto) should use heuristic: cache TTL + // Default cache TTL is 60 seconds, so threshold should follow that + assertThat(config.fetchLaggingConsumerThresholdMs()).isEqualTo(60_000L); + } + + @Test + void laggingConsumerConfigExplicitValues() { + // Test that explicit config values override defaults + final Map configs = new HashMap<>(); + configs.put("control.plane.class", InMemoryControlPlane.class.getCanonicalName()); + configs.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + configs.put("fetch.lagging.consumer.thread.pool.size", "32"); + configs.put("fetch.lagging.consumer.request.rate.limit", "500"); + configs.put("fetch.lagging.consumer.threshold.ms", "300000"); // 5 minutes explicit + + final var config = new InklessConfig(configs); + + assertThat(config.fetchLaggingConsumerThreadPoolSize()).isEqualTo(32); + assertThat(config.fetchLaggingConsumerRequestRateLimit()).isEqualTo(500); + assertThat(config.fetchLaggingConsumerThresholdMs()).isEqualTo(300_000L); + } + + @Test + void laggingConsumerThreadPoolSizeCanBeZero() { + // Test that thread pool size can be 0 (disables feature) + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "fetch.lagging.consumer.thread.pool.size", "0" + ); + + final var inklessConfig = new InklessConfig(config); + assertThat(inklessConfig.fetchLaggingConsumerThreadPoolSize()).isEqualTo(0); + } + + @Test + void laggingConsumerThreadPoolSizeNegativeInvalid() { + // Test that negative thread pool size is invalid + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "fetch.lagging.consumer.thread.pool.size", "-1" + ); + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessage("Invalid value -1 for configuration fetch.lagging.consumer.thread.pool.size: Value must be at least 0"); + } + + @Test + void laggingConsumerRateLimitCanBeZero() { + // Test that rate limit can be 0 (disables rate limiting) + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "fetch.lagging.consumer.request.rate.limit", "0" + ); + + final var inklessConfig = new InklessConfig(config); + assertThat(inklessConfig.fetchLaggingConsumerRequestRateLimit()).isEqualTo(0); + } + + @Test + void laggingConsumerRateLimitNegativeInvalid() { + // Test that negative rate limit is invalid + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "fetch.lagging.consumer.request.rate.limit", "-1" + ); + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessage("Invalid value -1 for configuration fetch.lagging.consumer.request.rate.limit: Value must be at least 0"); + } + + @Test + void laggingConsumerThresholdAutoHeuristic() { + // Test that threshold=-1 (auto) correctly applies cache TTL heuristic + final Map configs = new HashMap<>(); + configs.put("control.plane.class", InMemoryControlPlane.class.getCanonicalName()); + configs.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + configs.put("consume.cache.expiration.lifespan.sec", "300"); // 5 minutes + configs.put("fetch.lagging.consumer.threshold.ms", "-1"); // auto (default) + + final var config = new InklessConfig(configs); + + assertThat(config.fetchLaggingConsumerThresholdMs()).isEqualTo(300_000L); + } + + @Test + void laggingConsumerThresholdIndependentFromCacheTTL() { + // Test that explicit threshold decouples from cache TTL + final Map configs = new HashMap<>(); + configs.put("control.plane.class", InMemoryControlPlane.class.getCanonicalName()); + configs.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + configs.put("consume.cache.expiration.lifespan.sec", "60"); // 1 minute cache + configs.put("fetch.lagging.consumer.threshold.ms", "600000"); // 10 minutes threshold + + final var config = new InklessConfig(configs); + + // Threshold should be explicit value, not cache TTL + assertThat(config.cacheExpirationLifespanSec()).isEqualTo(60); + assertThat(config.fetchLaggingConsumerThresholdMs()).isEqualTo(600_000L); // Independent! + } + + @Test + void laggingConsumerThresholdBelowMinusOneInvalid() { + // Test that threshold < -1 is invalid + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "fetch.lagging.consumer.threshold.ms", "-2" + ); + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessage("Invalid value -2 for configuration fetch.lagging.consumer.threshold.ms: Value must be at least -1"); + } + + @Test + void laggingConsumerThresholdZeroInvalidWhenCacheEnabled() { + // Test that threshold=0 is invalid when cache lifespan > 0 + // This prevents routing cached data to the cold path + // Threshold=0 is explicitly rejected with a clear error message + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "consume.cache.expiration.lifespan.sec", "60", // Default 60 seconds + "fetch.lagging.consumer.threshold.ms", "0" + ); + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining("fetch.lagging.consumer.threshold.ms") + .hasMessageContaining("Lagging consumer threshold cannot be 0"); + } + + @Test + void laggingConsumerThresholdZeroValidWhenCacheMinimal() { + // Test that threshold can equal minimum cache lifespan (10 seconds) + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "consume.cache.expiration.lifespan.sec", "10", // Minimum allowed (10 seconds) + "fetch.lagging.consumer.threshold.ms", "10000" // 10 seconds - equal to cache lifespan + ); + + final var inklessConfig = new InklessConfig(config); + assertThat(inklessConfig.fetchLaggingConsumerThresholdMs()).isEqualTo(10000L); + } + + @Test + void laggingConsumerThresholdZeroInvalidWhenCacheMinimal() { + // Test that threshold=0 is invalid even with minimum cache lifespan (10 seconds) + // This ensures threshold=0 is always invalid when cache is enabled + // Threshold=0 is explicitly rejected with a clear error message + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "consume.cache.expiration.lifespan.sec", "10", // Minimum allowed (10 seconds = 10000ms) + "fetch.lagging.consumer.threshold.ms", "0" + ); + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining("fetch.lagging.consumer.threshold.ms") + .hasMessageContaining("Lagging consumer threshold cannot be 0"); + } + + @Test + void laggingConsumerRateLimitExceedsUpperBoundInvalid() { + // Test that rate limit exceeding upper bound (10000) is invalid + final Map config = Map.of( + "control.plane.class", InMemoryControlPlane.class.getCanonicalName(), + "storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName(), + "fetch.lagging.consumer.request.rate.limit", "10001" + ); + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining("fetch.lagging.consumer.request.rate.limit") + .hasMessageContaining("Value must be no more than 10000"); + } + + @Test + void laggingConsumerThresholdBelowCacheLifespanInvalid() { + // Test that threshold < cache lifespan is invalid when explicitly set + final Map config = new HashMap<>(); + config.put("control.plane.class", InMemoryControlPlane.class.getCanonicalName()); + config.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + config.put("consume.cache.expiration.lifespan.sec", "120"); // 2 minutes = 120000ms + config.put("fetch.lagging.consumer.threshold.ms", "60000"); // 1 minute - less than cache lifespan + + assertThatThrownBy(() -> new InklessConfig(config)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining("fetch.lagging.consumer.threshold.ms") + .hasMessageContaining("must be >= cache lifespan"); + } + + @Test + void laggingConsumerThresholdEqualToCacheLifespanValid() { + // Test that threshold == cache lifespan is valid + final Map config = new HashMap<>(); + config.put("control.plane.class", InMemoryControlPlane.class.getCanonicalName()); + config.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + config.put("consume.cache.expiration.lifespan.sec", "60"); // 1 minute = 60000ms + config.put("fetch.lagging.consumer.threshold.ms", "60000"); // 1 minute - equal to cache lifespan + + final var inklessConfig = new InklessConfig(config); + assertThat(inklessConfig.fetchLaggingConsumerThresholdMs()).isEqualTo(60_000L); + } + + @Test + void laggingConsumerThresholdAutoSkipsValidation() { + // Test that threshold=-1 (auto) skips validation even if cache lifespan is large + final Map config = new HashMap<>(); + config.put("control.plane.class", InMemoryControlPlane.class.getCanonicalName()); + config.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + config.put("consume.cache.expiration.lifespan.sec", "300"); // 5 minutes + config.put("fetch.lagging.consumer.threshold.ms", "-1"); // auto (default) + + // Should not throw - auto mode uses heuristic and skips validation + final var inklessConfig = new InklessConfig(config); + assertThat(inklessConfig.fetchLaggingConsumerThresholdMs()).isEqualTo(300_000L); + } + + @Test + void fullConfigWithLaggingConsumer() { + // Test complete configuration including all lagging consumer settings + final String controlPlaneClass = InMemoryControlPlane.class.getCanonicalName(); + Map configs = new HashMap<>(); + configs.put("control.plane.class", controlPlaneClass); + configs.put("object.key.prefix", "prefix/"); + configs.put("produce.commit.interval.ms", "100"); + configs.put("produce.buffer.max.bytes", "1024"); + configs.put("produce.max.upload.attempts", "5"); + configs.put("produce.upload.backoff.ms", "30"); + configs.put("storage.backend.class", ConfigTestStorageBackend.class.getCanonicalName()); + configs.put("file.cleaner.interval.ms", "100"); + configs.put("file.cleaner.retention.period.ms", "200"); + configs.put("file.merger.interval.ms", "100"); + configs.put("consume.cache.max.count", "100"); + configs.put("consume.cache.expiration.lifespan.sec", "200"); + configs.put("consume.cache.expiration.max.idle.sec", "100"); + configs.put("produce.upload.thread.pool.size", "16"); + configs.put("fetch.data.thread.pool.size", "12"); + configs.put("fetch.metadata.thread.pool.size", "14"); + configs.put("fetch.lagging.consumer.thread.pool.size", "24"); + configs.put("fetch.lagging.consumer.threshold.ms", "240000"); // 4 minutes (must be >= cache lifespan of 200 sec) + configs.put("fetch.lagging.consumer.request.rate.limit", "300"); + configs.put("retention.enforcement.max.batches.per.request", "10"); + + final var config = new InklessConfig(configs); + + assertThat(config.controlPlaneClass()).isEqualTo(InMemoryControlPlane.class); + assertThat(config.controlPlaneConfig()).isEqualTo(Map.of("class", controlPlaneClass)); + assertThat(config.objectKeyPrefix()).isEqualTo("prefix/"); + assertThat(config.commitInterval()).isEqualTo(Duration.ofMillis(100)); + assertThat(config.produceBufferMaxBytes()).isEqualTo(1024); + assertThat(config.produceMaxUploadAttempts()).isEqualTo(5); + assertThat(config.produceUploadBackoff()).isEqualTo(Duration.ofMillis(30)); + assertThat(config.storage(storageMetrics)).isInstanceOf(ConfigTestStorageBackend.class); + assertThat(config.fileCleanerInterval()).isEqualTo(Duration.ofMillis(100)); + assertThat(config.fileCleanerRetentionPeriod()).isEqualTo(Duration.ofMillis(200)); + assertThat(config.fileMergerInterval()).isEqualTo(Duration.ofMillis(100)); + assertThat(config.cacheMaxCount()).isEqualTo(100); + assertThat(config.cacheExpirationLifespanSec()).isEqualTo(200); + assertThat(config.cacheExpirationMaxIdleSec()).isEqualTo(100); + assertThat(config.produceUploadThreadPoolSize()).isEqualTo(16); + assertThat(config.fetchDataThreadPoolSize()).isEqualTo(12); + assertThat(config.fetchMetadataThreadPoolSize()).isEqualTo(14); + assertThat(config.fetchLaggingConsumerThreadPoolSize()).isEqualTo(24); + assertThat(config.fetchLaggingConsumerThresholdMs()).isEqualTo(240_000L); + assertThat(config.fetchLaggingConsumerRequestRateLimit()).isEqualTo(300); + assertThat(config.maxBatchesPerEnforcementRequest()).isEqualTo(10); + } } diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java index 5ba9ed1b1f..95412fab57 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java @@ -51,7 +51,6 @@ import io.aiven.inkless.control_plane.BatchInfo; import io.aiven.inkless.control_plane.BatchMetadata; import io.aiven.inkless.control_plane.FindBatchResponse; -import io.aiven.inkless.generated.FileExtent; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -171,8 +170,8 @@ public void testFetchSingleFile() { ), logStartOffset, highWatermark) ); - List files = List.of( - FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, records.sizeInBytes()), records.buffer()) + List files = List.of( + new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, records.sizeInBytes()), records.buffer())) ); FetchCompleter job = new FetchCompleter( new MockTime(), @@ -207,9 +206,9 @@ public void testFetchMultipleFiles() { ), logStartOffset, highWatermark) ); - List files = List.of( - FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, records.sizeInBytes()), records.buffer()), - FileFetchJob.createFileExtent(OBJECT_KEY_B, new ByteRange(0, records.sizeInBytes()), records.buffer()) + List files = List.of( + new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, records.sizeInBytes()), records.buffer())), + new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_B, new ByteRange(0, records.sizeInBytes()), records.buffer())) ); FetchCompleter job = new FetchCompleter( new MockTime(), @@ -249,14 +248,14 @@ public void testFetchMultipleFilesForSameBatch() { var fixedAlignment = new FixedBlockAlignment(blockSize); var ranges = fixedAlignment.align(List.of(new ByteRange(0, records.sizeInBytes()))); - var fileExtents = new ArrayList(); + var fileExtents = new ArrayList(); for (ByteRange range : ranges) { var startOffset = Math.toIntExact(range.offset()); var length = Math.min(blockSize, records.sizeInBytes() - startOffset); var endOffset = startOffset + length; ByteBuffer copy = ByteBuffer.allocate(length); copy.put(records.buffer().duplicate().position(startOffset).limit(endOffset).slice()); - fileExtents.add(FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy)); + fileExtents.add(new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); } FetchCompleter job = new FetchCompleter( @@ -305,8 +304,8 @@ public void testFetchMultipleBatches() { ), logStartOffset, highWatermark) ); - List files = List.of( - FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, totalSize), concatenatedBuffer) + List files = List.of( + new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, totalSize), concatenatedBuffer)) ); FetchCompleter job = new FetchCompleter( new MockTime(), @@ -359,14 +358,14 @@ public void testFetchMultipleFilesForMultipleBatches() { var fixedAlignment = new FixedBlockAlignment(blockSize); var ranges = fixedAlignment.align(List.of(new ByteRange(0, totalSize))); - var fileExtents = new ArrayList(); + var fileExtents = new ArrayList(); for (ByteRange range : ranges) { var startOffset = Math.toIntExact(range.offset()); var length = Math.min(blockSize, totalSize - startOffset); var endOffset = startOffset + length; ByteBuffer copy = ByteBuffer.allocate(blockSize); copy.put(concatenatedBuffer.duplicate().position(startOffset).limit(endOffset).slice()); - fileExtents.add(FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy)); + fileExtents.add(new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); } FetchCompleter job = new FetchCompleter( diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java index 35e21483c8..6d3ce37c49 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.utils.Time; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -36,10 +37,16 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import io.aiven.inkless.cache.CaffeineCache; @@ -48,6 +55,7 @@ import io.aiven.inkless.cache.NullCache; import io.aiven.inkless.cache.ObjectCache; import io.aiven.inkless.common.ByteRange; +import io.aiven.inkless.common.InklessThreadFactory; import io.aiven.inkless.common.ObjectKey; import io.aiven.inkless.common.ObjectKeyCreator; import io.aiven.inkless.common.PlainObjectKey; @@ -58,6 +66,7 @@ import io.aiven.inkless.generated.CacheKey; import io.aiven.inkless.generated.FileExtent; import io.aiven.inkless.storage_backend.common.ObjectFetcher; +import io.github.bucket4j.Bucket; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -68,6 +77,12 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +/** + * Structure: + * - Core Planning Tests: Basic fetch planning logic (no execution) + * - Execution Tests (No Lagging Consumer Feature): Tests with feature disabled + * - Lagging Consumer Feature Tests: Tests validating hot/cold path separation + */ @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.STRICT_STUBS) public class FetchPlannerTest { @@ -83,7 +98,8 @@ public class FetchPlannerTest { @Mock InklessFetchMetrics metrics; - ExecutorService dataExecutor = Executors.newSingleThreadExecutor(); + ExecutorService fetchDataExecutor = Executors.newSingleThreadExecutor(); + ExecutorService laggingFetchDataExecutor = Executors.newSingleThreadExecutor(); ObjectCache cache = new NullCache(); KeyAlignmentStrategy keyAlignmentStrategy = new FixedBlockAlignment(Integer.MAX_VALUE); @@ -95,510 +111,1280 @@ public class FetchPlannerTest { @AfterEach void tearDown() { - dataExecutor.shutdownNow(); + fetchDataExecutor.shutdownNow(); + laggingFetchDataExecutor.shutdownNow(); } - @Test - public void planEmptyRequest() { - Map coordinates = Map.of(); - FetchPlanner planner = getFetchPlanner(coordinates); - - List result = planner.planJobs(coordinates); - - assertThat(result).isEmpty(); - } + // Basic fetch planning logic (no execution) + @Nested + class PlanningTests { - @Test - public void planSingleRequest() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1) - ), - Set.of( - new ObjectFetchRequest(OBJECT_KEY_A, requestRange) - ) - ); - } - - @Test - public void planRequestsForMultipleObjects() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)), - new BatchInfo(2L, OBJECT_KEY_B.value(), BatchMetadata.of(partition0, 0, 10, 1, 1, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 2) - ), - Set.of( - new ObjectFetchRequest(OBJECT_KEY_A, requestRange), - new ObjectFetchRequest(OBJECT_KEY_B, requestRange) - ) - ); - } - - @Test - public void planRequestsForMultiplePartitions() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1), - partition1, FindBatchResponse.success(List.of( - new BatchInfo(2L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 1) - ), - Set.of( - new ObjectFetchRequest(OBJECT_KEY_A, requestRange), - new ObjectFetchRequest(OBJECT_KEY_B, requestRange) - ) - ); - } + @Test + public void planEmptyRequest() { + Map coordinates = Map.of(); + FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, cache, coordinates); - @Test - public void planMergedRequestsForSameObject() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1), - partition1, FindBatchResponse.success(List.of( - new BatchInfo(2L, OBJECT_KEY_A.value(), BatchMetadata.of(partition1, 30, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 1) - ), - Set.of( - // When batches for same object are merged, they create a single aligned fetch request - new ObjectFetchRequest(OBJECT_KEY_A, requestRange) - ) - ); - } + List result = planner.planJobs(coordinates); - @Test - public void planOffsetOutOfRange() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.offsetOutOfRange(0, 1), - partition1, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 1) - ), - Set.of( - new ObjectFetchRequest(OBJECT_KEY_B, requestRange) - ) - ); - } + assertThat(result).isEmpty(); + } - @Test - public void planUnknownTopicOrPartition() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.unknownTopicOrPartition(), - partition1, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 1) - ), - Set.of( - new ObjectFetchRequest(OBJECT_KEY_B, requestRange) - ) - ); - } + @Test + public void planRequestWithEmptyBatches() { + // Test that a response with Errors.NONE but empty batches list is handled gracefully. + // This verifies the defensive check in createFetchRequests that handles empty batches. + // In practice, empty batches won't create any object key groups, so createFetchRequests + // won't be called, but this test ensures the overall flow handles it correctly. + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of(), 0, 1) // Empty batches list + ); + FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, cache, coordinates); - @Test - public void planUnknownServerError() { - assertBatchPlan( - Map.of( - partition0, FindBatchResponse.unknownServerError(), - partition1, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 1) - ), - Set.of( - new ObjectFetchRequest(OBJECT_KEY_B, requestRange) - ) - ); - } + List result = planner.planJobs(coordinates); - private void assertBatchPlan( - final Map coordinates, - final Set expectedJobs - ) { - FetchPlanner planner = getFetchPlanner(coordinates); + // Should return empty list since there are no batches to process + assertThat(result).isEmpty(); + } - // Use the package-private planJobs method to verify the exact jobs planned - List actualJobs = planner.planJobs(coordinates); + @Test + public void planSingleRequest() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1) + ), + Set.of( + new ObjectFetchRequest(OBJECT_KEY_A, requestRange, 20) + ) + ); + } - assertThat(new HashSet<>(actualJobs)).isEqualTo(expectedJobs); - } + @Test + public void planRequestsForMultipleObjects() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_B.value(), BatchMetadata.of(partition0, 0, 10, 1, 1, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 2) + ), + Set.of( + new ObjectFetchRequest(OBJECT_KEY_A, requestRange, 20), + new ObjectFetchRequest(OBJECT_KEY_B, requestRange, 21) + ) + ); + } - @Test - public void testMultipleAsyncRequests() throws Exception { - // This test verifies that the FetchPlanner correctly handles multiple batches from different objects - // within a single fetch request. This scenario occurs when a Kafka consumer fetches from a partition - // that has records stored across multiple objects in remote storage. - // - // What we're testing: - // 1. Multiple batch coordinates are converted into separate fetch requests (one per object) - // 2. Each fetch request returns a CompletableFuture that executes asynchronously - // 3. All futures complete successfully with the correct data - // 4. Each object is fetched exactly once (not duplicated) - // - // Note: This test uses a single-threaded executor, so fetches execute sequentially. - // In production, a larger thread pool enables parallel execution for improved throughput. - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - // The different lengths help distinguish results when ordering is non-deterministic. - final byte[] dataA = "data-for-a".getBytes(); - final byte[] dataB = "data-for-bb".getBytes(); - - // Mock the fetcher's two-step process: fetch() is called first, then readToByteBuffer() - // For this test, we only care about the final data returned by readToByteBuffer() - when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))) - .thenReturn(null); // Return value doesn't matter, readToByteBuffer() is also mocked - when(fetcher.fetch(eq(OBJECT_KEY_B), any(ByteRange.class))) - .thenReturn(null); // Return value doesn't matter, readToByteBuffer() is also mocked - - // Mock readToByteBuffer to return the test data we want to verify - // Order matters: first call returns dataA, second call returns dataB - when(fetcher.readToByteBuffer(any())) - .thenReturn(ByteBuffer.wrap(dataA)) - .thenReturn(ByteBuffer.wrap(dataB)); + @Test + public void planRequestsForMultiplePartitions() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(2L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 1) + ), + Set.of( + new ObjectFetchRequest(OBJECT_KEY_A, requestRange, 20), + new ObjectFetchRequest(OBJECT_KEY_B, requestRange, 21) + ) + ); + } - final Map coordinates = Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)), - new BatchInfo(2L, OBJECT_KEY_B.value(), - BatchMetadata.of(partition0, 0, 10, 1, 1, 11, 21, TimestampType.CREATE_TIME)) - ), 0, 2) + @Test + public void planMergedRequestsForSameObject() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(2L, OBJECT_KEY_A.value(), BatchMetadata.of(partition1, 30, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 1) + ), + Set.of( + // When batches for same object are merged, timestamp is max(20, 21) = 21 + new ObjectFetchRequest(OBJECT_KEY_A, requestRange, 21) + ) ); + } - final FetchPlanner planner = getFetchPlanner(caffeineCache, coordinates); + @Test + public void planOffsetOutOfRange() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.offsetOutOfRange(0, 1), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 1) + ), + Set.of( + new ObjectFetchRequest(OBJECT_KEY_B, requestRange, 21) + ) + ); + } - // Execute and verify - assertThat(planner.get()) - .map(CompletableFuture::join) - .map(FileExtent::data) - .containsExactlyInAnyOrder(dataA, dataB); + @Test + public void planUnknownTopicOrPartition() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.unknownTopicOrPartition(), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 1) + ), + Set.of( + new ObjectFetchRequest(OBJECT_KEY_B, requestRange, 21) + ) + ); + } - verify(fetcher).fetch(eq(OBJECT_KEY_A), any(ByteRange.class)); - verify(fetcher).fetch(eq(OBJECT_KEY_B), any(ByteRange.class)); + @Test + public void planUnknownServerError() { + assertBatchPlan( + Map.of( + partition0, FindBatchResponse.unknownServerError(), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_B.value(), BatchMetadata.of(partition1, 0, 10, 0, 0, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 1) + ), + Set.of( + new ObjectFetchRequest(OBJECT_KEY_B, requestRange, 21) + ) + ); } - } - @Test - public void testCacheMiss() throws Exception { - // Setup: Cache miss scenario - data not in cache, must fetch from remote - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - final byte[] expectedData = "test-data".getBytes(); - final ByteBuffer byteBuffer = ByteBuffer.wrap(expectedData); + @Test + public void testKeyAlignmentCreatesMultipleFetchRequests() { + // Test that byte range alignment can split a single batch into multiple fetch requests + // when the batch spans multiple alignment blocks. - // Mock the fetcher to return data via ByteBuffer - when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) - .thenReturn(null); // channel not used directly - when(fetcher.readToByteBuffer(any())) - .thenReturn(byteBuffer); + final KeyAlignmentStrategy alignment = new FixedBlockAlignment(1024); final Map coordinates = Map.of( partition0, FindBatchResponse.success(List.of( new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + BatchMetadata.of(partition0, 0, 2000, 0, 0, 10, 20, TimestampType.CREATE_TIME)) ), 0, 1) ); - final FetchPlanner planner = getFetchPlanner(caffeineCache, coordinates); + final FetchPlanner planner = createFetchPlannerHotPathOnly(alignment, cache, coordinates); - // Execute: Trigger the fetch operation - final List> futures = planner.get(); + final List result = planner.planJobs(coordinates); - // Verify: Should have one future - assertThat(futures).hasSize(1); + // Should create 2 fetch requests due to alignment splitting the range + assertThat(result).hasSize(2); - // Wait for completion and verify the result - final FileExtent result = futures.get(0).get(); - assertThat(result).isNotNull(); - assertThat(result.data()).isEqualTo(expectedData); + // Both requests should be for the same object + assertThat(result.stream().map(ObjectFetchRequest::objectKey).distinct()).containsExactly(OBJECT_KEY_A); - // Verify remote fetch was called (cache miss) - verify(fetcher).fetch(any(ObjectKey.class), any(ByteRange.class)); + // Both should have the same timestamp (since they're from the same batch) + assertThat(result.stream().map(ObjectFetchRequest::timestamp).distinct()).containsExactly(20L); - // Verify the result is now in cache - final ObjectFetchRequest request = new ObjectFetchRequest(OBJECT_KEY_A, requestRange); - final CacheKey cacheKey = request.toCacheKey(); - assertThat(caffeineCache.get(cacheKey)).isNotNull(); + // The byte ranges should be aligned + final Set ranges = result.stream().map(ObjectFetchRequest::byteRange).collect(Collectors.toSet()); + // FixedBlockAlignment(1024) should create ranges aligned to 1024-byte blocks + assertThat(ranges).hasSize(2); } - } - @Test - public void testCacheHit() throws Exception { - // Setup: Cache hit scenario - data already in cache - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - final byte[] expectedData = "cached-data".getBytes(); - final FileExtent cachedFileExtent = new FileExtent().setData(expectedData); - - // Pre-populate the cache - final ObjectFetchRequest request = new ObjectFetchRequest(OBJECT_KEY_A, requestRange); - final CacheKey cacheKey = request.toCacheKey(); - caffeineCache.put(cacheKey, cachedFileExtent); + @Test + public void testTimestampAggregationUsesMaxValue() { + // Test that when multiple batches for the same object have different timestamps, + // the fetch request uses the maximum timestamp value. + // + // This is important for hot/cold path decisions - the most recent timestamp + // should be used to determine if data is "hot" (recent) or "cold" (old). final Map coordinates = Map.of( partition0, FindBatchResponse.success(List.of( new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 100, TimestampType.CREATE_TIME)) + ), 0, 1), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(2L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition1, 30, 10, 0, 0, 15, 500, TimestampType.CREATE_TIME)) ), 0, 1) ); - final FetchPlanner planner = getFetchPlanner(caffeineCache, coordinates); - - // Execute: Trigger the fetch operation - final List> futures = planner.get(); - - // Verify: Should have one future - assertThat(futures).hasSize(1); + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, cache, coordinates); + final List result = planner.planJobs(coordinates); - // Wait for completion and verify the result comes from cache - final FileExtent result = futures.get(0).get(); - assertThat(result).isNotNull(); - assertThat(result.data()).isEqualTo(expectedData); + // Should merge into single request for OBJECT_KEY_A + assertThat(result).hasSize(1); - // Verify remote fetch was NOT called (cache hit) - verify(fetcher, never()).fetch(any(ObjectKey.class), any(ByteRange.class)); + final ObjectFetchRequest request = result.get(0); + assertThat(request.objectKey()).isEqualTo(OBJECT_KEY_A); + // Timestamp should be max(100, 500) = 500 + assertThat(request.timestamp()).isEqualTo(500L); } } - @Test - public void testFetchFailure() throws Exception { - // Test that fetch failures are properly wrapped and propagated - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - - // Mock fetcher to throw exception - when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) - .thenThrow(new RuntimeException("S3 unavailable")); - - final Map coordinates = Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1) - ); - - final FetchPlanner planner = getFetchPlanner(caffeineCache, coordinates); - - // Execute: Trigger fetch operation - final List> futures = planner.get(); - - // Verify: Should have one future - assertThat(futures).hasSize(1); - - // Verify exception is wrapped in CompletableFuture - assertThatThrownBy(() -> futures.get(0).get()) - .isInstanceOf(ExecutionException.class) - .hasCauseInstanceOf(FileFetchException.class); - - // Verify remote fetch was attempted - verify(fetcher).fetch(any(ObjectKey.class), any(ByteRange.class)); + // Execution Tests - No Lagging Consumer Feature + // These tests validate core fetch execution with the lagging consumer feature disabled + @Nested + class ExecutionWithoutLaggingConsumerFeature { + + @Test + public void testMultipleAsyncFetchOperations() throws Exception { + // This test verifies that the FetchPlanner correctly handles multiple batches from different objects + // within a single fetch request. This scenario occurs when a Kafka consumer fetches from a partition + // that has records stored across multiple objects in remote storage. + // + // What we're testing: + // 1. Multiple batch coordinates are converted into separate fetch requests (one per object) + // 2. Each fetch request executes asynchronously (returns CompletableFuture immediately) + // 3. All futures complete successfully with the correct data + // 4. Each object is fetched exactly once (not duplicated) + // + // Note: Uses single-threaded executors (sequential execution), not concurrent. + // The test validates async API behavior (non-blocking submission), not parallel execution. + // + // Why this matters: + // - Ensures multiple objects can be fetched through async API + // - Validates that the async execution model works correctly + // - Confirms data integrity when handling multiple async operations + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] dataA = "data-for-a".getBytes(); + final byte[] dataB = "data-for-b".getBytes(); + + // Mock the fetcher's two-step process: fetch() is called first, then readToByteBuffer() + // For this test, we only care about the final data returned by readToByteBuffer() + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))) + .thenReturn(null); // Return value doesn't matter, readToByteBuffer() is also mocked + when(fetcher.fetch(eq(OBJECT_KEY_B), any(ByteRange.class))) + .thenReturn(null); // Return value doesn't matter, readToByteBuffer() is also mocked + + // Mock readToByteBuffer to return the test data we want to verify + // Order matters: first call returns dataA, second call returns dataB + when(fetcher.readToByteBuffer(any())) + .thenReturn(ByteBuffer.wrap(dataA)) + .thenReturn(ByteBuffer.wrap(dataB)); + + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_B.value(), + BatchMetadata.of(partition0, 0, 10, 1, 1, 11, 21, TimestampType.CREATE_TIME)) + ), 0, 2) + ); + + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + // Execute: Trigger fetch operations + final List> futures = planner.get(); + + // Verify: Should have two futures + assertThat(futures).hasSize(2); + + // Wait for all to complete + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); + + // Verify both were fetched + verify(fetcher).fetch(eq(OBJECT_KEY_A), any(ByteRange.class)); + verify(fetcher).fetch(eq(OBJECT_KEY_B), any(ByteRange.class)); + + // Verify correct data for each + final List results = futures.stream() + .map(f -> { + try { + return f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + + assertThat(results).hasSize(2); + + // Verify the actual data content matches expected values + // Note: We use containsExactlyInAnyOrder because the API doesn't guarantee + // ordering of results (even though this test uses sequential execution). + assertThat(results) + .extracting(FileExtent::data) + .containsExactlyInAnyOrder(dataA, dataB); + } } - } - @Test - public void testFailedFetchesAreRetried() throws Exception { - // Test that when a fetch fails, the error is not cached, and subsequent requests - // for the same key will retry the fetch operation. - // This ensures transient errors (network issues, temporary S3 unavailability) don't - // permanently block access to data. + @Test + public void testCacheMiss() throws Exception { + // Setup: Cache miss scenario - data not in cache, must fetch from remote + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "test-data".getBytes(); + final ByteBuffer byteBuffer = ByteBuffer.wrap(expectedData); + + // Mock the fetcher to return data via ByteBuffer + when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) + .thenReturn(null); // channel not used directly + when(fetcher.readToByteBuffer(any())) + .thenReturn(byteBuffer); + + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + // Execute: Trigger the fetch operation + final List> futures = planner.get(); + + // Verify: Should have one future + assertThat(futures).hasSize(1); + + // Wait for completion and verify the result + final FileExtent result = futures.get(0).get(); + assertThat(result).isNotNull(); + assertThat(result.data()).isEqualTo(expectedData); + + // Verify remote fetch was called (cache miss) + verify(fetcher).fetch(any(ObjectKey.class), any(ByteRange.class)); + + // Verify the result is now in cache + final ObjectFetchRequest request = new ObjectFetchRequest( + OBJECT_KEY_A, requestRange, 20 + ); + final CacheKey cacheKey = request.toCacheKey(); + assertThat(caffeineCache.get(cacheKey)).isNotNull(); + } + } - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - final byte[] expectedData = "recovered-data".getBytes(); + @Test + public void testCacheHit() throws Exception { + // Setup: Cache hit scenario - data already in cache + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "cached-data".getBytes(); + final FileExtent cachedFileExtent = new FileExtent().setData(expectedData); + + // Pre-populate the cache + final ObjectFetchRequest request = new ObjectFetchRequest( + OBJECT_KEY_A, requestRange, 20 + ); + final CacheKey cacheKey = request.toCacheKey(); + caffeineCache.put(cacheKey, cachedFileExtent); + + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + // Execute: Trigger the fetch operation + final List> futures = planner.get(); + + // Verify: Should have one future + assertThat(futures).hasSize(1); + + // Wait for completion and verify the result comes from cache + final FileExtent result = futures.get(0).get(); + assertThat(result).isNotNull(); + assertThat(result.data()).isEqualTo(expectedData); + + // Verify remote fetch was NOT called (cache hit) + verify(fetcher, never()).fetch(any(ObjectKey.class), any(ByteRange.class)); + } + } - // First call fails, second call succeeds - when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) - .thenThrow(new RuntimeException("Transient S3 error")) - .thenReturn(null); - when(fetcher.readToByteBuffer(any())) - .thenReturn(ByteBuffer.wrap(expectedData)); + @Test + public void testFetchFailure() throws Exception { + // Test that fetch failures are properly wrapped and propagated + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - final Map coordinates = Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1) - ); + // Mock fetcher to throw exception + when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) + .thenThrow(new RuntimeException("S3 unavailable")); - // First attempt - should fail - final FetchPlanner planner1 = getFetchPlanner(caffeineCache, coordinates); + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1) + ); - final List> futures1 = planner1.get(); - assertThatThrownBy(() -> futures1.get(0).get()) - .isInstanceOf(ExecutionException.class) - .hasCauseInstanceOf(FileFetchException.class); + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - // Second attempt - should retry and succeed - final FetchPlanner planner2 = getFetchPlanner(caffeineCache, coordinates); + // Execute: Trigger fetch operation + final List> futures = planner.get(); - final List> futures2 = planner2.get(); - final FileExtent result = futures2.get(0).get(); + // Verify: Should have one future + assertThat(futures).hasSize(1); - // Verify the second attempt succeeded - assertThat(result.data()).isEqualTo(expectedData); + // Verify exception is wrapped in CompletableFuture + assertThatThrownBy(() -> futures.get(0).get()) + .isInstanceOf(ExecutionException.class) + .hasCauseInstanceOf(FileFetchException.class); - // Verify fetch was called twice (once for failure, once for success) - verify(fetcher, times(2)).fetch(any(ObjectKey.class), any(ByteRange.class)); + // Verify remote fetch was attempted + verify(fetcher).fetch(any(ObjectKey.class), any(ByteRange.class)); + } } - } - @Test - public void testKeyAlignmentCreatesMultipleFetchRequests() { - // Test that byte range alignment can split a single batch into multiple fetch requests - // when the batch spans multiple alignment blocks. + @Test + public void testFailedFetchesAreRetried() throws Exception { + // Test that when a fetch fails, the error is not cached, and subsequent requests + // for the same key will retry the fetch operation. + // This ensures transient errors (network issues, temporary S3 unavailability) don't + // permanently block access to data. + + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "recovered-data".getBytes(); + + // First call fails, second call succeeds + when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) + .thenThrow(new RuntimeException("Transient S3 error")) + .thenReturn(null); + when(fetcher.readToByteBuffer(any())) + .thenReturn(ByteBuffer.wrap(expectedData)); + + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + // First attempt - should fail + final FetchPlanner planner1 = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + final List> futures1 = planner1.get(); + assertThatThrownBy(() -> futures1.get(0).get()) + .isInstanceOf(ExecutionException.class) + .hasCauseInstanceOf(FileFetchException.class); + + // Second attempt - should retry and succeed + final FetchPlanner planner2 = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + final List> futures2 = planner2.get(); + final FileExtent result = futures2.get(0).get(); + + // Verify the second attempt succeeded + assertThat(result.data()).isEqualTo(expectedData); + + // Verify fetch was called twice (once for failure, once for success) + verify(fetcher, times(2)).fetch(any(ObjectKey.class), any(ByteRange.class)); + } + } - final KeyAlignmentStrategy alignment = new FixedBlockAlignment(1024); + @Test + public void testConcurrentRequestsToSameKeyFetchOnlyOnce() throws Exception { + // Test that when multiple requests for the SAME cache key arrive concurrently, + // the underlying fetch operation is performed only ONCE, and all requesters + // receive the same result. + // + // This validates the cache's deduplication behavior which prevents redundant + // fetches to object storage when multiple threads/requests need the same data. + + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "shared-data".getBytes(); + + // Mock fetcher to return data + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))) + .thenReturn(null); + when(fetcher.readToByteBuffer(any())) + .thenReturn(ByteBuffer.wrap(expectedData)); + + // Create coordinates with TWO batches that map to the SAME cache key + // (same object, same byte range after alignment) + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1), + partition1, FindBatchResponse.success(List.of( + new BatchInfo(2L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition1, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + // Execute: Both batches will create fetch requests for the same cache key + final List> futures = planner.get(); + + // Should have only 1 future because the cache deduplicates same-key requests + assertThat(futures).hasSize(1); + + // Wait for completion + final FileExtent result = futures.get(0).get(); + + // Verify data is correct + assertThat(result.data()).isEqualTo(expectedData); + + // Verify fetch was called **only once** despite multiple requests + verify(fetcher).fetch(eq(OBJECT_KEY_A), any(ByteRange.class)); + } + } - final Map coordinates = Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 2000, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1) - ); + @Test + public void testMetricsAreRecordedCorrectly() throws Exception { + // Test that the FetchPlanner records metrics at various stages of the fetch operation. + // This ensures observability into the system's behavior. - final FetchPlanner planner = getFetchPlanner(alignment, coordinates); + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] dataA = "data-a".getBytes(); + final byte[] dataB = "data-bb".getBytes(); - final List result = planner.planJobs(coordinates); + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.fetch(eq(OBJECT_KEY_B), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())) + .thenReturn(ByteBuffer.wrap(dataA)) + .thenReturn(ByteBuffer.wrap(dataB)); - // Should create 2 fetch requests due to alignment splitting the range - assertThat(result).hasSize(2); + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 5, 20, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_B.value(), + BatchMetadata.of(partition0, 0, 10, 1, 1, 7, 21, TimestampType.CREATE_TIME)) + ), 0, 2) + ); - // Both requests should be for the same object - assertThat(result.stream().map(ObjectFetchRequest::objectKey).distinct()).containsExactly(OBJECT_KEY_A); + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - // The byte ranges should be aligned - final Set ranges = result.stream().map(ObjectFetchRequest::byteRange).collect(Collectors.toSet()); - // FixedBlockAlignment(1024) should create ranges aligned to 1024-byte blocks - assertThat(ranges).hasSize(2); - } + final List> futures = planner.get(); + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); - @Test - public void testPlanningDeduplicatesSameObjectRequests() throws Exception { - // Test that when multiple batches for the SAME object and byte range arrive, - // the planning phase merges them into a single fetch request via groupingBy(objectKey). - // - // This test demonstrates fetch deduplication at the planning level, not at the cache level. - // When batches with the same objectKey are grouped together during planning, - // only one ObjectFetchRequest is created, resulting in a single fetch operation. - // - // This validates the planner's ability to eliminate redundant fetches by merging - // requests for the same object before they're even submitted to the cache. - - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - final byte[] expectedData = "shared-data".getBytes(); - - // Mock fetcher to return data - when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))) - .thenReturn(null); - when(fetcher.readToByteBuffer(any())) - .thenReturn(ByteBuffer.wrap(expectedData)); - - // Create coordinates with TWO batches that map to the SAME cache key - // (same object, same byte range after alignment) - final Map coordinates = Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1), - partition1, FindBatchResponse.success(List.of( - new BatchInfo(2L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition1, 0, 10, 0, 0, 10, 20, TimestampType.CREATE_TIME)) - ), 0, 1) - ); + // Verify fetch batch size metric was recorded (2 batches in the response) + verify(metrics).recordFetchBatchSize(2); - final FetchPlanner planner = getFetchPlanner(caffeineCache, coordinates); + // Verify fetch objects size metric was recorded (2 unique objects) + verify(metrics).recordFetchObjectsSize(2); - // Execute: Both batches will create fetch requests for the same cache key - final List> futures = planner.get(); + // Verify cache entry size metrics were recorded for both fetches + verify(metrics).cacheEntrySize(dataA.length); + verify(metrics).cacheEntrySize(dataB.length); - // Should have only 1 future because planning merges batches with the same object key - assertThat(futures).hasSize(1); - - // Wait for completion - final FileExtent result = futures.get(0).get(); - - // Verify data is correct - assertThat(result.data()).isEqualTo(expectedData); + // Verify fetch plan finished metric was recorded + verify(metrics).fetchPlanFinished(any(Long.class)); + } + } - // Verify fetch was called **only once** despite multiple requests - verify(fetcher).fetch(eq(OBJECT_KEY_A), any(ByteRange.class)); + @Test + public void testOldDataUsesHotPathWhenLaggingConsumerFeatureDisabled() throws Exception { + // When lagging consumer feature is disabled (laggingConsumerExecutor = null), + // ALL data should use the hot path (cache + recentDataExecutor), regardless of data age. + // This test verifies that old data that would normally use cold path + // still uses hot path when the feature is disabled. + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "old-data-but-hot-path".getBytes(); + + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + // Very old timestamp - would be "lagging" if feature was enabled + final long veryOldTimestamp = time.milliseconds() - 3600_000L; // 1 hour ago + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, veryOldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + // Feature disabled: laggingConsumerExecutor = null + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Verify HOT path metrics were recorded (not cold path) + verify(metrics).recordRecentDataRequest(); + verify(metrics, never()).recordLaggingConsumerRequest(); + verify(metrics, never()).recordRateLimitWaitTime(any(Long.class)); + + // Verify data was fetched successfully + verify(fetcher).fetch(eq(OBJECT_KEY_A), any(ByteRange.class)); + } } - } - @Test - public void testMetricsAreRecordedCorrectly() throws Exception { - // Test that the FetchPlanner records metrics at various stages of the fetch operation. - // This ensures observability into the system's behavior. + @Test + public void testExecutionWithEmptyBatches() throws Exception { + // Test that execution path handles empty batches gracefully. + // This verifies the defensive check in createFetchRequests works correctly + // and that the full execution flow (planning + execution) handles empty batches. + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + // Response with Errors.NONE but empty batches list + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of(), 0, 1) // Empty batches + ); - try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { - // The different lengths help distinguish results when ordering is non-deterministic. - final byte[] dataA = "data-a".getBytes(); - final byte[] dataB = "data-bb".getBytes(); + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); - when(fetcher.fetch(eq(OBJECT_KEY_B), any(ByteRange.class))).thenReturn(null); - when(fetcher.readToByteBuffer(any())) - .thenReturn(ByteBuffer.wrap(dataA)) - .thenReturn(ByteBuffer.wrap(dataB)); + // Execute: Should return empty list of futures (no batches to fetch) + final List> futures = planner.get(); - final Map coordinates = Map.of( - partition0, FindBatchResponse.success(List.of( - new BatchInfo(1L, OBJECT_KEY_A.value(), - BatchMetadata.of(partition0, 0, 10, 0, 0, 5, 20, TimestampType.CREATE_TIME)), - new BatchInfo(2L, OBJECT_KEY_B.value(), - BatchMetadata.of(partition0, 0, 10, 1, 1, 7, 21, TimestampType.CREATE_TIME)) - ), 0, 2) - ); + // Verify: Should have no futures since there are no batches + assertThat(futures).isEmpty(); - final FetchPlanner planner = getFetchPlanner(caffeineCache, coordinates); + // Verify no fetch operations were attempted + verify(fetcher, never()).fetch(any(ObjectKey.class), any(ByteRange.class)); - final List> futures = planner.get(); - CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); + // Verify metrics were still recorded (batch size = 0) + verify(metrics).recordFetchBatchSize(0); + } + } + } - // Verify fetch batch size metric was recorded (2 batches in the response) - verify(metrics).recordFetchBatchSize(2); + // Lagging Consumer Feature Tests + // These tests validate the hot/cold path separation when the feature is enabled + @Nested + class LaggingConsumerFeatureTests { + + @Nested + class HotPathTests { + + @Test + public void recentDataUsesRecentExecutorWithoutRateLimit() throws Exception { + // Validates hot path: recent data uses recent executor, no rate limiting + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "recent-data".getBytes(); + final Bucket rateLimiter = Bucket.builder() + .addLimit(limit -> limit.capacity(1).refillGreedy(1, java.time.Duration.ofSeconds(1))) + .build(); + + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long recentTimestamp = time.milliseconds() - 30000L; // 30 seconds ago (recent) + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, recentTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final long threshold = 60 * 1000L; + final FetchPlanner planner = createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, caffeineCache, coordinates, + threshold, laggingFetchDataExecutor, rateLimiter + ); + + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Verify hot path metrics + verify(metrics).recordRecentDataRequest(); + verify(metrics, never()).recordLaggingConsumerRequest(); + verify(metrics, never()).recordRateLimitWaitTime(any(Long.class)); + } + } + + @Test + public void boundaryConditionExactlyAtThreshold() throws Exception { + // Validates boundary: dataAge == threshold uses hot path + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "boundary-data".getBytes(); + final long threshold = 60 * 1000L; + + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long exactThresholdTimestamp = time.milliseconds() - threshold; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, exactThresholdTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final FetchPlanner planner = createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, caffeineCache, coordinates, + threshold, laggingFetchDataExecutor, null + ); + + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + futures.get(0).get(); + + // dataAge <= threshold → hot path + verify(metrics).recordRecentDataRequest(); + verify(metrics, never()).recordLaggingConsumerRequest(); + } + } + } - // Verify fetch objects size metric was recorded (2 unique objects) - verify(metrics).recordFetchObjectsSize(2); + @Nested + class ColdPathTests { + + @Test + public void laggingDataUsesLaggingExecutorWithRateLimit() throws Exception { + // Validates cold path: old data uses lagging executor with rate limiting + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "old-data".getBytes(); + final Bucket rateLimiter = Bucket.builder() + .addLimit(limit -> limit.capacity(10).refillGreedy(10, java.time.Duration.ofSeconds(1))) + .build(); + + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long oldTimestamp = time.milliseconds() - 120000L; // 2 minutes ago (old) + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final long threshold = 60 * 1000L; + final FetchPlanner planner = createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, caffeineCache, coordinates, + threshold, laggingFetchDataExecutor, rateLimiter + ); + + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Verify cold path metrics with rate limiting + verify(metrics, never()).recordRecentDataRequest(); + verify(metrics).recordLaggingConsumerRequest(); + // Rate limit wait time should be recorded (including zero-wait cases for accurate histogram) + verify(metrics).recordRateLimitWaitTime(any(Long.class)); + } + } + + @Test + public void laggingDataWithoutRateLimiter() throws Exception { + // Validates cold path works without rate limiter (optional feature) + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "old-data-no-limit".getBytes(); + + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long oldTimestamp = time.milliseconds() - 120000L; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final long threshold = 60 * 1000L; + final FetchPlanner planner = createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, caffeineCache, coordinates, + threshold, laggingFetchDataExecutor, null // No rate limiter + ); + + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Verify cold path but no rate limiting metrics + verify(metrics, never()).recordRecentDataRequest(); + verify(metrics).recordLaggingConsumerRequest(); + verify(metrics, never()).recordRateLimitWaitTime(any(Long.class)); + } + } + + @Test + public void fetchFailureInColdPathPropagatesException() throws Exception { + // Test that fetch failures in the cold path are properly wrapped and propagated + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + // Mock fetcher to throw exception + when(fetcher.fetch(any(ObjectKey.class), any(ByteRange.class))) + .thenThrow(new RuntimeException("S3 unavailable")); + + final long oldTimestamp = time.milliseconds() - 120000L; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final long threshold = 60 * 1000L; + final FetchPlanner planner = createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, caffeineCache, coordinates, + threshold, laggingFetchDataExecutor, null // No rate limiter + ); + + // Execute: Trigger fetch operation via cold path + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + // Verify exception is wrapped in CompletableFuture + assertThatThrownBy(() -> futures.get(0).get()) + .isInstanceOf(ExecutionException.class) + .hasCauseInstanceOf(FileFetchException.class); + + // Verify cold path was used + verify(metrics).recordLaggingConsumerRequest(); + verify(metrics, never()).recordRecentDataRequest(); + + // Verify remote fetch was attempted + verify(fetcher).fetch(any(ObjectKey.class), any(ByteRange.class)); + } + } + + @Test + public void executorQueueFullThrowsRejectedExecutionException() throws Exception { + // Test that RejectedExecutionException is thrown when lagging consumer executor queue is full. + // This validates the AbortPolicy backpressure mechanism. + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + // Create a saturated executor: 1 thread + 1 queue slot, both occupied + final CountDownLatch blockLatch = new CountDownLatch(1); + final CountDownLatch threadStartedLatch = new CountDownLatch(1); + + final ExecutorService saturatedExecutor = new ThreadPoolExecutor( + 1, // corePoolSize + 1, // maximumPoolSize + 0L, TimeUnit.MILLISECONDS, + new ArrayBlockingQueue<>(1), // Queue capacity of 1 + new InklessThreadFactory("test-saturated-", false), + new ThreadPoolExecutor.AbortPolicy() // Reject when full + ); + + try { + // Block the single thread + saturatedExecutor.submit(() -> { + threadStartedLatch.countDown(); + try { + blockLatch.await(); // Block until test completes + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + + // Wait for thread to start + assertThat(threadStartedLatch.await(2, TimeUnit.SECONDS)) + .as("Thread should have started").isTrue(); + + // Fill the queue (1 slot) - task will be queued but not execute until thread is free + saturatedExecutor.submit(() -> { + try { + blockLatch.await(); // Block until test completes + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + + // Executor is now saturated (thread busy, queue full) + + // Setup coordinates for old data (triggers cold path) + final long oldTimestamp = time.milliseconds() - 120000L; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final long threshold = 60 * 1000L; + final FetchPlanner planner = new FetchPlanner( + time, + OBJECT_KEY_CREATOR, + keyAlignmentStrategy, + caffeineCache, + fetcher, + fetchDataExecutor, + fetcher, // laggingConsumerFetcher + threshold, + null, // No rate limiter + saturatedExecutor, // Saturated executor + coordinates, + metrics + ); + + // Execute: Cold path returns failed future instead of throwing + final List> futures = planner.get(); + + // Verify: Got one future + assertThat(futures).hasSize(1); + + // Verify: Future completed exceptionally with RejectedExecutionException + assertThat(futures.get(0)).isCompletedExceptionally(); + assertThatThrownBy(() -> futures.get(0).join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(RejectedExecutionException.class); + + // Verify: Rejection metric was recorded by FetchPlanner.submitSingleRequest() + verify(metrics).recordLaggingConsumerRejection(); + + // Verify: Cold path was attempted + verify(metrics).recordLaggingConsumerRequest(); + + } finally { + // Cleanup: Unblock and shutdown executor + blockLatch.countDown(); + saturatedExecutor.shutdown(); + saturatedExecutor.awaitTermination(2, TimeUnit.SECONDS); + } + } + } + + @Test + public void executorShutdownTrackedAsRejection() throws Exception { + // Test that RejectedExecutionException (from executor shutdown) is tracked as rejection. + // This validates that shutdown scenarios are properly monitored. + // Note: This tests shutdown behavior, which is different from queue saturation + // (tested in executorQueueFullThrowsRejectedExecutionException). + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + // Create an executor that we'll shut down + final ExecutorService shutdownExecutor = new ThreadPoolExecutor( + 1, // corePoolSize + 1, // maximumPoolSize + 0L, TimeUnit.MILLISECONDS, + new ArrayBlockingQueue<>(10), + new InklessThreadFactory("test-shutdown-", false), + new ThreadPoolExecutor.AbortPolicy() + ); + + // Shutdown immediately to cause rejection + shutdownExecutor.shutdown(); + + // Setup coordinates for old data (triggers cold path) + final long oldTimestamp = time.milliseconds() - 120000L; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + final long threshold = 60 * 1000L; + final FetchPlanner planner = new FetchPlanner( + time, + OBJECT_KEY_CREATOR, + keyAlignmentStrategy, + caffeineCache, + fetcher, + fetchDataExecutor, + fetcher, // laggingConsumerFetcher + threshold, + null, // No rate limiter + shutdownExecutor, // Shutdown executor + coordinates, + metrics + ); + + // Execute: Cold path returns failed future instead of throwing + final List> futures = planner.get(); + + // Verify: Got one future + assertThat(futures).hasSize(1); + + // Verify: Future completed exceptionally with RejectedExecutionException + assertThat(futures.get(0)).isCompletedExceptionally(); + assertThatThrownBy(() -> futures.get(0).join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(RejectedExecutionException.class); + + // Verify: Rejection metric was recorded + verify(metrics).recordLaggingConsumerRejection(); + + // Cleanup + shutdownExecutor.awaitTermination(1, TimeUnit.SECONDS); + } + } + } - // Verify cache entry size metrics were recorded for both fetches - verify(metrics).cacheEntrySize(dataA.length); - verify(metrics).cacheEntrySize(dataB.length); + @Nested + class MixedWorkloadTests { + + @Test + public void multipleRequestsMixedHotAndColdPaths() throws Exception { + // Validates path selection is per-request: some hot, some cold + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] recentData = "recent".getBytes(); + final byte[] oldData = "old".getBytes(); + final long threshold = 60 * 1000L; + + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.fetch(eq(OBJECT_KEY_B), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())) + .thenReturn(ByteBuffer.wrap(recentData)) + .thenReturn(ByteBuffer.wrap(oldData)); + + final long recentTimestamp = time.milliseconds() - 30000L; // 30s ago (recent) + final long oldTimestamp = time.milliseconds() - 120000L; // 2min ago (old) + + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, recentTimestamp, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_B.value(), + BatchMetadata.of(partition0, 0, 10, 1, 1, 11, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 2) + ); + + final FetchPlanner planner = createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, caffeineCache, coordinates, + threshold, laggingFetchDataExecutor, null + ); + + final List> futures = planner.get(); + assertThat(futures).hasSize(2); + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); + + // One request on each path + verify(metrics).recordRecentDataRequest(); + verify(metrics).recordLaggingConsumerRequest(); + } + } + + @Test + public void hotAndColdPathsExecuteConcurrently() throws Exception { + // Validates that hot and cold paths execute concurrently on separate executors. + // This ensures thread pool isolation works correctly and both paths can make progress + // simultaneously without blocking each other. + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] recentData = "recent-concurrent".getBytes(); + final byte[] oldData = "old-concurrent".getBytes(); + final long threshold = 60 * 1000L; + + // Use separate executors with multiple threads to verify concurrency + final ExecutorService hotExecutor = Executors.newFixedThreadPool(2); + final ExecutorService coldExecutor = Executors.newFixedThreadPool(2); + + try { + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.fetch(eq(OBJECT_KEY_B), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())) + .thenReturn(ByteBuffer.wrap(recentData)) + .thenReturn(ByteBuffer.wrap(oldData)); + + final long recentTimestamp = time.milliseconds() - 30000L; // 30s ago (recent) + final long oldTimestamp = time.milliseconds() - 120000L; // 2min ago (old) + + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, recentTimestamp, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_B.value(), + BatchMetadata.of(partition1, 0, 10, 0, 0, 11, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 2) + ); + + final FetchPlanner planner = new FetchPlanner( + time, + OBJECT_KEY_CREATOR, + keyAlignmentStrategy, + caffeineCache, + fetcher, + hotExecutor, // Hot path executor + fetcher, // laggingConsumerFetcher + threshold, + null, // No rate limiter + coldExecutor, // Cold path executor + coordinates, + metrics + ); + + // Execute both paths concurrently + final List> futures = planner.get(); + assertThat(futures).hasSize(2); + + // Both should complete successfully + final List results = futures.stream() + .map(f -> { + try { + return f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + + assertThat(results).hasSize(2); + assertThat(results) + .extracting(FileExtent::data) + .containsExactlyInAnyOrder(recentData, oldData); + + // Verify both paths were used + verify(metrics).recordRecentDataRequest(); + verify(metrics).recordLaggingConsumerRequest(); + } finally { + hotExecutor.shutdown(); + coldExecutor.shutdown(); + hotExecutor.awaitTermination(2, TimeUnit.SECONDS); + coldExecutor.awaitTermination(2, TimeUnit.SECONDS); + } + } + } + } - // Verify fetch plan finished metric was recorded - verify(metrics).fetchPlanFinished(any(Long.class)); + @Nested + class FeatureToggleTests { + + @Test + public void allRequestsUseRecentPathWhenFeatureDisabled() throws Exception { + // Validates: laggingConsumerExecutor = null → feature disabled, all use hot path + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "all-recent".getBytes(); + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long oldTimestamp = time.milliseconds() - 120000L; // Would be lagging if feature enabled + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + // Pass null for laggingConsumerExecutor to disable feature + final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Even with old timestamp, uses hot path + verify(metrics).recordRecentDataRequest(); + verify(metrics, never()).recordLaggingConsumerRequest(); + } + } + + @Test + public void rateLimiterCanBeDisabledIndependently() throws Exception { + // Validates: rateLimiter = null → cold path without rate limiting + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "cold-no-limit".getBytes(); + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long oldTimestamp = time.milliseconds() - 120000L; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + // Executor enabled, but no rate limiter + final FetchPlanner planner = createFetchPlannerWithLaggingConsumerFeature(keyAlignmentStrategy, caffeineCache, coordinates, laggingFetchDataExecutor, null); + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Uses cold path but no rate limiting + verify(metrics, never()).recordRecentDataRequest(); + verify(metrics).recordLaggingConsumerRequest(); + verify(metrics, never()).recordRateLimitWaitTime(any(Long.class)); + } + } + + @Test + public void bothFeaturesCanBeEnabled() throws Exception { + // Validates: both executor and rate limiter enabled → full cold path + try (CaffeineCache caffeineCache = new CaffeineCache(100, 3600, 180)) { + final byte[] expectedData = "cold-with-limit".getBytes(); + final Bucket rateLimiter = Bucket.builder() + .addLimit(limit -> limit.capacity(10).refillGreedy(10, java.time.Duration.ofSeconds(1))) + .build(); + when(fetcher.fetch(eq(OBJECT_KEY_A), any(ByteRange.class))).thenReturn(null); + when(fetcher.readToByteBuffer(any())).thenReturn(ByteBuffer.wrap(expectedData)); + + final long oldTimestamp = time.milliseconds() - 120000L; + final Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), + BatchMetadata.of(partition0, 0, 10, 0, 0, 10, oldTimestamp, TimestampType.CREATE_TIME)) + ), 0, 1) + ); + + // Both features enabled + final FetchPlanner planner = createFetchPlannerWithLaggingConsumerFeature(keyAlignmentStrategy, caffeineCache, coordinates, laggingFetchDataExecutor, rateLimiter); + final List> futures = planner.get(); + assertThat(futures).hasSize(1); + + final FileExtent result = futures.get(0).get(); + assertThat(result.data()).isEqualTo(expectedData); + + // Full cold path with rate limiting + verify(metrics, never()).recordRecentDataRequest(); + verify(metrics).recordLaggingConsumerRequest(); + verify(metrics).recordRateLimitWaitTime(any(Long.class)); + } + } } } - private FetchPlanner getFetchPlanner( - final KeyAlignmentStrategy keyAlignment, - final ObjectCache objectCache, - final Map batchCoordinatesFuture + /** + * Creates a FetchPlanner with full control over all parameters. + * Use when testing specific threshold values or custom configurations. + */ + private FetchPlanner createFetchPlannerWithCustomThreshold( + KeyAlignmentStrategy keyAlignmentStrategy, + ObjectCache cache, + Map batchCoordinatesFuture, + long thresholdMs, + ExecutorService laggingConsumerExecutor, + Bucket rateLimiter ) { return new FetchPlanner( - time, FetchPlannerTest.OBJECT_KEY_CREATOR, keyAlignment, - objectCache, fetcher, dataExecutor, batchCoordinatesFuture, metrics + time, + FetchPlannerTest.OBJECT_KEY_CREATOR, + keyAlignmentStrategy, + cache, + fetcher, + fetchDataExecutor, + fetcher, + thresholdMs, + rateLimiter, + laggingConsumerExecutor, + batchCoordinatesFuture, + metrics ); } - private FetchPlanner getFetchPlanner( - final KeyAlignmentStrategy keyAlignment, - final Map batchCoordinatesFuture + /** + * Creates a FetchPlanner with lagging consumer feature enabled (default 60s threshold). + * Use for testing hot/cold path separation with standard threshold. + */ + private FetchPlanner createFetchPlannerWithLaggingConsumerFeature( + KeyAlignmentStrategy keyAlignmentStrategy, + ObjectCache cache, + Map batchCoordinatesFuture, + ExecutorService laggingConsumerExecutor, + Bucket rateLimiter ) { - return getFetchPlanner(keyAlignment, cache, batchCoordinatesFuture); + return createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, cache, batchCoordinatesFuture, + 60 * 1000L, laggingConsumerExecutor, rateLimiter + ); } - private FetchPlanner getFetchPlanner( - final ObjectCache cache, - final Map batchCoordinatesFuture + /** + * Creates a FetchPlanner with lagging consumer feature DISABLED (hot path only). + * Use for testing basic fetch functionality without hot/cold path separation. + */ + private FetchPlanner createFetchPlannerHotPathOnly( + KeyAlignmentStrategy keyAlignmentStrategy, + ObjectCache cache, + Map batchCoordinatesFuture ) { - return getFetchPlanner(keyAlignmentStrategy, cache, batchCoordinatesFuture); + return createFetchPlannerWithCustomThreshold( + keyAlignmentStrategy, cache, batchCoordinatesFuture, + 60 * 1000L, null, null + ); } - private FetchPlanner getFetchPlanner(final Map batchCoordinatesFuture) { - return getFetchPlanner(keyAlignmentStrategy, cache, batchCoordinatesFuture); + private void assertBatchPlan(Map coordinates, Set expectedJobs) { + FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, cache, coordinates); + List actualJobs = planner.planJobs(coordinates); + assertThat(new HashSet<>(actualJobs)).isEqualTo(expectedJobs); } } diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java index a3af37b1aa..968dba2589 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java @@ -46,6 +46,7 @@ import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -66,16 +67,20 @@ import io.aiven.inkless.control_plane.BatchInfo; import io.aiven.inkless.control_plane.BatchMetadata; import io.aiven.inkless.control_plane.ControlPlane; +import io.aiven.inkless.control_plane.FindBatchRequest; import io.aiven.inkless.control_plane.FindBatchResponse; import io.aiven.inkless.generated.FileExtent; import io.aiven.inkless.storage_backend.common.ObjectFetcher; import io.aiven.inkless.storage_backend.common.StorageBackendException; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -90,10 +95,16 @@ public class ReaderTest { private static final KeyAlignmentStrategy KEY_ALIGNMENT_STRATEGY = new FixedBlockAlignment(Integer.MAX_VALUE); private static final ObjectCache OBJECT_CACHE = new NullCache(); + // Spy executors for top-level tests (testReaderEmptyRequests, testClose) + // Note: Nested test classes (RateLimitingTests, ErrorMetricTests) create their own executors + // in @BeforeEach for proper lifecycle management and avoid interfering with spy verification @Spy private final ExecutorService metadataExecutor = Executors.newSingleThreadExecutor(); @Spy - private final ExecutorService dataExecutor = Executors.newSingleThreadExecutor(); + private final ExecutorService fetchDataExecutor = Executors.newSingleThreadExecutor(); + // Spy executor for lagging fetch operations used by top-level tests; nested tests manage their own lagging executors + @Spy + private final ExecutorService laggingFetchDataExecutor = Executors.newSingleThreadExecutor(); @Mock private ControlPlane controlPlane; @@ -111,7 +122,7 @@ public void testReaderEmptyRequests() throws IOException { try (final var reader = getReader()) { final CompletableFuture> fetch = reader.fetch(fetchParams, Collections.emptyMap()); verify(metadataExecutor, atLeastOnce()).execute(any()); - verifyNoInteractions(dataExecutor); + verifyNoInteractions(fetchDataExecutor); assertThat(fetch.join()).isEqualTo(Collections.emptyMap()); } } @@ -121,7 +132,8 @@ public void testClose() throws Exception { final var reader = getReader(); reader.close(); verify(metadataExecutor, atLeastOnce()).shutdown(); - verify(dataExecutor, atLeastOnce()).shutdown(); + verify(fetchDataExecutor, atLeastOnce()).shutdown(); + verify(laggingFetchDataExecutor, atLeastOnce()).shutdown(); } @Nested @@ -154,13 +166,14 @@ public void testAllOfFileExtentsPreservesOrder() { final List> orderedFutures = List.of(futureA, futureB, futureC); // Call allOfFileExtents - final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedFutures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedFutures); // Verify result order is preserved as A, B, C (not C, B, A which was the completion order) - final List result = resultFuture.join(); + final List result = resultFuture.join(); assertThat(result) .hasSize(3) - .extracting(FileExtent::object) + .allSatisfy(r -> assertThat(r).isInstanceOf(FileExtentResult.Success.class)) + .extracting(r -> ((FileExtentResult.Success) r).extent().object()) .containsExactly(objectKeyA.value(), objectKeyB.value(), objectKeyC.value()); } @@ -175,7 +188,7 @@ public void testAllOfFileExtentsDoesNotBlock() { final List> futures = List.of(incompleteFuture); // Call allOfFileExtents - this should return immediately without blocking - final CompletableFuture> resultFuture = Reader.allOfFileExtents(futures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(futures); // Verify the result is not yet complete (proves non-blocking behavior) assertThat(resultFuture).isNotCompleted(); @@ -188,9 +201,201 @@ public void testAllOfFileExtentsDoesNotBlock() { // Verify the result completes and contains the expected value assertThat(resultFuture.join()) .hasSize(1) - .extracting(FileExtent::object) + .allSatisfy(r -> assertThat(r).isInstanceOf(FileExtentResult.Success.class)) + .extracting(r -> ((FileExtentResult.Success) r).extent().object()) .containsExactly(objectKey.value()); } + + /** + * Tests partial failure handling: when some futures fail, allOfFileExtents returns Failure results + * for failed fetches while preserving successful results. + *

+ * This simulates the critical scenario where lagging consumer requests are rejected (e.g., due to + * rate limiting or executor rejection) while hot path requests succeed. + */ + @Test + public void testPartialFailureHandling() { + // Create successful extents for hot path + final ObjectKey hotKey1 = PlainObjectKey.create("prefix", "hot-object-1"); + final ObjectKey hotKey2 = PlainObjectKey.create("prefix", "hot-object-2"); + final FileExtent hotExtent1 = FileFetchJob.createFileExtent(hotKey1, new ByteRange(0, 10), ByteBuffer.allocate(10)); + final FileExtent hotExtent2 = FileFetchJob.createFileExtent(hotKey2, new ByteRange(0, 10), ByteBuffer.allocate(10)); + + // Create futures: success, failure, success pattern + final CompletableFuture successFuture1 = CompletableFuture.completedFuture(hotExtent1); + final CompletableFuture failedFuture = CompletableFuture.failedFuture( + new RuntimeException("Lagging consumer request rejected") + ); + final CompletableFuture successFuture2 = CompletableFuture.completedFuture(hotExtent2); + + final List> mixedFutures = List.of( + successFuture1, + failedFuture, + successFuture2 + ); + + // Call allOfFileExtents + final CompletableFuture> resultFuture = Reader.allOfFileExtents(mixedFutures); + + // Verify the result completes successfully (no exception propagation) + assertThat(resultFuture).succeedsWithin(java.time.Duration.ofSeconds(1)); + + final List result = resultFuture.join(); + + // Verify we have 3 results (not failing the entire request) + assertThat(result).hasSize(3); + + // Verify first result is successful + assertThat(result.get(0)).isInstanceOf(FileExtentResult.Success.class); + final FileExtent extent1 = ((FileExtentResult.Success) result.get(0)).extent(); + assertThat(extent1.object()).isEqualTo(hotKey1.value()); + assertThat(extent1.data()).isNotEmpty(); + + // Verify second result is failure + assertThat(result.get(1)).isInstanceOf(FileExtentResult.Failure.class); + final Throwable error = ((FileExtentResult.Failure) result.get(1)).error(); + assertThat(error).hasMessageContaining("Lagging consumer request rejected"); + + // Verify third result is successful + assertThat(result.get(2)).isInstanceOf(FileExtentResult.Success.class); + final FileExtent extent3 = ((FileExtentResult.Success) result.get(2)).extent(); + assertThat(extent3.object()).isEqualTo(hotKey2.value()); + assertThat(extent3.data()).isNotEmpty(); + } + + /** + * Tests that when all futures fail, allOfFileExtents returns all Failure results. + * This ensures the partial failure mechanism works even in worst-case scenarios. + */ + @Test + public void testAllFailuresReturnAllFailureResults() { + // Create multiple failed futures + final CompletableFuture failedFuture1 = CompletableFuture.failedFuture( + new RuntimeException("Failure 1") + ); + final CompletableFuture failedFuture2 = CompletableFuture.failedFuture( + new RuntimeException("Failure 2") + ); + final CompletableFuture failedFuture3 = CompletableFuture.failedFuture( + new RuntimeException("Failure 3") + ); + + final List> allFailedFutures = List.of( + failedFuture1, + failedFuture2, + failedFuture3 + ); + + // Call allOfFileExtents + final CompletableFuture> resultFuture = Reader.allOfFileExtents(allFailedFutures); + + // Verify the result completes successfully (no exception propagation) + assertThat(resultFuture).succeedsWithin(java.time.Duration.ofSeconds(1)); + + final List result = resultFuture.join(); + + // Verify all results are Failure instances + assertThat(result).hasSize(3); + result.forEach(r -> { + assertThat(r).isInstanceOf(FileExtentResult.Failure.class); + assertThat(((FileExtentResult.Failure) r).error()).isInstanceOf(RuntimeException.class); + }); + } + + /** + * Tests mixed success and failure with different exception types. + * Verifies that the partial failure mechanism handles various exception types uniformly. + */ + @Test + public void testPartialFailureWithDifferentExceptionTypes() { + // Create one successful extent + final ObjectKey successKey = PlainObjectKey.create("prefix", "success-object"); + final FileExtent successExtent = FileFetchJob.createFileExtent( + successKey, new ByteRange(0, 10), ByteBuffer.allocate(10) + ); + + // Create futures with different exception types + final CompletableFuture successFuture = CompletableFuture.completedFuture(successExtent); + final CompletableFuture rejectedExecutionFuture = CompletableFuture.failedFuture( + new java.util.concurrent.RejectedExecutionException("Executor queue full") + ); + final CompletableFuture storageBackendFuture = CompletableFuture.failedFuture( + new StorageBackendException("S3 error") + ); + final CompletableFuture genericFuture = CompletableFuture.failedFuture( + new IllegalStateException("Unexpected error") + ); + + final List> mixedFutures = List.of( + successFuture, + rejectedExecutionFuture, + storageBackendFuture, + genericFuture + ); + + // Call allOfFileExtents + final CompletableFuture> resultFuture = Reader.allOfFileExtents(mixedFutures); + + // Verify the result completes successfully + assertThat(resultFuture).succeedsWithin(java.time.Duration.ofSeconds(1)); + + final List result = resultFuture.join(); + + // Verify we have all results + assertThat(result).hasSize(4); + + // First should be successful + assertThat(result.get(0)).isInstanceOf(FileExtentResult.Success.class); + final FileExtent extent0 = ((FileExtentResult.Success) result.get(0)).extent(); + assertThat(extent0.object()).isEqualTo(successKey.value()); + assertThat(extent0.data()).isNotEmpty(); + + // Rest should be Failure results + for (int i = 1; i < 4; i++) { + assertThat(result.get(i)).isInstanceOf(FileExtentResult.Failure.class); + } + } + + /** + * Tests that order is preserved even with partial failures. + * This is critical for FetchCompleter to correctly map extents to partitions. + */ + @Test + public void testPartialFailurePreservesOrder() { + // Create extents with identifiable data + final ObjectKey key1 = PlainObjectKey.create("prefix", "object-1"); + final ObjectKey key2 = PlainObjectKey.create("prefix", "object-2"); + final ObjectKey key3 = PlainObjectKey.create("prefix", "object-3"); + final ObjectKey key4 = PlainObjectKey.create("prefix", "object-4"); + + final FileExtent extent1 = FileFetchJob.createFileExtent(key1, new ByteRange(0, 10), ByteBuffer.allocate(10)); + final FileExtent extent3 = FileFetchJob.createFileExtent(key3, new ByteRange(0, 10), ByteBuffer.allocate(10)); + + // Create pattern: success, failure, success, failure + final CompletableFuture future1 = CompletableFuture.completedFuture(extent1); + final CompletableFuture future2 = CompletableFuture.failedFuture(new RuntimeException("Failed")); + final CompletableFuture future3 = CompletableFuture.completedFuture(extent3); + final CompletableFuture future4 = CompletableFuture.failedFuture(new RuntimeException("Failed")); + + final List> orderedFutures = List.of(future1, future2, future3, future4); + + // Call allOfFileExtents + final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedFutures); + + final List result = resultFuture.join(); + + // Verify order: success at 0, failure at 1, success at 2, failure at 3 + assertThat(result).hasSize(4); + assertThat(result.get(0)).isInstanceOf(FileExtentResult.Success.class); + assertThat(((FileExtentResult.Success) result.get(0)).extent().object()).isEqualTo(key1.value()); + + assertThat(result.get(1)).isInstanceOf(FileExtentResult.Failure.class); + + assertThat(result.get(2)).isInstanceOf(FileExtentResult.Success.class); + assertThat(((FileExtentResult.Success) result.get(2)).extent().object()).isEqualTo(key3.value()); + + assertThat(result.get(3)).isInstanceOf(FileExtentResult.Failure.class); + } } /** @@ -277,8 +482,12 @@ public void testFindBatchesException() throws IOException { } /** - * Tests that FileFetchException is properly caught and metrics are recorded. - * Exception hierarchy: FileFetchException -> CompletionException + * Tests that FileFetchException is properly handled with partial failure support. + * With partial failure handling, exceptions are caught and converted to empty FileExtents, + * resulting in KAFKA_STORAGE_ERROR responses that allow consumers to retry. + *

+ * Exception flow: StorageBackendException -> FileFetchException -> caught by allOfFileExtents + * -> empty FileExtent -> KAFKA_STORAGE_ERROR */ @Test public void testFileFetchException() throws Exception { @@ -293,21 +502,23 @@ public void testFileFetchException() throws Exception { try (final var reader = getReader()) { final CompletableFuture> fetch = reader.fetch(fetchParams, fetchInfos); - // Verify the exception is properly wrapped - assertThatThrownBy(fetch::join) - .isInstanceOf(CompletionException.class) - .hasCauseInstanceOf(FileFetchException.class) - .satisfies(e -> { - assertThat(e.getCause()).isInstanceOf(FileFetchException.class); - assertThat(e.getCause().getCause()).isInstanceOf(StorageBackendException.class); - assertThat(e.getCause().getCause().getMessage()).isEqualTo("Storage backend error"); - }); + // Verify fetch completes successfully (partial failure handling) + final Map result = fetch.join(); + + // Verify partition returns KAFKA_STORAGE_ERROR (from empty FileExtent) + assertThat(result).hasSize(1); + final FetchPartitionData data = result.get(partition); + assertThat(data.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(data.records.records()).isEmpty(); // Verify metrics are properly recorded - verify(fetchMetrics).fetchFailed(); + // Note: fileFetchFailed() is not called with partial failure handling. + // Individual failures are handled gracefully and don't trigger failure metrics. + // This is intentional - only complete fetch failures trigger fetchFailed(). + verify(fetchMetrics, never()).fetchFailed(); verify(fetchMetrics, never()).findBatchesFailed(); - verify(fetchMetrics).fileFetchFailed(); - verify(fetchMetrics, never()).fetchCompleted(any()); + verify(fetchMetrics, never()).fileFetchFailed(); + verify(fetchMetrics).fetchCompleted(any()); } } @@ -394,8 +605,433 @@ private Reader getReader() { objectFetcher, 0, metadataExecutor, - dataExecutor, + fetchDataExecutor, + objectFetcher, + Long.MAX_VALUE, + 0, + laggingFetchDataExecutor, fetchMetrics, new BrokerTopicStats()); } + + @Nested + class RateLimitingTests { + private final int RATE_LIMIT_REQ_PER_SEC = 10; // Low rate limit for testing + private final long LAGGING_THRESHOLD_MS = 1000; // 1 second threshold + + // Common test data for fetch requests + private final Uuid topicId = Uuid.randomUuid(); + private final TopicIdPartition partition = new TopicIdPartition(topicId, 0, "test-topic"); + private final Map fetchInfos = Map.of( + partition, + new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + private final MemoryRecords records = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord((byte[]) null)); + + private ExecutorService metadataExecutor; + private ExecutorService fetchDataExecutor; + private ExecutorService laggingFetchDataExecutor; + + @BeforeEach + public void setup() { + metadataExecutor = Executors.newSingleThreadExecutor(); + fetchDataExecutor = Executors.newSingleThreadExecutor(); + laggingFetchDataExecutor = Executors.newSingleThreadExecutor(); + } + + @AfterEach + public void cleanup() { + metadataExecutor.shutdown(); + fetchDataExecutor.shutdown(); + laggingFetchDataExecutor.shutdown(); + } + + /** + * Tests that rate limiting is applied when enabled and metrics are properly recorded. + * Verifies that: + * 1. Requests are throttled to the configured rate + * 2. Wait time metrics include both zero-wait and non-zero-wait cases + * 3. Lagging consumer requests are properly tracked + */ + @Test + public void testRateLimitingWithLoad() throws Exception { + // Create old data (beyond lagging threshold) to trigger cold path + final long oldTimestamp = time.milliseconds() - (LAGGING_THRESHOLD_MS * 2); + final FindBatchResponse oldResponse = FindBatchResponse.success( + List.of( + new BatchInfo( + 1L, + "object-key", + BatchMetadata.of( + partition, + 0, + records.sizeInBytes(), + 0, + 0, + oldTimestamp, + oldTimestamp, + TimestampType.CREATE_TIME + ) + ) + ), + 0L, + 1L + ); + + when(controlPlane.findBatches(any(), anyInt(), anyInt())) + .thenReturn(List.of(oldResponse)); + + final ReadableByteChannel channel = mock(ReadableByteChannel.class); + when(objectFetcher.fetch(any(), any())).thenReturn(channel); + when(objectFetcher.readToByteBuffer(channel)).thenReturn(records.buffer()); + + try (final var reader = new Reader( + time, + OBJECT_KEY_CREATOR, + KEY_ALIGNMENT_STRATEGY, + OBJECT_CACHE, + controlPlane, + objectFetcher, + 0, + metadataExecutor, + fetchDataExecutor, + objectFetcher, + LAGGING_THRESHOLD_MS, + RATE_LIMIT_REQ_PER_SEC, + laggingFetchDataExecutor, + fetchMetrics, + new BrokerTopicStats())) { + + // Submit multiple requests to trigger rate limiting + final int numRequests = RATE_LIMIT_REQ_PER_SEC * 2; // 2 seconds worth of requests + final List>> futures = new ArrayList<>(); + + for (int i = 0; i < numRequests; i++) { + futures.add(reader.fetch(fetchParams, fetchInfos)); + } + + // Wait for all requests to complete with rate limiting applied + // 20 requests @ 10 req/s = ~2 seconds minimum + await().atMost(5, SECONDS) + .pollDelay(1, SECONDS) + .untilAsserted(() -> + assertThat(CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))) + .isCompleted()); + + // Verify lagging consumer request metrics were recorded + verify(fetchMetrics, atLeastOnce()).recordLaggingConsumerRequest(); + // Verify rate limit wait time metrics were recorded (including zero-wait cases) + verify(fetchMetrics, atLeastOnce()).recordRateLimitWaitTime(anyLong()); + } + } + + /** + * Tests that rate limiting is NOT applied when disabled (rate limit = 0). + * Verifies that: + * 1. Requests complete quickly without throttling + * 2. Rate limit wait time metrics are NOT recorded + * 3. Lagging consumer requests are still tracked + */ + @Test + public void testRateLimitingDisabled() throws Exception { + // Create old data to trigger cold path + final long oldTimestamp = time.milliseconds() - (LAGGING_THRESHOLD_MS * 2); + final FindBatchResponse oldResponse = FindBatchResponse.success( + List.of( + new BatchInfo( + 1L, + "object-key", + BatchMetadata.of( + partition, + 0, + records.sizeInBytes(), + 0, + 0, + oldTimestamp, + oldTimestamp, + TimestampType.CREATE_TIME + ) + ) + ), + 0L, + 1L + ); + + when(controlPlane.findBatches(any(), anyInt(), anyInt())) + .thenReturn(List.of(oldResponse)); + + final ReadableByteChannel channel = mock(ReadableByteChannel.class); + when(objectFetcher.fetch(any(), any())).thenReturn(channel); + when(objectFetcher.readToByteBuffer(channel)).thenReturn(records.buffer()); + + try (final var reader = new Reader( + time, + OBJECT_KEY_CREATOR, + KEY_ALIGNMENT_STRATEGY, + OBJECT_CACHE, + controlPlane, + objectFetcher, + 0, + metadataExecutor, + fetchDataExecutor, + objectFetcher, + LAGGING_THRESHOLD_MS, + 0, // Rate limiting disabled + laggingFetchDataExecutor, + fetchMetrics, + new BrokerTopicStats())) { + + // Submit multiple requests + final int numRequests = RATE_LIMIT_REQ_PER_SEC * 2; + final List>> futures = new ArrayList<>(); + + for (int i = 0; i < numRequests; i++) { + futures.add(reader.fetch(fetchParams, fetchInfos)); + } + + // Verify requests complete quickly without rate limiting (within 2 seconds) + await().atMost(2, SECONDS) + .untilAsserted(() -> + assertThat(CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))) + .isCompleted()); + + // Verify lagging consumer request metrics were recorded + verify(fetchMetrics, atLeastOnce()).recordLaggingConsumerRequest(); + // Verify rate limit wait time metrics were NOT recorded (rate limiting disabled) + verify(fetchMetrics, never()).recordRateLimitWaitTime(anyLong()); + } + } + + /** + * Tests that when a single fetch request includes BOTH lagging and recent partitions, + * lagging partition failures don't prevent recent partitions from returning data. + *

+ * This is the critical scenario for handling mixed hot/cold workloads: + * - Recent partitions (hot path) use cache + fetchDataExecutor + * - Lagging partitions (cold path) use laggingFetchDataExecutor + rate limiting + * - If lagging executor is saturated and rejects requests, those partitions return empty + * - Recent partitions continue to work and return data + * - Consumers can retry only the failed lagging partitions + */ + @Test + public void testMixedLaggingAndRecentPartitions() throws Exception { + // Setup: Create two partitions - one recent, one lagging + final Uuid topicId = Uuid.randomUuid(); + final TopicIdPartition recentPartition = new TopicIdPartition(topicId, 0, "test-topic"); + final TopicIdPartition laggingPartition = new TopicIdPartition(topicId, 1, "test-topic"); + + // Recent partition data (within threshold) + final long recentTimestamp = time.milliseconds() - (LAGGING_THRESHOLD_MS / 2); + final FindBatchResponse recentResponse = FindBatchResponse.success( + List.of( + new BatchInfo( + 1L, + "recent-object-key", + BatchMetadata.of( + recentPartition, + 0, + records.sizeInBytes(), + 0, + 0, + recentTimestamp, + recentTimestamp, + TimestampType.CREATE_TIME + ) + ) + ), + 0L, + 1L + ); + + // Lagging partition data (beyond threshold) + final long laggingTimestamp = time.milliseconds() - (LAGGING_THRESHOLD_MS * 2); + final FindBatchResponse laggingResponse = FindBatchResponse.success( + List.of( + new BatchInfo( + 1L, + "lagging-object-key", + BatchMetadata.of( + laggingPartition, + 0, + records.sizeInBytes(), + 0, + 0, + laggingTimestamp, + laggingTimestamp, + TimestampType.CREATE_TIME + ) + ) + ), + 0L, + 1L + ); + + // Configure control plane to return responses in the same order as requested + // This is critical because FindBatchesJob maps responses by index + when(controlPlane.findBatches(anyList(), anyInt(), anyInt())) + .thenAnswer(invocation -> { + @SuppressWarnings("unchecked") + List requests = invocation.getArgument(0); + List responses = new ArrayList<>(); + + for (FindBatchRequest request : requests) { + if (request.topicIdPartition().equals(recentPartition)) { + responses.add(recentResponse); + } else if (request.topicIdPartition().equals(laggingPartition)) { + responses.add(laggingResponse); + } + } + + return responses; + }); + + // Setup object fetcher to succeed for all requests (hot path will use this) + final ReadableByteChannel channel = mock(ReadableByteChannel.class); + when(objectFetcher.fetch(any(ObjectKey.class), any(ByteRange.class))).thenReturn(channel); + // Return a fresh buffer each time to avoid buffer exhaustion issues + when(objectFetcher.readToByteBuffer(channel)).thenAnswer(invocation -> records.buffer().duplicate()); + + // Create a lagging executor and immediately shut it down - will reject all tasks + final ExecutorService saturatedLaggingExecutor = Executors.newSingleThreadExecutor(); + saturatedLaggingExecutor.shutdownNow(); // Force immediate rejection + + // Use dedicated executors for this test to avoid interference + final ExecutorService testMetadataExecutor = Executors.newSingleThreadExecutor(); + final ExecutorService testFetchDataExecutor = Executors.newSingleThreadExecutor(); + + try (final var reader = new Reader( + time, + OBJECT_KEY_CREATOR, + KEY_ALIGNMENT_STRATEGY, + OBJECT_CACHE, + controlPlane, + objectFetcher, + 0, + testMetadataExecutor, + testFetchDataExecutor, + objectFetcher, // Use same fetcher for lagging to simplify test + LAGGING_THRESHOLD_MS, + RATE_LIMIT_REQ_PER_SEC, + saturatedLaggingExecutor, // Saturated executor for lagging path + fetchMetrics, + new BrokerTopicStats())) { + + // Create a fetch request with BOTH partitions + final Map mixedFetchInfos = Map.of( + recentPartition, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()), + laggingPartition, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + + final CompletableFuture> fetchFuture = + reader.fetch(fetchParams, mixedFetchInfos); + + // Wait for the fetch to complete + await().atMost(3, SECONDS) + .untilAsserted(() -> assertThat(fetchFuture).isCompleted()); + + final Map result = fetchFuture.join(); + + // Verify we get results for both partitions + assertThat(result).hasSize(2); + + // Verify recent partition has data (hot path succeeded) + assertThat(result).containsKey(recentPartition); + final FetchPartitionData recentData = result.get(recentPartition); + assertThat(recentData.error).isEqualTo(Errors.NONE); + assertThat(recentData.records.records()).isNotEmpty(); + + // Verify lagging partition returns KAFKA_STORAGE_ERROR (cold path rejected) + // Empty FileExtent from rejection results in KAFKA_STORAGE_ERROR, signaling Kafka + // to return empty response so consumer can retry only the failed partition + assertThat(result).containsKey(laggingPartition); + final FetchPartitionData laggingData = result.get(laggingPartition); + assertThat(laggingData.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(laggingData.records.records()).isEmpty(); + + // Verify metrics recorded both paths + verify(fetchMetrics, atLeastOnce()).recordRecentDataRequest(); + verify(fetchMetrics, atLeastOnce()).recordLaggingConsumerRequest(); + verify(fetchMetrics, atLeastOnce()).recordLaggingConsumerRejection(); + } + } + + /** + * Tests that recent data requests bypass rate limiting. + * Verifies that: + * 1. Recent data uses hot path (cache + dataExecutor) + * 2. Rate limiting is NOT applied to recent data + * 3. Recent data request metrics are recorded + */ + @Test + public void testRecentDataBypassesRateLimiting() throws Exception { + // Create recent data (within lagging threshold) to trigger hot path + final long recentTimestamp = time.milliseconds() - (LAGGING_THRESHOLD_MS / 2); + final FindBatchResponse recentResponse = FindBatchResponse.success( + List.of( + new BatchInfo( + 1L, + "object-key", + BatchMetadata.of( + partition, + 0, + records.sizeInBytes(), + 0, + 0, + recentTimestamp, + recentTimestamp, + TimestampType.CREATE_TIME + ) + ) + ), + 0L, + 1L + ); + + when(controlPlane.findBatches(any(), anyInt(), anyInt())) + .thenReturn(List.of(recentResponse)); + + final ReadableByteChannel channel = mock(ReadableByteChannel.class); + when(objectFetcher.fetch(any(), any())).thenReturn(channel); + when(objectFetcher.readToByteBuffer(channel)).thenReturn(records.buffer()); + + try (final var reader = new Reader( + time, + OBJECT_KEY_CREATOR, + KEY_ALIGNMENT_STRATEGY, + OBJECT_CACHE, + controlPlane, + objectFetcher, + 0, + metadataExecutor, + fetchDataExecutor, + objectFetcher, + LAGGING_THRESHOLD_MS, + RATE_LIMIT_REQ_PER_SEC, + laggingFetchDataExecutor, + fetchMetrics, + new BrokerTopicStats())) { + + // Submit multiple requests + final int numRequests = RATE_LIMIT_REQ_PER_SEC * 2; + final List>> futures = new ArrayList<>(); + + for (int i = 0; i < numRequests; i++) { + futures.add(reader.fetch(fetchParams, fetchInfos)); + } + + // Verify requests complete quickly without rate limiting (within 2 seconds) + await().atMost(2, SECONDS) + .untilAsserted(() -> + assertThat(CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))) + .isCompleted()); + + // Verify recent data request metrics were recorded + verify(fetchMetrics, atLeastOnce()).recordRecentDataRequest(); + // Verify lagging consumer metrics were NOT recorded (hot path) + verify(fetchMetrics, never()).recordLaggingConsumerRequest(); + // Verify rate limit wait time metrics were NOT recorded (hot path bypasses rate limiting) + verify(fetchMetrics, never()).recordRateLimitWaitTime(anyLong()); + } + } + } } From 6afbe62d9c9767a96420a2e532984e1541f4d63a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Sun, 4 Jan 2026 15:46:39 +0200 Subject: [PATCH 2/6] fixup! feat(inkless): rate-limited hot/cold path for fetch operations Reformat to align style and reorder methods for clarity --- .../aiven/inkless/consume/FetchCompleter.java | 144 ++++++++++-------- 1 file changed, 79 insertions(+), 65 deletions(-) diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java index cab0762355..ffeaa40837 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java @@ -103,68 +103,76 @@ private Map> groupFileData() { } private Map serveFetch( - Map metadata, - Map> files + final Map metadata, + final Map> files ) { return fetchInfos.entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> servePartition(e.getKey(), metadata, files))); + .stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + e -> servePartition(e.getKey(), metadata, files)) + ); } - private FetchPartitionData servePartition(TopicIdPartition key, Map allMetadata, Map> allFiles) { + private FetchPartitionData servePartition( + final TopicIdPartition key, + final Map allMetadata, + final Map> allFiles + ) { FindBatchResponse metadata = allMetadata.get(key); if (metadata == null) { return new FetchPartitionData( - Errors.KAFKA_STORAGE_ERROR, - -1, - -1, - MemoryRecords.EMPTY, - Optional.empty(), - OptionalLong.empty(), - Optional.empty(), - OptionalInt.empty(), - false + Errors.KAFKA_STORAGE_ERROR, + -1, + -1, + MemoryRecords.EMPTY, + Optional.empty(), + OptionalLong.empty(), + Optional.empty(), + OptionalInt.empty(), + false ); } if (metadata.errors() != Errors.NONE || metadata.batches().isEmpty()) { return new FetchPartitionData( - metadata.errors(), - metadata.highWatermark(), - metadata.logStartOffset(), - MemoryRecords.EMPTY, - Optional.empty(), - OptionalLong.empty(), - Optional.empty(), - OptionalInt.empty(), - false + metadata.errors(), + metadata.highWatermark(), + metadata.logStartOffset(), + MemoryRecords.EMPTY, + Optional.empty(), + OptionalLong.empty(), + Optional.empty(), + OptionalInt.empty(), + false ); } List foundRecords = extractRecords(metadata, allFiles); if (foundRecords.isEmpty()) { // If there is no FetchedFile to serve this topic id partition, the earlier steps which prepared the metadata + data have an error. return new FetchPartitionData( - Errors.KAFKA_STORAGE_ERROR, - -1, - -1, - MemoryRecords.EMPTY, - Optional.empty(), - OptionalLong.empty(), - Optional.empty(), - OptionalInt.empty(), - false - ); - } - - return new FetchPartitionData( - Errors.NONE, - metadata.highWatermark(), - metadata.logStartOffset(), - new ConcatenatedRecords(foundRecords), + Errors.KAFKA_STORAGE_ERROR, + -1, + -1, + MemoryRecords.EMPTY, Optional.empty(), - OptionalLong.of(metadata.highWatermark()), + OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false + ); + } + + return new FetchPartitionData( + Errors.NONE, + metadata.highWatermark(), + metadata.logStartOffset(), + new ConcatenatedRecords(foundRecords), + Optional.empty(), + OptionalLong.of(metadata.highWatermark()), + Optional.empty(), + OptionalInt.empty(), + false ); } @@ -199,7 +207,36 @@ private List extractRecords(FindBatchResponse metadata, Map files + ) { + byte[] buffer = null; + for (FileExtent file : files) { + final ByteRange batchRange = batch.metadata().range(); + final ByteRange fileRange = new ByteRange(file.range().offset(), file.range().length()); + ByteRange intersection = ByteRange.intersect(batchRange, fileRange); + if (intersection.size() > 0) { + if (buffer == null) { + buffer = new byte[Math.toIntExact(batchRange.bufferSize())]; + } + final int position = intersection.bufferOffset() - batchRange.bufferOffset(); + final int from = intersection.bufferOffset() - fileRange.bufferOffset(); + final int to = intersection.bufferOffset() - fileRange.bufferOffset() + intersection.bufferSize(); + final byte[] fileData = file.data(); + System.arraycopy(fileData, from, buffer, position, Math.min(fileData.length - from, to - from)); + } + } + if (buffer == null) { + return null; + } + return createMemoryRecords(ByteBuffer.wrap(buffer), batch); + } + + private static MemoryRecords createMemoryRecords( + final ByteBuffer buffer, + final BatchInfo batch + ) { MemoryRecords records = MemoryRecords.readableRecords(buffer); Iterator iterator = records.batches().iterator(); if (!iterator.hasNext()) { @@ -222,27 +259,4 @@ private static MemoryRecords createMemoryRecords(ByteBuffer buffer, BatchInfo ba return records; } - - private static MemoryRecords constructRecordsFromFile(BatchInfo batch, List files) { - byte[] buffer = null; - for (FileExtent file : files) { - final ByteRange batchRange = batch.metadata().range(); - final ByteRange fileRange = new ByteRange(file.range().offset(), file.range().length()); - ByteRange intersection = ByteRange.intersect(batchRange, fileRange); - if (intersection.size() > 0) { - if (buffer == null) { - buffer = new byte[Math.toIntExact(batchRange.bufferSize())]; - } - final int position = intersection.bufferOffset() - batchRange.bufferOffset(); - final int from = intersection.bufferOffset() - fileRange.bufferOffset(); - final int to = intersection.bufferOffset() - fileRange.bufferOffset() + intersection.bufferSize(); - final byte[] fileData = file.data(); - System.arraycopy(fileData, from, buffer, position, Math.min(fileData.length - from, to - from)); - } - } - if (buffer == null) { - return null; - } - return createMemoryRecords(ByteBuffer.wrap(buffer), batch); - } } From 9786e62e09172b5552fcf448b4e98d4e0a3689f9 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 5 Jan 2026 10:58:00 +0200 Subject: [PATCH 3/6] fixup! feat(inkless): rate-limited hot/cold path for fetch operations --- .../inkless/consume/InklessFetchMetrics.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java index 107f7efeab..35d379622c 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/InklessFetchMetrics.java @@ -57,12 +57,12 @@ public class InklessFetchMetrics { private static final String FETCH_OBJECTS_PER_FETCH_COUNT = "FetchObjectsPerFetchCount"; private static final String RECENT_DATA_REQUEST_RATE = "RecentDataRequestRate"; private static final String LAGGING_CONSUMER_REQUEST_RATE = "LaggingConsumerRequestRate"; - private static final String LAGGING_CONSUMER_REJECTED_RATE = "LaggingConsumerRejectedRate"; + private static final String LAGGING_CONSUMER_REQUEST_REJECTED_RATE = "LaggingConsumerRequestRejectedRate"; // Tracks wait time (including zero-wait) for ALL lagging consumer requests when rate limiting is enabled. // When rate limiter is disabled (config = 0), LaggingConsumerRequestRate > 0 but this metric rate = 0. // Always records wait time to avoid histogram bias - zero-wait cases show when rate limiting is NOT a bottleneck. // Use to monitor: rate limiting latency distribution, actual throttling pressure, and limiter effectiveness. - private static final String LAGGING_RATE_LIMIT_WAIT_TIME = "LaggingRateLimitWaitTime"; + private static final String LAGGING_CONSUMER_RATE_LIMIT_WAIT_TIME = "LaggingConsumerRateLimitWaitTime"; private final Time time; @@ -114,8 +114,8 @@ public InklessFetchMetrics(final Time time, final ObjectCache cache) { cacheSize = metricsGroup.newGauge(CACHE_SIZE, () -> cache.size()); recentDataRequestRate = metricsGroup.newMeter(RECENT_DATA_REQUEST_RATE, "requests", TimeUnit.SECONDS, Map.of()); laggingConsumerRequestRate = metricsGroup.newMeter(LAGGING_CONSUMER_REQUEST_RATE, "requests", TimeUnit.SECONDS, Map.of()); - laggingConsumerRejectedRate = metricsGroup.newMeter(LAGGING_CONSUMER_REJECTED_RATE, "rejections", TimeUnit.SECONDS, Map.of()); - laggingRateLimitWaitTime = metricsGroup.newHistogram(LAGGING_RATE_LIMIT_WAIT_TIME, true, Map.of()); + laggingConsumerRejectedRate = metricsGroup.newMeter(LAGGING_CONSUMER_REQUEST_REJECTED_RATE, "rejections", TimeUnit.SECONDS, Map.of()); + laggingRateLimitWaitTime = metricsGroup.newHistogram(LAGGING_CONSUMER_RATE_LIMIT_WAIT_TIME, true, Map.of()); } public void fetchCompleted(Instant startAt) { @@ -197,8 +197,8 @@ public void close() { metricsGroup.removeMetric(FETCH_OBJECTS_PER_FETCH_COUNT); metricsGroup.removeMetric(RECENT_DATA_REQUEST_RATE); metricsGroup.removeMetric(LAGGING_CONSUMER_REQUEST_RATE); - metricsGroup.removeMetric(LAGGING_CONSUMER_REJECTED_RATE); - metricsGroup.removeMetric(LAGGING_RATE_LIMIT_WAIT_TIME); + metricsGroup.removeMetric(LAGGING_CONSUMER_REQUEST_REJECTED_RATE); + metricsGroup.removeMetric(LAGGING_CONSUMER_RATE_LIMIT_WAIT_TIME); } public void fetchStarted(int partitionSize) { @@ -261,11 +261,11 @@ public void recordLaggingConsumerRejection() { *

Recording zero-wait cases is intentional to avoid histogram bias. Zero-wait entries * show when rate limiting is NOT a bottleneck, which is valuable monitoring data. * - *

Metric: LaggingRateLimitWaitTime (Histogram) + *

Metric: LaggingConsumerRateLimitWaitTime (Histogram) * *

Relationship: - * - When rate limiting is ENABLED: LaggingRateLimitWaitTime.Rate ≈ LaggingConsumerRequestRate - * - When rate limiting is DISABLED: LaggingRateLimitWaitTime.Rate = 0, LaggingConsumerRequestRate > 0 + * - When rate limiting is ENABLED: LaggingConsumerRateLimitWaitTime.Rate ≈ LaggingConsumerRequestRate + * - When rate limiting is DISABLED: LaggingConsumerRateLimitWaitTime.Rate = 0, LaggingConsumerRequestRate > 0 * *

Use this metric to: * - Monitor rate limiting latency distribution (including p50, p99, p999) From 96e0270a6087105bfada919145a4ef286baab08e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Sun, 4 Jan 2026 15:49:43 +0200 Subject: [PATCH 4/6] fixup! feat(inkless): rate-limited hot/cold path for fetch operations Cover for potential bug where multiple file extents are found for the same object, and the first fails but the second succeeds, potentially returning incomplete data. This fixup includes the additional data to keep to enable an ordered response on the FetchCompleter, and properly serve up to the available data in order. --- .../aiven/inkless/consume/FetchCompleter.java | 134 ++++-- .../aiven/inkless/consume/FetchPlanner.java | 16 +- .../inkless/consume/FileExtentResult.java | 27 +- .../java/io/aiven/inkless/consume/Reader.java | 28 +- .../inkless/consume/FetchCompleterTest.java | 412 +++++++++++++++++- .../inkless/consume/FetchPlannerTest.java | 116 +++-- .../io/aiven/inkless/consume/ReaderTest.java | 178 ++++++-- 7 files changed, 787 insertions(+), 124 deletions(-) diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java index ffeaa40837..cae5ea5a66 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java @@ -28,6 +28,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -79,25 +80,41 @@ public Map get() { } } + // Groups file extents by their object keys, handling failures per object. + // When a failure occurs for an object key, processing stops for that object and only + // successfully fetched ranges up to that point are used. private Map> groupFileData() { - Map> files = new HashMap<>(); + // First, group all results by object key + final Map> resultsByObject = new HashMap<>(); for (FileExtentResult result : backingData) { - // Only process successful fetches - failures are handled as missing data - // which results in KAFKA_STORAGE_ERROR in extractRecords/servePartition - if (result instanceof FileExtentResult.Success success) { - final FileExtent fileExtent = success.extent(); - files.compute(fileExtent.object(), (k, v) -> { - if (v == null) { - List out = new ArrayList<>(1); - out.add(fileExtent); - return out; - } else { - v.add(fileExtent); - return v; - } - }); + final String objectKey = result.objectKey().value(); + resultsByObject.computeIfAbsent(objectKey, k -> new ArrayList<>()).add(result); + } + + final Map> files = new HashMap<>(); + for (Map.Entry> entry : resultsByObject.entrySet()) { + final String objectKey = entry.getKey(); + final List results = entry.getValue(); + + // Sort results by range offset to process in order + results.sort(Comparator.comparingLong(a -> a.byteRange().offset())); + + // Process results in order, stopping at first failure + final List successfulExtents = new ArrayList<>(); + for (FileExtentResult result : results) { + if (result instanceof FileExtentResult.Failure) { + // Stop processing this object key on first failure + // Only successfully fetched ranges up to this point will be used + break; + } else if (result instanceof FileExtentResult.Success success) { + successfulExtents.add(success.extent()); + } + } + + // Only add object key to files map if we have at least one successful fetch + if (!successfulExtents.isEmpty()) { + files.put(objectKey, successfulExtents); } - // Failure results are intentionally skipped - they don't contribute to files map } return files; } @@ -179,27 +196,34 @@ private FetchPartitionData servePartition( /** * Extracts memory records from file extents for a partition's batches. * + *

Batch Completeness: + * A batch must be complete (all required file extents present) to be useful to consumers. + * Partial batches are corrupted and cannot be parsed correctly, so we fail incomplete batches + * rather than returning corrupted data. This method returns partial results (successful batches + * so far) when a batch fails, allowing other complete batches to be returned when possible. + * *

Partial Failure Handling: - * This method returns partial results if some batches fail to extract (missing files or null records). - * This is intentional to support partial failure scenarios where some batches succeed while others fail. - * The calling code (servePartition) will check if foundRecords is empty and return KAFKA_STORAGE_ERROR - * if no records were found, allowing successful batches to be returned when possible. + * If a batch is incomplete (missing extents or failed to construct), that batch is skipped + * and we return only the successfully constructed batches up to that point. The calling code + * (servePartition) will check if foundRecords is empty and return KAFKA_STORAGE_ERROR if no + * records were found, allowing successful batches to be returned when possible. * * @param metadata the batch metadata for the partition * @param allFiles the map of object keys to fetched file extents - * @return list of memory records (may be partial if some batches failed) + * @return list of memory records (may be partial if some batches failed - only complete batches are included) */ private List extractRecords(FindBatchResponse metadata, Map> allFiles) { List foundRecords = new ArrayList<>(); for (BatchInfo batch : metadata.batches()) { List files = allFiles.get(objectKeyCreator.from(batch.objectKey()).value()); if (files == null || files.isEmpty()) { - // Missing file extent for this batch - return partial results (successful batches so far) + // Missing file extent for this batch - incomplete batch, fail it and return successful batches so far return foundRecords; } MemoryRecords fileRecords = constructRecordsFromFile(batch, files); if (fileRecords == null) { - // Failed to construct records from file - return partial results (successful batches so far) + // Failed to construct records (incomplete batch - missing extents or validation failed) + // Incomplete batches are useless to consumers, so fail this batch and return successful batches so far return foundRecords; } foundRecords.add(fileRecords); @@ -207,19 +231,69 @@ private List extractRecords(FindBatchResponse metadata, MapBatch Completeness Requirement: + * A batch must be complete (all required file extents present and contiguous) to be useful. + * Partial batches are corrupted and cannot be parsed correctly by consumers. This method + * validates completeness before allocating the buffer to avoid memory waste and returns + * null for incomplete batches, causing the batch to be failed. + * + * @param batch the batch metadata + * @param files the list of file extents (should be contiguous from groupFileData) + * @return MemoryRecords if batch is complete, null if incomplete (missing extents or gaps) + */ private static MemoryRecords constructRecordsFromFile( final BatchInfo batch, final List files ) { - byte[] buffer = null; + if (files == null || files.isEmpty()) { + return null; + } + + final ByteRange batchRange = batch.metadata().range(); + + // Validate that extents fully cover the batch range before allocating buffer. + // Incomplete batches are useless to consumers (corrupted data), so we fail early. + // This also prevents allocating large buffers for incomplete batches (saves memory). + // Collect intersections and check they're contiguous and cover the entire batch range. + List intersections = new ArrayList<>(); + for (FileExtent file : files) { + final ByteRange fileRange = new ByteRange(file.range().offset(), file.range().length()); + ByteRange intersection = ByteRange.intersect(batchRange, fileRange); + if (intersection.size() > 0) { + intersections.add(intersection); + } + } + + if (intersections.isEmpty()) { + return null; + } + + // Sort by offset to check contiguity + intersections.sort(Comparator.comparingLong(ByteRange::offset)); + + // Check that intersections are contiguous and cover the entire batch range + long expectedStart = batchRange.offset(); + for (ByteRange intersection : intersections) { + if (intersection.offset() > expectedStart) { + return null; // Gap detected - incomplete batch + } + expectedStart = Math.max(expectedStart, intersection.offset() + intersection.size()); + } + + if (expectedStart < batchRange.offset() + batchRange.size()) { + return null; // Doesn't cover entire batch range - incomplete batch + } + + // All extents cover the batch range, safe to allocate full buffer + final byte[] buffer = new byte[Math.toIntExact(batchRange.bufferSize())]; + for (FileExtent file : files) { - final ByteRange batchRange = batch.metadata().range(); final ByteRange fileRange = new ByteRange(file.range().offset(), file.range().length()); ByteRange intersection = ByteRange.intersect(batchRange, fileRange); if (intersection.size() > 0) { - if (buffer == null) { - buffer = new byte[Math.toIntExact(batchRange.bufferSize())]; - } final int position = intersection.bufferOffset() - batchRange.bufferOffset(); final int from = intersection.bufferOffset() - fileRange.bufferOffset(); final int to = intersection.bufferOffset() - fileRange.bufferOffset() + intersection.bufferSize(); @@ -227,9 +301,7 @@ private static MemoryRecords constructRecordsFromFile( System.arraycopy(fileData, from, buffer, position, Math.min(fileData.length - from, to - from)); } } - if (buffer == null) { - return null; - } + return createMemoryRecords(ByteBuffer.wrap(buffer), batch); } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java index 17104e022c..1fb56644c4 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchPlanner.java @@ -59,7 +59,7 @@ *

This class uses async cache operations to avoid blocking threads. The cache returns * CompletableFuture immediately, and actual fetches run on the provided executor. */ -public class FetchPlanner implements Supplier>> { +public class FetchPlanner implements Supplier> { private final Time time; private final ObjectKeyCreator objectKeyCreator; @@ -107,10 +107,10 @@ public FetchPlanner( /** * Executes the fetch plan: plans requests, submits to cache, returns futures. * - * @return list of futures that will complete with fetched file extents + * @return list of fetch requests paired with their futures */ @Override - public List> get() { + public List get() { return TimeUtils.measureDurationMsSupplier( time, () -> { @@ -191,9 +191,9 @@ private Stream createFetchRequests( )); } - private List> submitAllRequests(final List requests) { + private List submitAllRequests(final List requests) { return requests.stream() - .map(this::submitSingleRequest) + .map(request -> new FetchRequestWithFuture(request, submitSingleRequest(request))) .collect(Collectors.toList()); } @@ -342,4 +342,10 @@ CacheKey toCacheKey() { .setLength(byteRange.size())); } } + + /** + * Pairs an ObjectFetchRequest with its corresponding CompletableFuture. + * This allows preserving request information (object key, range) even when the fetch fails. + */ + public record FetchRequestWithFuture(ObjectFetchRequest request, CompletableFuture future) {} } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java index 9d836b5feb..5c599c4811 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FileExtentResult.java @@ -17,6 +17,8 @@ */ package io.aiven.inkless.consume; +import io.aiven.inkless.common.ByteRange; +import io.aiven.inkless.common.ObjectKey; import io.aiven.inkless.generated.FileExtent; /** @@ -26,6 +28,13 @@ * the use of sentinel values (like empty FileExtent) to signal failures. This makes the code * more type-safe and self-documenting. * + *

Both Success and Failure include objectKey and byteRange to enable: + *

    + *
  • Grouping results by object key
  • + *
  • Ordering results by range offset for proper sequential reading
  • + *
  • Stopping processing when a failure occurs for an object key
  • + *
+ * *

Usage in fetch pipeline: *

    *
  • FetchPlanner returns CompletableFuture<FileExtentResult> for each object fetch
  • @@ -34,12 +43,24 @@ *
*/ public sealed interface FileExtentResult { + /** + * Returns the object key for this result. + */ + ObjectKey objectKey(); + + /** + * Returns the byte range for this result. + */ + ByteRange byteRange(); + /** * Successful file extent fetch with data. * + * @param objectKey the object key that was fetched + * @param byteRange the byte range that was fetched * @param extent the fetched file extent containing object data */ - record Success(FileExtent extent) implements FileExtentResult {} + record Success(ObjectKey objectKey, ByteRange byteRange, FileExtent extent) implements FileExtentResult {} /** * Failed file extent fetch with error information. @@ -51,7 +72,9 @@ record Success(FileExtent extent) implements FileExtentResult {} *
  • FileFetchException: file fetch or processing failed
  • * * + * @param objectKey the object key that failed to fetch + * @param byteRange the byte range that failed to fetch * @param error the exception that caused the fetch to fail */ - record Failure(Throwable error) implements FileExtentResult {} + record Failure(ObjectKey objectKey, ByteRange byteRange, Throwable error) implements FileExtentResult {} } diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java index 8b9719ea38..80228c6811 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/Reader.java @@ -44,6 +44,7 @@ import io.aiven.inkless.TimeUtils; import io.aiven.inkless.cache.KeyAlignmentStrategy; import io.aiven.inkless.cache.ObjectCache; +import io.aiven.inkless.common.ByteRange; import io.aiven.inkless.common.InklessThreadFactory; import io.aiven.inkless.common.ObjectKeyCreator; import io.aiven.inkless.common.metrics.ThreadPoolMonitor; @@ -364,17 +365,22 @@ public CompletableFuture> fetch( * This allows successful partitions to return data while failed partitions are marked as failures, * enabling consumers to retry only the failed partitions. * - * @param fileExtentFutures the list of futures to wait for + *

    Both Success and Failure results include object key and range information to enable + * proper grouping and ordering when processing results. + * + * @param requestsWithFutures the list of fetch requests paired with their futures * @return a future that completes with a list of file extent results in the same order as the input */ static CompletableFuture> allOfFileExtents( - List> fileExtentFutures + List requestsWithFutures ) { // Handle each future individually to support partial failures. // Convert exceptions to Failure results so successful partitions still get their data. - final List> handledFutures = fileExtentFutures.stream() - .map(future -> future - .handle((extent, throwable) -> { + final List> handledFutures = requestsWithFutures.stream() + .>map(requestWithFuture -> { + final FetchPlanner.ObjectFetchRequest request = requestWithFuture.request(); + final CompletableFuture future = requestWithFuture.future(); + return future.handle((extent, throwable) -> { if (throwable != null) { // Restore interrupt status if the exception is InterruptedException. // This callback may execute on various threads (executor threads, completing thread, etc.), @@ -385,11 +391,17 @@ static CompletableFuture> allOfFileExtents( } // Log at debug level - metrics are recorded in FetchPlanner LOGGER.debug("File extent fetch failed, returning failure result", throwable); - return new FileExtentResult.Failure(throwable); + // Extract object key and range from request for Failure result + return new FileExtentResult.Failure(request.objectKey(), request.byteRange(), throwable); } else { - return (FileExtentResult) new FileExtentResult.Success(extent); + // Extract object key and range from FileExtent for Success result + // FileExtent.object() returns String, we need to create ObjectKey from it + // For Success, we can use the request's objectKey since it matches + final ByteRange byteRange = new ByteRange(extent.range().offset(), extent.range().length()); + return new FileExtentResult.Success(request.objectKey(), byteRange, extent); } - })) + }); + }) .toList(); final CompletableFuture[] futuresArray = handledFutures.toArray(CompletableFuture[]::new); diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java index 95412fab57..19ef09dab6 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java @@ -170,8 +170,9 @@ public void testFetchSingleFile() { ), logStartOffset, highWatermark) ); + final ByteRange range = new ByteRange(0, records.sizeInBytes()); List files = List.of( - new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, records.sizeInBytes()), records.buffer())) + new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, records.buffer())) ); FetchCompleter job = new FetchCompleter( new MockTime(), @@ -206,9 +207,10 @@ public void testFetchMultipleFiles() { ), logStartOffset, highWatermark) ); + final ByteRange range = new ByteRange(0, records.sizeInBytes()); List files = List.of( - new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, records.sizeInBytes()), records.buffer())), - new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_B, new ByteRange(0, records.sizeInBytes()), records.buffer())) + new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, records.buffer())), + new FileExtentResult.Success(OBJECT_KEY_B, range, FileFetchJob.createFileExtent(OBJECT_KEY_B, range, records.buffer())) ); FetchCompleter job = new FetchCompleter( new MockTime(), @@ -255,7 +257,7 @@ public void testFetchMultipleFilesForSameBatch() { var endOffset = startOffset + length; ByteBuffer copy = ByteBuffer.allocate(length); copy.put(records.buffer().duplicate().position(startOffset).limit(endOffset).slice()); - fileExtents.add(new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); + fileExtents.add(new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); } FetchCompleter job = new FetchCompleter( @@ -304,8 +306,9 @@ public void testFetchMultipleBatches() { ), logStartOffset, highWatermark) ); + final ByteRange range = new ByteRange(0, totalSize); List files = List.of( - new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, totalSize), concatenatedBuffer)) + new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, concatenatedBuffer)) ); FetchCompleter job = new FetchCompleter( new MockTime(), @@ -365,7 +368,7 @@ public void testFetchMultipleFilesForMultipleBatches() { var endOffset = startOffset + length; ByteBuffer copy = ByteBuffer.allocate(blockSize); copy.put(concatenatedBuffer.duplicate().position(startOffset).limit(endOffset).slice()); - fileExtents.add(new FileExtentResult.Success(FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); + fileExtents.add(new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); } FetchCompleter job = new FetchCompleter( @@ -387,4 +390,401 @@ public void testFetchMultipleFilesForMultipleBatches() { assertTrue(iterator.hasNext()); assertEquals(ByteBuffer.wrap(secondValue), iterator.next().value()); } + + @Test + public void testMultipleExtentsForSameObjectReturnedInOrder() { + // Test that when multiple file extents for the same object all succeed, + // both are returned in order (sorted by range offset) + MemoryRecords records1 = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord("data1".getBytes())); + MemoryRecords records2 = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord("data2".getBytes())); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 2; + + final int batch1Size = records1.sizeInBytes(); + final int batch2Size = records2.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batch1Size, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, batch1Size, batch2Size, 1, 1, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Both ranges succeed - should return both in order + final ByteRange range1 = new ByteRange(0, batch1Size); + final ByteRange range2 = new ByteRange(batch1Size, batch2Size); + // Note: order in list doesn't matter - groupFileData sorts by range offset + List files = List.of( + new FileExtentResult.Success(OBJECT_KEY_A, range2, FileFetchJob.createFileExtent(OBJECT_KEY_A, range2, records2.buffer())), + new FileExtentResult.Success(OBJECT_KEY_A, range1, FileFetchJob.createFileExtent(OBJECT_KEY_A, range1, records1.buffer())) + ); + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + files, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should succeed with both batches in order + assertEquals(Errors.NONE, data.error); + assertEquals(batch1Size + batch2Size, data.records.sizeInBytes()); + } + + @Test + public void testFirstFailureStopsProcessingNoDataReturned() { + // Test that when the first extent for an object key fails, processing stops + // and no data is returned (to avoid gaps) + MemoryRecords records1 = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord("data1".getBytes())); + MemoryRecords records2 = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord("data2".getBytes())); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 2; + + final int batch1Size = records1.sizeInBytes(); + final int batch2Size = records2.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batch1Size, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, batch1Size, batch2Size, 1, 1, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // First range fails, second succeeds - should stop at failure and not return second + final ByteRange range1 = new ByteRange(0, batch1Size); + final ByteRange range2 = new ByteRange(batch1Size, batch2Size); + List files = List.of( + new FileExtentResult.Failure(OBJECT_KEY_A, range1, new RuntimeException("Fetch failed for first range")), + new FileExtentResult.Success(OBJECT_KEY_A, range2, FileFetchJob.createFileExtent(OBJECT_KEY_A, range2, records2.buffer())) + ); + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + files, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should return KAFKA_STORAGE_ERROR because no data is available + // (first range failed, so we stop and don't process the second range to avoid gaps) + assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); + assertEquals(MemoryRecords.EMPTY, data.records); + } + + @Test + public void testMultipleBatchesPartialFailureReturnsCompleteBatches() { + // Test that when we have multiple separate batches and one fails, we return only complete batches. + // This is different from a single batch spanning multiple extents - here batch 1 and batch 2 are separate. + // If batch 1 is complete and batch 2 fails, we return batch 1 (complete batches are useful). + MemoryRecords records1 = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord("data1".getBytes())); + MemoryRecords records2 = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord("data2".getBytes())); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 2; + + final int batch1Size = records1.sizeInBytes(); + final int batch2Size = records2.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batch1Size, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)), + new BatchInfo(2L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, batch1Size, batch2Size, 1, 1, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Batch 1 extent succeeds, batch 2 extent fails - should return batch 1 only (complete batch) + final ByteRange range1 = new ByteRange(0, batch1Size); + final ByteRange range2 = new ByteRange(batch1Size, batch2Size); + List files = List.of( + new FileExtentResult.Success(OBJECT_KEY_A, range1, FileFetchJob.createFileExtent(OBJECT_KEY_A, range1, records1.buffer())), + new FileExtentResult.Failure(OBJECT_KEY_A, range2, new RuntimeException("Fetch failed for second range")) + ); + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + files, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should return batch 1 only (complete batch) since batch 2 is incomplete + assertEquals(Errors.NONE, data.error); + assertEquals(batch1Size, data.records.sizeInBytes()); + } + + @Test + public void testSingleBatchWithMultipleExtentsAllSucceed() { + // Test that a single batch spanning multiple extents succeeds when all extents are present + var blockSize = 16; + byte[] firstValue = {1}; + byte[] secondValue = {2}; + MemoryRecords records = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord(firstValue), new SimpleRecord(secondValue)); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 1; + + final int batchSize = records.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batchSize, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Split batch into multiple extents (simulating block alignment) + var fixedAlignment = new FixedBlockAlignment(blockSize); + var ranges = fixedAlignment.align(List.of(new ByteRange(0, batchSize))); + + var fileExtents = new ArrayList(); + for (ByteRange range : ranges) { + var startOffset = Math.toIntExact(range.offset()); + var length = Math.min(blockSize, batchSize - startOffset); + var endOffset = startOffset + length; + ByteBuffer copy = ByteBuffer.allocate(length); + copy.put(records.buffer().duplicate().position(startOffset).limit(endOffset).slice()); + fileExtents.add(new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); + } + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + fileExtents, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should succeed with complete batch + assertEquals(Errors.NONE, data.error); + assertEquals(batchSize, data.records.sizeInBytes()); + Iterator iterator = data.records.records().iterator(); + assertTrue(iterator.hasNext()); + assertEquals(ByteBuffer.wrap(firstValue), iterator.next().value()); + assertTrue(iterator.hasNext()); + assertEquals(ByteBuffer.wrap(secondValue), iterator.next().value()); + } + + @Test + public void testSingleBatchWithMissingMiddleExtentFails() { + // Test that a single batch spanning multiple extents fails when a middle extent is missing + // Incomplete batches are useless to consumers, so we fail the entire batch + var blockSize = 16; + byte[] firstValue = {1}; + byte[] secondValue = {2}; + MemoryRecords records = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord(firstValue), new SimpleRecord(secondValue)); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 1; + + final int batchSize = records.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batchSize, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Split batch into multiple extents, but simulate missing middle extent + var fixedAlignment = new FixedBlockAlignment(blockSize); + var ranges = fixedAlignment.align(List.of(new ByteRange(0, batchSize))); + + var fileExtents = new ArrayList(); + boolean skipMiddle = true; + for (ByteRange range : ranges) { + if (skipMiddle && range.offset() > 0 && range.offset() < batchSize / 2) { + // Simulate missing middle extent - don't add it + skipMiddle = false; + continue; + } + var startOffset = Math.toIntExact(range.offset()); + var length = Math.min(blockSize, batchSize - startOffset); + var endOffset = startOffset + length; + ByteBuffer copy = ByteBuffer.allocate(length); + copy.put(records.buffer().duplicate().position(startOffset).limit(endOffset).slice()); + fileExtents.add(new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); + } + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + fileExtents, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should fail because batch is incomplete (missing middle extent) + // Incomplete batches are useless to consumers, so we return KAFKA_STORAGE_ERROR + assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); + assertEquals(MemoryRecords.EMPTY, data.records); + } + + @Test + public void testSingleBatchWithFailedMiddleExtentFails() { + // Test that a single batch spanning multiple extents fails when a middle extent fetch fails + // groupFileData() stops at first failure, so only extents before failure are included + // The batch is incomplete and should fail + var blockSize = 16; + byte[] firstValue = {1}; + byte[] secondValue = {2}; + MemoryRecords records = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord(firstValue), new SimpleRecord(secondValue)); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 1; + + final int batchSize = records.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batchSize, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Split batch into multiple extents, with middle extent failing + var fixedAlignment = new FixedBlockAlignment(blockSize); + var ranges = fixedAlignment.align(List.of(new ByteRange(0, batchSize))); + + var fileExtents = new ArrayList(); + boolean failMiddle = true; + for (ByteRange range : ranges) { + if (failMiddle && range.offset() > 0 && range.offset() < batchSize / 2) { + // Simulate failed middle extent + fileExtents.add(new FileExtentResult.Failure(OBJECT_KEY_A, range, new RuntimeException("Fetch failed for middle extent"))); + failMiddle = false; + // groupFileData() stops at first failure, so subsequent extents won't be processed + continue; + } + var startOffset = Math.toIntExact(range.offset()); + var length = Math.min(blockSize, batchSize - startOffset); + var endOffset = startOffset + length; + ByteBuffer copy = ByteBuffer.allocate(length); + copy.put(records.buffer().duplicate().position(startOffset).limit(endOffset).slice()); + fileExtents.add(new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, copy))); + } + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + fileExtents, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should fail because batch is incomplete (middle extent failed, groupFileData stops at first failure) + // Incomplete batches are useless to consumers, so we return KAFKA_STORAGE_ERROR + assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); + assertEquals(MemoryRecords.EMPTY, data.records); + } + + @Test + public void testSingleBatchWithGapInExtentsFails() { + // Test that a single batch fails when extents have a gap (not contiguous) + // This shouldn't happen with groupFileData() which stops at first failure, + // but we validate it anyway to be safe + byte[] firstValue = {1}; + byte[] secondValue = {2}; + MemoryRecords records = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord(firstValue), new SimpleRecord(secondValue)); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 1; + + final int batchSize = records.sizeInBytes(); + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, batchSize, 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Create extents with a gap (first and last, missing middle) + final int firstExtentSize = batchSize / 3; + final int gapSize = batchSize / 3; + final int lastExtentSize = batchSize - firstExtentSize - gapSize; + + ByteBuffer firstExtent = ByteBuffer.allocate(firstExtentSize); + firstExtent.put(records.buffer().duplicate().position(0).limit(firstExtentSize).slice()); + + ByteBuffer lastExtent = ByteBuffer.allocate(lastExtentSize); + lastExtent.put(records.buffer().duplicate().position(firstExtentSize + gapSize).limit(batchSize).slice()); + + List files = List.of( + new FileExtentResult.Success(OBJECT_KEY_A, new ByteRange(0, firstExtentSize), + FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(0, firstExtentSize), firstExtent)), + // Gap: missing extent from firstExtentSize to firstExtentSize + gapSize + new FileExtentResult.Success(OBJECT_KEY_A, new ByteRange(firstExtentSize + gapSize, lastExtentSize), + FileFetchJob.createFileExtent(OBJECT_KEY_A, new ByteRange(firstExtentSize + gapSize, lastExtentSize), lastExtent)) + ); + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + files, + durationMs -> {} + ); + + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + + // Should fail because batch has a gap (not contiguous) + // Incomplete batches are useless to consumers, so we return KAFKA_STORAGE_ERROR + assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); + assertEquals(MemoryRecords.EMPTY, data.records); + } } diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java index 6d3ce37c49..fced294f9a 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchPlannerTest.java @@ -375,12 +375,15 @@ public void testMultipleAsyncFetchOperations() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); // Execute: Trigger fetch operations - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); - // Verify: Should have two futures - assertThat(futures).hasSize(2); + // Verify: Should have two requests + assertThat(requestsWithFutures).hasSize(2); - // Wait for all to complete + // Extract futures and wait for all to complete + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); // Verify both were fetched @@ -432,7 +435,10 @@ public void testCacheMiss() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); // Execute: Trigger the fetch operation - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Verify: Should have one future assertThat(futures).hasSize(1); @@ -478,7 +484,10 @@ public void testCacheHit() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); // Execute: Trigger the fetch operation - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Verify: Should have one future assertThat(futures).hasSize(1); @@ -512,7 +521,10 @@ public void testFetchFailure() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); // Execute: Trigger fetch operation - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Verify: Should have one future assertThat(futures).hasSize(1); @@ -554,7 +566,10 @@ public void testFailedFetchesAreRetried() throws Exception { // First attempt - should fail final FetchPlanner planner1 = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - final List> futures1 = planner1.get(); + final List requestsWithFutures1 = planner1.get(); + final List> futures1 = requestsWithFutures1.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThatThrownBy(() -> futures1.get(0).get()) .isInstanceOf(ExecutionException.class) .hasCauseInstanceOf(FileFetchException.class); @@ -562,7 +577,10 @@ public void testFailedFetchesAreRetried() throws Exception { // Second attempt - should retry and succeed final FetchPlanner planner2 = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - final List> futures2 = planner2.get(); + final List requestsWithFutures2 = planner2.get(); + final List> futures2 = requestsWithFutures2.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); final FileExtent result = futures2.get(0).get(); // Verify the second attempt succeeded @@ -607,7 +625,10 @@ public void testConcurrentRequestsToSameKeyFetchOnlyOnce() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); // Execute: Both batches will create fetch requests for the same cache key - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Should have only 1 future because the cache deduplicates same-key requests assertThat(futures).hasSize(1); @@ -649,7 +670,10 @@ public void testMetricsAreRecordedCorrectly() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); // Verify fetch batch size metric was recorded (2 batches in the response) @@ -691,7 +715,10 @@ public void testOldDataUsesHotPathWhenLaggingConsumerFeatureDisabled() throws Ex // Feature disabled: laggingConsumerExecutor = null final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); @@ -721,7 +748,10 @@ public void testExecutionWithEmptyBatches() throws Exception { final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); // Execute: Should return empty list of futures (no batches to fetch) - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Verify: Should have no futures since there are no batches assertThat(futures).isEmpty(); @@ -769,7 +799,10 @@ public void recentDataUsesRecentExecutorWithoutRateLimit() throws Exception { threshold, laggingFetchDataExecutor, rateLimiter ); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); @@ -805,7 +838,10 @@ public void boundaryConditionExactlyAtThreshold() throws Exception { threshold, laggingFetchDataExecutor, null ); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); futures.get(0).get(); @@ -845,7 +881,10 @@ public void laggingDataUsesLaggingExecutorWithRateLimit() throws Exception { threshold, laggingFetchDataExecutor, rateLimiter ); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); @@ -882,7 +921,10 @@ public void laggingDataWithoutRateLimiter() throws Exception { threshold, laggingFetchDataExecutor, null // No rate limiter ); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); @@ -918,7 +960,10 @@ public void fetchFailureInColdPathPropagatesException() throws Exception { ); // Execute: Trigger fetch operation via cold path - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); // Verify exception is wrapped in CompletableFuture @@ -1005,7 +1050,10 @@ public void executorQueueFullThrowsRejectedExecutionException() throws Exception ); // Execute: Cold path returns failed future instead of throwing - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Verify: Got one future assertThat(futures).hasSize(1); @@ -1077,7 +1125,10 @@ public void executorShutdownTrackedAsRejection() throws Exception { ); // Execute: Cold path returns failed future instead of throwing - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); // Verify: Got one future assertThat(futures).hasSize(1); @@ -1131,7 +1182,10 @@ public void multipleRequestsMixedHotAndColdPaths() throws Exception { threshold, laggingFetchDataExecutor, null ); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(2); CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); @@ -1190,7 +1244,10 @@ public void hotAndColdPathsExecuteConcurrently() throws Exception { ); // Execute both paths concurrently - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(2); // Both should complete successfully @@ -1243,7 +1300,10 @@ public void allRequestsUseRecentPathWhenFeatureDisabled() throws Exception { // Pass null for laggingConsumerExecutor to disable feature final FetchPlanner planner = createFetchPlannerHotPathOnly(keyAlignmentStrategy, caffeineCache, coordinates); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); @@ -1273,7 +1333,10 @@ public void rateLimiterCanBeDisabledIndependently() throws Exception { // Executor enabled, but no rate limiter final FetchPlanner planner = createFetchPlannerWithLaggingConsumerFeature(keyAlignmentStrategy, caffeineCache, coordinates, laggingFetchDataExecutor, null); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); @@ -1307,7 +1370,10 @@ public void bothFeaturesCanBeEnabled() throws Exception { // Both features enabled final FetchPlanner planner = createFetchPlannerWithLaggingConsumerFeature(keyAlignmentStrategy, caffeineCache, coordinates, laggingFetchDataExecutor, rateLimiter); - final List> futures = planner.get(); + final List requestsWithFutures = planner.get(); + final List> futures = requestsWithFutures.stream() + .map(FetchPlanner.FetchRequestWithFuture::future) + .toList(); assertThat(futures).hasSize(1); final FileExtent result = futures.get(0).get(); diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java index 968dba2589..cfbcc6d80e 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/ReaderTest.java @@ -147,10 +147,11 @@ public void testAllOfFileExtentsPreservesOrder() { final ObjectKey objectKeyA = PlainObjectKey.create("prefix", "object-a"); final ObjectKey objectKeyB = PlainObjectKey.create("prefix", "object-b"); final ObjectKey objectKeyC = PlainObjectKey.create("prefix", "object-c"); + final ByteRange range = new ByteRange(0, 10); - final FileExtent extentA = FileFetchJob.createFileExtent(objectKeyA, new ByteRange(0, 10), ByteBuffer.allocate(10)); - final FileExtent extentB = FileFetchJob.createFileExtent(objectKeyB, new ByteRange(0, 10), ByteBuffer.allocate(10)); - final FileExtent extentC = FileFetchJob.createFileExtent(objectKeyC, new ByteRange(0, 10), ByteBuffer.allocate(10)); + final FileExtent extentA = FileFetchJob.createFileExtent(objectKeyA, range, ByteBuffer.allocate(10)); + final FileExtent extentB = FileFetchJob.createFileExtent(objectKeyB, range, ByteBuffer.allocate(10)); + final FileExtent extentC = FileFetchJob.createFileExtent(objectKeyC, range, ByteBuffer.allocate(10)); // Create uncompleted futures final CompletableFuture futureA = new CompletableFuture<>(); @@ -162,11 +163,24 @@ public void testAllOfFileExtentsPreservesOrder() { futureB.complete(extentB); futureA.complete(extentA); - // Create the ordered list: A, B, C - final List> orderedFutures = List.of(futureA, futureB, futureC); + // Create requests with futures: A, B, C + final List orderedRequests = List.of( + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(objectKeyA, range, 0L), + futureA + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(objectKeyB, range, 0L), + futureB + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(objectKeyC, range, 0L), + futureC + ) + ); // Call allOfFileExtents - final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedFutures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedRequests); // Verify result order is preserved as A, B, C (not C, B, A which was the completion order) final List result = resultFuture.join(); @@ -183,12 +197,19 @@ public void testAllOfFileExtentsPreservesOrder() { @Test public void testAllOfFileExtentsDoesNotBlock() { // Create an incomplete future + final ObjectKey testObjectKey = PlainObjectKey.create("prefix", "object"); + final ByteRange range = new ByteRange(0, 10); final CompletableFuture incompleteFuture = new CompletableFuture<>(); - final List> futures = List.of(incompleteFuture); + final List requests = List.of( + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(testObjectKey, range, 0L), + incompleteFuture + ) + ); // Call allOfFileExtents - this should return immediately without blocking - final CompletableFuture> resultFuture = Reader.allOfFileExtents(futures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(requests); // Verify the result is not yet complete (proves non-blocking behavior) assertThat(resultFuture).isNotCompleted(); @@ -218,8 +239,9 @@ public void testPartialFailureHandling() { // Create successful extents for hot path final ObjectKey hotKey1 = PlainObjectKey.create("prefix", "hot-object-1"); final ObjectKey hotKey2 = PlainObjectKey.create("prefix", "hot-object-2"); - final FileExtent hotExtent1 = FileFetchJob.createFileExtent(hotKey1, new ByteRange(0, 10), ByteBuffer.allocate(10)); - final FileExtent hotExtent2 = FileFetchJob.createFileExtent(hotKey2, new ByteRange(0, 10), ByteBuffer.allocate(10)); + final ByteRange range = new ByteRange(0, 10); + final FileExtent hotExtent1 = FileFetchJob.createFileExtent(hotKey1, range, ByteBuffer.allocate(10)); + final FileExtent hotExtent2 = FileFetchJob.createFileExtent(hotKey2, range, ByteBuffer.allocate(10)); // Create futures: success, failure, success pattern final CompletableFuture successFuture1 = CompletableFuture.completedFuture(hotExtent1); @@ -228,14 +250,23 @@ public void testPartialFailureHandling() { ); final CompletableFuture successFuture2 = CompletableFuture.completedFuture(hotExtent2); - final List> mixedFutures = List.of( - successFuture1, - failedFuture, - successFuture2 + final List mixedRequests = List.of( + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(hotKey1, range, 0L), + successFuture1 + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(hotKey2, range, 0L), + failedFuture + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(hotKey2, range, 0L), + successFuture2 + ) ); // Call allOfFileExtents - final CompletableFuture> resultFuture = Reader.allOfFileExtents(mixedFutures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(mixedRequests); // Verify the result completes successfully (no exception propagation) assertThat(resultFuture).succeedsWithin(java.time.Duration.ofSeconds(1)); @@ -270,6 +301,11 @@ public void testPartialFailureHandling() { @Test public void testAllFailuresReturnAllFailureResults() { // Create multiple failed futures + final ObjectKey key1 = PlainObjectKey.create("prefix", "object-1"); + final ObjectKey key2 = PlainObjectKey.create("prefix", "object-2"); + final ObjectKey key3 = PlainObjectKey.create("prefix", "object-3"); + final ByteRange range = new ByteRange(0, 10); + final CompletableFuture failedFuture1 = CompletableFuture.failedFuture( new RuntimeException("Failure 1") ); @@ -280,14 +316,23 @@ public void testAllFailuresReturnAllFailureResults() { new RuntimeException("Failure 3") ); - final List> allFailedFutures = List.of( - failedFuture1, - failedFuture2, - failedFuture3 + final List allFailedRequests = List.of( + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key1, range, 0L), + failedFuture1 + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key2, range, 0L), + failedFuture2 + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key3, range, 0L), + failedFuture3 + ) ); // Call allOfFileExtents - final CompletableFuture> resultFuture = Reader.allOfFileExtents(allFailedFutures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(allFailedRequests); // Verify the result completes successfully (no exception propagation) assertThat(resultFuture).succeedsWithin(java.time.Duration.ofSeconds(1)); @@ -310,8 +355,12 @@ public void testAllFailuresReturnAllFailureResults() { public void testPartialFailureWithDifferentExceptionTypes() { // Create one successful extent final ObjectKey successKey = PlainObjectKey.create("prefix", "success-object"); + final ObjectKey failKey1 = PlainObjectKey.create("prefix", "fail-object-1"); + final ObjectKey failKey2 = PlainObjectKey.create("prefix", "fail-object-2"); + final ObjectKey failKey3 = PlainObjectKey.create("prefix", "fail-object-3"); + final ByteRange range = new ByteRange(0, 10); final FileExtent successExtent = FileFetchJob.createFileExtent( - successKey, new ByteRange(0, 10), ByteBuffer.allocate(10) + successKey, range, ByteBuffer.allocate(10) ); // Create futures with different exception types @@ -326,15 +375,27 @@ successKey, new ByteRange(0, 10), ByteBuffer.allocate(10) new IllegalStateException("Unexpected error") ); - final List> mixedFutures = List.of( - successFuture, - rejectedExecutionFuture, - storageBackendFuture, - genericFuture + final List mixedRequests = List.of( + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(successKey, range, 0L), + successFuture + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(failKey1, range, 0L), + rejectedExecutionFuture + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(failKey2, range, 0L), + storageBackendFuture + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(failKey3, range, 0L), + genericFuture + ) ); // Call allOfFileExtents - final CompletableFuture> resultFuture = Reader.allOfFileExtents(mixedFutures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(mixedRequests); // Verify the result completes successfully assertThat(resultFuture).succeedsWithin(java.time.Duration.ofSeconds(1)); @@ -368,8 +429,9 @@ public void testPartialFailurePreservesOrder() { final ObjectKey key3 = PlainObjectKey.create("prefix", "object-3"); final ObjectKey key4 = PlainObjectKey.create("prefix", "object-4"); - final FileExtent extent1 = FileFetchJob.createFileExtent(key1, new ByteRange(0, 10), ByteBuffer.allocate(10)); - final FileExtent extent3 = FileFetchJob.createFileExtent(key3, new ByteRange(0, 10), ByteBuffer.allocate(10)); + final ByteRange range = new ByteRange(0, 10); + final FileExtent extent1 = FileFetchJob.createFileExtent(key1, range, ByteBuffer.allocate(10)); + final FileExtent extent3 = FileFetchJob.createFileExtent(key3, range, ByteBuffer.allocate(10)); // Create pattern: success, failure, success, failure final CompletableFuture future1 = CompletableFuture.completedFuture(extent1); @@ -377,10 +439,27 @@ public void testPartialFailurePreservesOrder() { final CompletableFuture future3 = CompletableFuture.completedFuture(extent3); final CompletableFuture future4 = CompletableFuture.failedFuture(new RuntimeException("Failed")); - final List> orderedFutures = List.of(future1, future2, future3, future4); + final List orderedRequests = List.of( + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key1, range, 0L), + future1 + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key2, range, 0L), + future2 + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key3, range, 0L), + future3 + ), + new FetchPlanner.FetchRequestWithFuture( + new FetchPlanner.ObjectFetchRequest(key4, range, 0L), + future4 + ) + ); // Call allOfFileExtents - final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedFutures); + final CompletableFuture> resultFuture = Reader.allOfFileExtents(orderedRequests); final List result = resultFuture.join(); @@ -523,8 +602,12 @@ public void testFileFetchException() throws Exception { } /** - * Tests that FetchException is properly caught and metrics are recorded. - * Exception hierarchy: FetchException -> CompletionException + * Tests that corrupted/invalid data is handled gracefully and returns KAFKA_STORAGE_ERROR. + * + *

    With the new validation logic, when corrupted data is provided that doesn't match + * the expected batch size, the validation detects that extents don't cover the batch range + * and returns null early. This results in empty records being returned with KAFKA_STORAGE_ERROR + * instead of throwing a FetchException. */ @Test public void testFetchException() throws Exception { @@ -532,30 +615,31 @@ public void testFetchException() throws Exception { when(controlPlane.findBatches(any(), anyInt(), anyInt())) .thenReturn(List.of(singleResponse)); - // Simulate fetcher returning invalid data that causes FetchException + // Simulate fetcher returning invalid/corrupted data that doesn't match expected size final ReadableByteChannel file1Channel = mock(ReadableByteChannel.class); when(objectFetcher.fetch(any(), any())).thenReturn(file1Channel); - // Will be read as MemoryRecords but invalid data will cause exception - final ByteBuffer corruptedRecords = ByteBuffer.wrap("corrupt-data".getBytes(StandardCharsets.UTF_8)); + // Corrupted data with size that doesn't match expected batch size + final ByteBuffer corruptedRecords = ByteBuffer.wrap("invalid-batch-data".getBytes(StandardCharsets.UTF_8)); when(objectFetcher.readToByteBuffer(file1Channel)).thenReturn(corruptedRecords); try (final var reader = getReader()) { final CompletableFuture> fetch = reader.fetch(fetchParams, fetchInfos); - // Verify the exception is properly wrapped - assertThatThrownBy(fetch::join) - .isInstanceOf(CompletionException.class) - .hasCauseInstanceOf(FetchException.class) - .satisfies(e -> { - assertThat(e.getCause()).isInstanceOf(FetchException.class); - assertThat(e.getCause().getCause()).isInstanceOf(IllegalStateException.class); - assertThat(e.getCause().getCause().getMessage()).isEqualTo("Backing file should have at least one batch"); - }); + + // Verify fetch completes successfully but returns KAFKA_STORAGE_ERROR + // (validation detects incomplete batch and returns empty records) + final Map result = fetch.join(); + assertThat(result).hasSize(1); + final FetchPartitionData data = result.get(partition); + assertThat(data.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(data.records).isEqualTo(MemoryRecords.EMPTY); // Verify metrics are properly recorded - verify(fetchMetrics).fetchFailed(); + // Note: fetchCompleted is called because the fetch itself succeeded, + // even though the data was invalid (returned KAFKA_STORAGE_ERROR) + verify(fetchMetrics, never()).fetchFailed(); verify(fetchMetrics, never()).findBatchesFailed(); verify(fetchMetrics, never()).fileFetchFailed(); - verify(fetchMetrics, never()).fetchCompleted(any()); + verify(fetchMetrics).fetchCompleted(any()); } } From 67f36b34db7e26329cd5db9c130ed540443a7207 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 5 Jan 2026 13:56:02 +0200 Subject: [PATCH 5/6] fixup! feat(inkless): rate-limited hot/cold path for fetch operations --- .../inkless/consume/FetchCompleterTest.java | 136 +++++++++++------- 1 file changed, 82 insertions(+), 54 deletions(-) diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java index 19ef09dab6..e219bc1896 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java @@ -52,8 +52,8 @@ import io.aiven.inkless.control_plane.BatchMetadata; import io.aiven.inkless.control_plane.FindBatchResponse; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.STRICT_STUBS) @@ -79,7 +79,7 @@ public void testEmptyFetch() { durationMs -> {} ); Map result = job.get(); - assertTrue(result.isEmpty()); + assertThat(result).isEmpty(); } @Test @@ -97,7 +97,7 @@ public void testFetchWithoutCoordinates() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); + assertThat(data.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); } @Test @@ -120,10 +120,10 @@ public void testFetchWithoutBatches() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(Errors.NONE, data.error); - assertEquals(MemoryRecords.EMPTY, data.records); - assertEquals(logStartOffset, data.logStartOffset); - assertEquals(highWatermark, data.highWatermark); + assertThat(data.error).isEqualTo(Errors.NONE); + assertThat(data.records).isEqualTo(MemoryRecords.EMPTY); + assertThat(data.logStartOffset).isEqualTo(logStartOffset); + assertThat(data.highWatermark).isEqualTo(highWatermark); } @Test @@ -150,7 +150,7 @@ public void testFetchWithoutFiles() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); + assertThat(data.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); } @Test @@ -184,9 +184,9 @@ public void testFetchSingleFile() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(records.sizeInBytes(), data.records.sizeInBytes()); - assertEquals(logStartOffset, data.logStartOffset); - assertEquals(highWatermark, data.highWatermark); + assertThat(data.records.sizeInBytes()).isEqualTo(records.sizeInBytes()); + assertThat(data.logStartOffset).isEqualTo(logStartOffset); + assertThat(data.highWatermark).isEqualTo(highWatermark); } @Test @@ -222,9 +222,9 @@ public void testFetchMultipleFiles() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(2 * records.sizeInBytes(), data.records.sizeInBytes()); - assertEquals(logStartOffset, data.logStartOffset); - assertEquals(highWatermark, data.highWatermark); + assertThat(data.records.sizeInBytes()).isEqualTo(2 * records.sizeInBytes()); + assertThat(data.logStartOffset).isEqualTo(logStartOffset); + assertThat(data.highWatermark).isEqualTo(highWatermark); } @Test @@ -270,14 +270,18 @@ public void testFetchMultipleFilesForSameBatch() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(records.sizeInBytes(), data.records.sizeInBytes()); - assertEquals(logStartOffset, data.logStartOffset); - assertEquals(highWatermark, data.highWatermark); + assertThat(data) + .satisfies(d -> { + assertThat(d.records.sizeInBytes()).isEqualTo(records.sizeInBytes()); + assertThat(d.logStartOffset).isEqualTo(logStartOffset); + assertThat(d.highWatermark).isEqualTo(highWatermark); + }); + Iterator iterator = data.records.records().iterator(); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(firstValue), iterator.next().value()); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(secondValue), iterator.next().value()); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(firstValue)); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(secondValue)); } @Test @@ -320,14 +324,18 @@ public void testFetchMultipleBatches() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(totalSize, data.records.sizeInBytes()); - assertEquals(logStartOffset, data.logStartOffset); - assertEquals(highWatermark, data.highWatermark); + assertThat(data) + .satisfies(d -> { + assertThat(d.records.sizeInBytes()).isEqualTo(totalSize); + assertThat(d.logStartOffset).isEqualTo(logStartOffset); + assertThat(d.highWatermark).isEqualTo(highWatermark); + }); + Iterator iterator = data.records.records().iterator(); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(firstValue), iterator.next().value()); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(secondValue), iterator.next().value()); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(firstValue)); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(secondValue)); } @Test @@ -381,14 +389,18 @@ public void testFetchMultipleFilesForMultipleBatches() { ); Map result = job.get(); FetchPartitionData data = result.get(partition0); - assertEquals(totalSize, data.records.sizeInBytes()); - assertEquals(logStartOffset, data.logStartOffset); - assertEquals(highWatermark, data.highWatermark); + assertThat(data) + .satisfies(d -> { + assertThat(d.records.sizeInBytes()).isEqualTo(totalSize); + assertThat(d.logStartOffset).isEqualTo(logStartOffset); + assertThat(d.highWatermark).isEqualTo(highWatermark); + }); + Iterator iterator = data.records.records().iterator(); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(firstValue), iterator.next().value()); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(secondValue), iterator.next().value()); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(firstValue)); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(secondValue)); } @Test @@ -437,8 +449,8 @@ public void testMultipleExtentsForSameObjectReturnedInOrder() { FetchPartitionData data = result.get(partition0); // Should succeed with both batches in order - assertEquals(Errors.NONE, data.error); - assertEquals(batch1Size + batch2Size, data.records.sizeInBytes()); + assertThat(data.error).isEqualTo(Errors.NONE); + assertThat(data.records.sizeInBytes()).isEqualTo(batch1Size + batch2Size); } @Test @@ -487,8 +499,8 @@ public void testFirstFailureStopsProcessingNoDataReturned() { // Should return KAFKA_STORAGE_ERROR because no data is available // (first range failed, so we stop and don't process the second range to avoid gaps) - assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); - assertEquals(MemoryRecords.EMPTY, data.records); + assertThat(data.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(data.records).isEqualTo(MemoryRecords.EMPTY); } @Test @@ -537,8 +549,11 @@ public void testMultipleBatchesPartialFailureReturnsCompleteBatches() { FetchPartitionData data = result.get(partition0); // Should return batch 1 only (complete batch) since batch 2 is incomplete - assertEquals(Errors.NONE, data.error); - assertEquals(batch1Size, data.records.sizeInBytes()); + assertThat(data) + .satisfies(d -> { + assertThat(d.error).isEqualTo(Errors.NONE); + assertThat(d.records.sizeInBytes()).isEqualTo(batch1Size); + }); } @Test @@ -591,13 +606,17 @@ public void testSingleBatchWithMultipleExtentsAllSucceed() { FetchPartitionData data = result.get(partition0); // Should succeed with complete batch - assertEquals(Errors.NONE, data.error); - assertEquals(batchSize, data.records.sizeInBytes()); + assertThat(data) + .satisfies(d -> { + assertThat(d.error).isEqualTo(Errors.NONE); + assertThat(d.records.sizeInBytes()).isEqualTo(batchSize); + }); + Iterator iterator = data.records.records().iterator(); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(firstValue), iterator.next().value()); - assertTrue(iterator.hasNext()); - assertEquals(ByteBuffer.wrap(secondValue), iterator.next().value()); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(firstValue)); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next().value()).isEqualTo(ByteBuffer.wrap(secondValue)); } @Test @@ -658,8 +677,11 @@ public void testSingleBatchWithMissingMiddleExtentFails() { // Should fail because batch is incomplete (missing middle extent) // Incomplete batches are useless to consumers, so we return KAFKA_STORAGE_ERROR - assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); - assertEquals(MemoryRecords.EMPTY, data.records); + assertThat(data) + .satisfies(d -> { + assertThat(d.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(d.records).isEqualTo(MemoryRecords.EMPTY); + }); } @Test @@ -723,8 +745,11 @@ public void testSingleBatchWithFailedMiddleExtentFails() { // Should fail because batch is incomplete (middle extent failed, groupFileData stops at first failure) // Incomplete batches are useless to consumers, so we return KAFKA_STORAGE_ERROR - assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); - assertEquals(MemoryRecords.EMPTY, data.records); + assertThat(data) + .satisfies(d -> { + assertThat(d.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(d.records).isEqualTo(MemoryRecords.EMPTY); + }); } @Test @@ -784,7 +809,10 @@ public void testSingleBatchWithGapInExtentsFails() { // Should fail because batch has a gap (not contiguous) // Incomplete batches are useless to consumers, so we return KAFKA_STORAGE_ERROR - assertEquals(Errors.KAFKA_STORAGE_ERROR, data.error); - assertEquals(MemoryRecords.EMPTY, data.records); + assertThat(data) + .satisfies(d -> { + assertThat(d.error).isEqualTo(Errors.KAFKA_STORAGE_ERROR); + assertThat(d.records).isEqualTo(MemoryRecords.EMPTY); + }); } } From 4ac0caab0c2c725b840c40b96f4605a6f58d6068 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 5 Jan 2026 14:01:03 +0200 Subject: [PATCH 6/6] fixup! feat(inkless): rate-limited hot/cold path for fetch operations Adding CRC check after reading from remote to ensure batch is valid. --- .../aiven/inkless/consume/FetchCompleter.java | 5 ++ .../inkless/consume/FetchCompleterTest.java | 89 +++++++++++++++++++ 2 files changed, 94 insertions(+) diff --git a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java index cae5ea5a66..97ee7975ec 100644 --- a/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java +++ b/storage/inkless/src/main/java/io/aiven/inkless/consume/FetchCompleter.java @@ -316,6 +316,11 @@ private static MemoryRecords createMemoryRecords( } MutableRecordBatch mutableRecordBatch = iterator.next(); + // Validate batch integrity (CRC checksum, size) before modifying it. + // This catches corrupted data that passed size validation but has invalid checksums. + // We validate before setLastOffset/setMaxTimestamp to avoid modifying corrupted batches. + mutableRecordBatch.ensureValid(); + // set last offset mutableRecordBatch.setLastOffset(batch.metadata().lastOffset()); diff --git a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java index e219bc1896..a6da71dd61 100644 --- a/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java +++ b/storage/inkless/src/test/java/io/aiven/inkless/consume/FetchCompleterTest.java @@ -20,7 +20,9 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.DefaultRecordBatch; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.SimpleRecord; @@ -815,4 +817,91 @@ public void testSingleBatchWithGapInExtentsFails() { assertThat(d.records).isEqualTo(MemoryRecords.EMPTY); }); } + + @Test + public void testCorruptedBatchWithInvalidCrcThrowsFetchException() { + // Test that ensureValid() catches corrupted batches with invalid CRC checksums. + // This validates data integrity even when size validation passes. + // + // Scenario: Data passes size validation (extents cover batch range) but has corrupted + // CRC checksum. ensureValid() should detect this and throw CorruptRecordException, + // which gets wrapped in FetchException. + byte[] value = {1, 2, 3}; + MemoryRecords records = MemoryRecords.withRecords(0L, Compression.NONE, new SimpleRecord(value)); + + Map fetchInfos = Map.of( + partition0, new FetchRequest.PartitionData(topicId, 0, 0, 1000, Optional.empty()) + ); + int logStartOffset = 0; + long logAppendTimestamp = 10L; + long maxBatchTimestamp = 20L; + int highWatermark = 1; + + Map coordinates = Map.of( + partition0, FindBatchResponse.success(List.of( + new BatchInfo(1L, OBJECT_KEY_A.value(), BatchMetadata.of(partition0, 0, records.sizeInBytes(), 0, 0, logAppendTimestamp, maxBatchTimestamp, TimestampType.CREATE_TIME)) + ), logStartOffset, highWatermark) + ); + + // Create corrupted data: valid structure but invalid CRC checksum + // Corrupt a field that's covered by CRC (LAST_OFFSET_DELTA) to invalidate the checksum + // This will pass size validation but fail ensureValid() CRC check + ByteBuffer corruptedBuffer = records.buffer().duplicate(); + // Modify LAST_OFFSET_DELTA which is covered by CRC, causing CRC mismatch + int lastOffsetDeltaOffset = DefaultRecordBatch.LAST_OFFSET_DELTA_OFFSET; + corruptedBuffer.putInt(lastOffsetDeltaOffset, 999); // Corrupt the value + + final ByteRange range = new ByteRange(0, records.sizeInBytes()); + List files = List.of( + new FileExtentResult.Success(OBJECT_KEY_A, range, FileFetchJob.createFileExtent(OBJECT_KEY_A, range, corruptedBuffer)) + ); + + FetchCompleter job = new FetchCompleter( + new MockTime(), + OBJECT_KEY_CREATOR, + fetchInfos, + coordinates, + files, + durationMs -> {} + ); + + // Should throw FetchException because ensureValid() detects corrupted CRC + // Exception chain: CorruptRecordException -> FetchException + // + // ensureValid() validates: + // 1. Batch size >= minimum overhead + // 2. CRC checksum matches computed value + // When CRC is invalid, it throws CorruptRecordException which gets wrapped in FetchException + assertThatThrownBy(() -> { + Map result = job.get(); + FetchPartitionData data = result.get(partition0); + // If we get here without exception, the error should indicate corruption + if (data.error == Errors.NONE) { + throw new AssertionError("Expected exception for corrupted batch, but got Errors.NONE"); + } + }) + .isInstanceOf(FetchException.class) + .satisfies(exception -> { + FetchException fetchException = (FetchException) exception; + Throwable cause = fetchException.getCause(); + + assertThat(cause) + .as("Cause should be CorruptRecordException from ensureValid() CRC validation") + .isNotNull() + .isInstanceOf(CorruptRecordException.class); + + CorruptRecordException corruptException = (CorruptRecordException) cause; + + // Validate the error message indicates CRC corruption + // Example message: "Record is corrupt (stored crc = X, computed crc = Y)" + String errorMessage = corruptException.getMessage(); + assertThat(errorMessage) + .as("Error message should indicate corruption/CRC issue") + .isNotNull() + .satisfies(msg -> { + assertThat(msg.toLowerCase()) + .contains("record is corrupt", "crc"); + }); + }); + } }