Skip to content

Conversation

@lk-chen
Copy link
Contributor

@lk-chen lk-chen commented Apr 27, 2025

Why are these changes needed?

LLM task is usually long-running, and duration varies a lot. This can easily cause long-tail problem if batch size is too large.

For example, within a batch, most prompts finished, while there's one prompt keep decoding, blocking the whole batch to finish. Ray data cannot schedule more batches, if the long-tail happens in all running batches. And vLLM engine is not saturated in this case (only decoding one prompt from each batch, while vLLM could potentially handle 256 sequence concurrently), causing low throughput.

This PR

  • changes the default value of certain fields. For LLM batch inference case, we actually want a small batch_size to avoid long tail, and large max_concurrent_batches to saturate engine.
  • log warning if the config cannot saturate engine

Benchmarking on a 10k ShareGPT dataset, on L40S GPU (vLLM 0.8.4, VLLM_USE_V1=0)

batch_size concurrent_batches throughput (req./s) output_throughput (tk/s)
1 512 12.08 2357.43
2 256 11.93 2350.91
4 128 11.86 2333.53
8 64 12.20 2383.35
16 32 12.18 2404.00
32 16 11.94 2347.59
64 8 11.91 2347.94
128 4 11.47 2272.45
256 2 10.48 2055.64
512 1 8.04 1602.82
1 256 11.65 2318.72
2 128 11.90 2348.74
4 64 12.02 2353.33
8 32 11.86 2315.15
16 16 12.10 2368.09
32 8 11.91 2320.09
64 4 11.68 2280.55
128 2 10.30 2018.99
256 1 7.96 1575.01

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: Linkun Chen <github@lkchen.net>
@lk-chen lk-chen requested a review from a team as a code owner April 27, 2025 06:45
@lk-chen lk-chen added the go add ONLY when ready to merge, run all tests label Apr 27, 2025

# Core stage -- the vLLM engine.

if config.batch_size * config.max_concurrent_batches < DEFAULT_VLLM_BATCH_SIZE:
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't get why DEFAULT_VLLM_BATCH_SIZE is set to be 256? and why does this warning make sense?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

256 comes from vLLM, I've refactored to always read from vLLM instead of hardcoding.

This warning has two parts:

  1. product of batch_size and max_concurrent_batches indicates total concurrent prompts, if this product is too small, vLLM is under-utilized
  2. I want user increase max_concurrent_batches instead of batch_size, since the latter cause long-tail blocking

which part doesn't make sense to you, could you clarify?

Copy link
Contributor

Choose a reason for hiding this comment

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

ok explanation is clear now. The 256 is really coming from engine_kwargs of vllm. It's not hardcoded inside vllm either. Basically you are saying ray data will adjust itself to the corresponding max_seq number set on the vllm engine replica, by adjusting the max_concurrent_batches instead of adjusting the batch size. Can we get some reliable benchmark datapoints attached to this PR for different combos of batch_size and max_concurrent_batches to show the basis of this choice?

What I mean is that we should run a benchmark for sweep of batch_size and max_concurrent_batches under similar max_seqs.

Basically

for max_seq: [128, 256, 512]:
for (bsize, max_concurrent_batches) in [(1, max_seq), (2, max_seq/2), ..., (max_seq, 1)]:
Measure: E2E runtime on a fixed dataset of say 10k rows
For baseline comparisons also measure E2E time when bsize=10k, max_concurrent_batches=1 on similar max_seq levels. 

max_tasks_in_flight_per_actor=max(
DEFAULT_MAX_TASKS_IN_FLIGHT, config.max_concurrent_batches
),
),
Copy link
Contributor

Choose a reason for hiding this comment

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

@raulchen if this is deprecated, what is the right way to control the max_tasks_in_flight_per_actor properly?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's what the comment says, this deprecated field is the only way to control max_tasks_in_flight_per_actor

Copy link
Contributor

Choose a reason for hiding this comment

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

As users of ray data yes, but ray data should either not deprecate this, or give a more stable solution. I want to understand if this is what is recommended for the issue above. cc @alexeykudinkin @gvspraveen @richardliaw

Copy link
Contributor

Choose a reason for hiding this comment

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

unfortunately, we haven't exposed a new API for this yet.
created a ticket here #52667
For now, let's use the current way.

@hainesmichaelc hainesmichaelc added the community-contribution Contributed by the community label Apr 28, 2025
Signed-off-by: lkchen <github@lkchen.net>
@mascharkh mascharkh added data Ray Data-related issues performance labels Apr 28, 2025
lk-chen added 2 commits April 28, 2025 13:51
Signed-off-by: Linkun Chen <github@lkchen.net>
Signed-off-by: Linkun Chen <github@lkchen.net>
@kouroshHakha kouroshHakha removed the community-contribution Contributed by the community label Apr 28, 2025
@lk-chen
Copy link
Contributor Author

lk-chen commented Apr 28, 2025

@kouroshHakha yes, I started running max_seq=256 an hour ago

@kouroshHakha kouroshHakha enabled auto-merge (squash) April 29, 2025 16:27
@kouroshHakha
Copy link
Contributor

kouroshHakha commented Apr 29, 2025

ok discussed offline. With this PR we are basically enabling configuring max_concurrency on a udf actor pools. By modifying bsize and max_concurrency we can shave off an overhead of 20 ish % to 10 ish % compared to async vllm for single replica. The rest of the remaining overhead must be ray serialization, etc which will be insignificant cost for the value of horizontal scaling. Both @lk-chen and I agree that we should put a pin on this and just be aware that in single replica there could be an overhead of 7-10% compared to async vllm.

@kouroshHakha kouroshHakha merged commit eda17e7 into ray-project:master Apr 29, 2025
6 checks passed
iamjustinhsu pushed a commit that referenced this pull request May 3, 2025
Signed-off-by: Linkun Chen <github@lkchen.net>
Signed-off-by: lkchen <github@lkchen.net>
Signed-off-by: jhsu <jhsu@anyscale.com>
@lk-chen lk-chen deleted the lkchen-raydata_high_throuput branch May 28, 2025 17:03
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-backlog data Ray Data-related issues go add ONLY when ready to merge, run all tests performance

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants