Skip to content

Add request coalescing and relevant configuration#356

Merged
fuatbasik merged 4 commits intoawslabs:mainfrom
fuatbasik:add-request-coalescing
Oct 9, 2025
Merged

Add request coalescing and relevant configuration#356
fuatbasik merged 4 commits intoawslabs:mainfrom
fuatbasik:add-request-coalescing

Conversation

@fuatbasik
Copy link
Collaborator

@fuatbasik fuatbasik commented Sep 29, 2025

Description of change

With this change, AAL can coalesce requests in a close proximity to each other.

This enables to take spatial locality
into account and reduce request count. In my testing I observed that this is particularly important for vectored reads
reading vectors near each other.

There are two configurations to manage this feature. First one is to enable disable request coalescing:
request.coalesce which is by default set to true.

Second one is request.coalesce.tolerance, which is the number of bytes to tolerate (read even if not immediately needed) when merging ranges.

Two ranges will be merged if end of the first range is less than tolerate bytes away from the second one.

Relevant issues

N/A

Does this contribution introduce any breaking changes to the existing APIs or behaviors?

No. It changes the behaviour of vectored reads and merges smaller requests to bigger requests.
These bigger requests might be split to smaller requests later to make sure all requests are around target.request.size

Does this contribution introduce any new public APIs or behaviors?

How was the contribution tested?

Added new unit tests. Extended existing IntegrationTests to run for both coalescing enabled and disabled.

Does this contribution need a changelog entry?

  • Yes, i shall be updating the README after we finalise the behaviour part of this PR.

By submitting this pull request, I confirm that my contribution is made under the terms of the Apache 2.0 license and I agree to the terms of the Developer Certificate of Origin (DCO).

Copy link
Contributor

@stubz151 stubz151 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ty this awesome, just some smallish comments.


@Override
public String toString() {
return String.format("offset: %d, length: %d", offset, length);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what do u think about adding some info about the future as well? like is there any id? or can even add it's current state? it might come in handy down the line.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, i am now adding future's to-string here

@@ -264,6 +264,45 @@ void testGetRequiredStringThrowsIfNotSet() {
IllegalArgumentException.class, () -> configuration.getRequiredString("stringConfig1"));
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we squash some of these tests down using parameterized tests?

/** Flag to enable request Coalescing */
@Builder.Default private boolean requestCoalesce = DEFAULT_COALESCE_REQUEST;

private static final String REQUEST_COALESCE_KEY = "request.coalesce";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why add it as a config? when would we not want to do this?

currentRange = nextRange;
}
}
coalescedRanges.add(currentRange);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

public void coalesce(long tolerance) {
    if (this.prefetchRanges.size() < 2) {
        return;
    }
    
    // Ensure ranges are ordered by their start position
    Collections.sort(this.prefetchRanges);
    
    int writeIndex = 0;
    Range currentRange = this.prefetchRanges.get(0);

    for (int i = 1; i < this.prefetchRanges.size(); i++) {
        Range nextRange = this.prefetchRanges.get(i);
        
        if (currentRange.getEnd() + tolerance >= nextRange.getStart()) {
            // Merge ranges
            currentRange = new Range(currentRange.getStart(), 
                Math.max(currentRange.getEnd(), nextRange.getEnd()));
        } else {
            // Store the completed merged range
            this.prefetchRanges.set(writeIndex++, currentRange);
            currentRange = nextRange;
        }
    }
    
    this.prefetchRanges.set(writeIndex++, currentRange);
    
   //test this a lil.
    while (this.prefetchRanges.size() > writeIndex) {
        this.prefetchRanges.remove(this.prefetchRanges.size() - 1);
    }
}

What do you think of this? you don't need a second list to keep track of it then which I like. I also find it a bit easier to follow.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. I updated this part in rev2

Copy link
Collaborator

@ahmarsuhail ahmarsuhail left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good, just think we need to add some additional logic in that execute function

@@ -391,12 +404,12 @@ protected void testReadVectoredInSingleBlock(
protected void testReadVectoredForSequentialRanges(
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i see that this tests range coalescing, but it would be good to add a range coalescing test as well, which will test ranges are within that 1MB distance. eg: [0-4MB, 4.3MB - 5MB, 5.9MB - 7MB].

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @ahmarsuhail. Do test under IOPlanTest cover your concern. I was trying to put corner cases for actual coalesing business logic there instead of here.

@fuatbasik fuatbasik force-pushed the add-request-coalescing branch from c6cf086 to c6960a7 Compare October 7, 2025 15:24
@fuatbasik fuatbasik temporarily deployed to integration-tests October 7, 2025 15:24 — with GitHub Actions Inactive
ozkoca
ozkoca previously approved these changes Oct 7, 2025
Copy link
Contributor

@ozkoca ozkoca left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

/** Flag to enable request Coalescing */
@Builder.Default private boolean requestCoalesce = DEFAULT_COALESCE_REQUEST;

private static final String REQUEST_COALESCE_KEY = "request.coalesce";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: We may rename like request.coalesce.enabled

ahmarsuhail
ahmarsuhail previously approved these changes Oct 8, 2025
Copy link
Collaborator

@ahmarsuhail ahmarsuhail left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@fuatbasik fuatbasik dismissed stale reviews from ahmarsuhail and ozkoca via d3fa0ba October 8, 2025 15:41
@fuatbasik fuatbasik force-pushed the add-request-coalescing branch from d3fa0ba to 19c1be7 Compare October 8, 2025 15:50
Copy link
Collaborator

@ahmarsuhail ahmarsuhail left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo in the coalesce method, using the wrong variable

@fuatbasik fuatbasik temporarily deployed to integration-tests October 8, 2025 18:54 — with GitHub Actions Inactive
Copy link
Collaborator

@ahmarsuhail ahmarsuhail left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM

@fuatbasik fuatbasik merged commit e2b2de6 into awslabs:main Oct 9, 2025
4 checks passed
dongjoon-hyun added a commit to apache/spark that referenced this pull request Jan 28, 2026
### What changes were proposed in this pull request?

This PR aims to upgrade `analyticsaccelerator-s3` to 1.3.1 for Apache Spark 4.2.0 in line with Apache Hadoop 3.4.3 (HADOOP-19742).
- apache/hadoop#8093

### Why are the changes needed?

To bring the latest fixes.
- https://github.com/awslabs/analytics-accelerator-s3/releases/tag/v1.3.1
  - awslabs/analytics-accelerator-s3#360
  - awslabs/analytics-accelerator-s3#361
  - awslabs/analytics-accelerator-s3#363
  - awslabs/analytics-accelerator-s3#356
  - awslabs/analytics-accelerator-s3#358

### Does this PR introduce _any_ user-facing change?

No behavior change.

### How was this patch tested?

Pass the CIs.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #54031 from dongjoon-hyun/SPARK-55254.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants