Skip to content

[Serve][4/5] Add HAProxy ingress request router dispatch path#62669

Merged
kouroshHakha merged 70 commits into
ray-project:masterfrom
eicherseiji:ingress-bypass/3-haproxy-lua
May 6, 2026
Merged

[Serve][4/5] Add HAProxy ingress request router dispatch path#62669
kouroshHakha merged 70 commits into
ray-project:masterfrom
eicherseiji:ingress-bypass/3-haproxy-lua

Conversation

@eicherseiji
Copy link
Copy Markdown
Contributor

@eicherseiji eicherseiji commented Apr 16, 2026

Why this is needed

PR3 introduces TargetGroup.ingress_request_router_targets. This PR uses it: HAProxy asks a router replica which data-plane replica to use, then pins the request to that replica.

Router contract

HAProxy calls the router with the original request body and expects a JSON replica_id:

POST /internal/route HTTP/1.0
Content-Type: application/json

{ ...original request body... }
{ "replica_id": "<actor name>" }

If the router errors, times out, or returns an unknown replica_id, the request falls through to normal load balancing.

HAProxy routing table

For each router-bearing backend, the frontend gates the Lua action on POST and a matched path prefix:

http-request set-var(txn.ingress_request_router_app) str(<backend>) if is_<backend> !{ var(txn.ingress_request_router_app) -m found }
http-request wait-for-body time 5s if METH_POST has_ingress_request_router_app
http-request lua.route_via_ingress_request_router if METH_POST has_ingress_request_router_app
use_backend <backend>-via-ingress-request-router if is_<backend> { var(txn.via_ingress_request_router) -m found }

The mirror backend pins via use-server and shares health checks with the primary via track:

backend <backend>-via-ingress-request-router
    option redispatch
    use-server <server> if { var(txn.ingress_request_router_target) -m str "<server>" }
    server <server> <host>:<port> track <backend>/<server>

Config shape

ServerConfig(name=..., host=..., port=..., replica_id=...)  # new field
BackendConfig(..., ingress_request_router_servers=[ServerConfig, ...])  # new field

Stack: #62667 -> #62680 -> #62668 -> this -> #62670

Test plan

  • Unit coverage in python/ray/serve/tests/test_haproxy_api.py for Lua rendering, frontend ACLs, mirror backend generation, and config-write idempotency
  • git diff --check
  • Repo commit hooks / linters on commit
  • Targeted HAProxy config / integration tests

Copy link
Copy Markdown
Contributor

@gemini-code-assist gemini-code-assist Bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request implements an "ingress bypass" optimization for Ray Serve, enabling HAProxy to route data plane traffic directly to replicas via a Lua routing script. This mechanism bypasses the standard Serve request pipeline for specific streaming endpoints to improve throughput. The changes include new state management for router deployments, support for custom ASGI apps within replicas, and updated HAProxy templates to handle custom-routed backends. Review feedback highlights several critical areas for improvement in the HAProxy Lua implementation, including the need for high availability in router selection, reducing internal routing timeouts, avoiding blocking I/O on the request path, and preventing resource naming collisions in multi-tenant environments.

Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/replica.py Outdated
@eicherseiji eicherseiji force-pushed the ingress-bypass/3-haproxy-lua branch from a5995ff to acb3b40 Compare April 16, 2026 17:25
@eicherseiji eicherseiji changed the title [Serve] Add HAProxy Lua routing for ingress bypass [Serve][3/4] Add HAProxy Lua routing for ingress bypass Apr 16, 2026
@eicherseiji eicherseiji force-pushed the ingress-bypass/3-haproxy-lua branch from ded732a to 58974a3 Compare April 16, 2026 20:18
@eicherseiji eicherseiji changed the title [Serve][3/4] Add HAProxy Lua routing for ingress bypass [Serve][4/5] Add HAProxy Lua routing for ingress bypass Apr 16, 2026
@eicherseiji eicherseiji force-pushed the ingress-bypass/3-haproxy-lua branch from 58974a3 to 2bf1b9f Compare April 16, 2026 20:21
@eicherseiji eicherseiji force-pushed the ingress-bypass/3-haproxy-lua branch 6 times, most recently from 7d87d82 to 8b78005 Compare April 18, 2026 01:56
@eicherseiji eicherseiji changed the title [Serve][4/5] Add HAProxy Lua routing for ingress bypass [Serve][4/5] Route HAProxy requests by replica ID Apr 20, 2026
@eicherseiji eicherseiji force-pushed the ingress-bypass/3-haproxy-lua branch 6 times, most recently from b1ab19e to 0311193 Compare April 21, 2026 06:24
@eicherseiji eicherseiji force-pushed the ingress-bypass/3-haproxy-lua branch 5 times, most recently from 91dba79 to c88f269 Compare May 1, 2026 16:30
Wire up the controller to support "ingress bypass" mode where a router
deployment (e.g. OpenAiIngress) serves /internal/route for Lua routing
decisions, while non-router deployments (e.g. LLMServer) serve data
plane traffic directly via their direct ingress ports.

Key changes:
- controller.py: _get_target_groups_for_app_with_router splits targets
  into router_targets (for Lua) and main targets (data plane)
- application_state.py: track router deployment name per app
- build_app.py, client.py, deploy_utils.py, deployment_info.py: plumb
  router flag through deployment args
- default_impl.py: factory plumbing for router deployment
- schema.py: router_targets field on TargetGroup

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
- Remove unused _summarize_target, _summarize_target_groups,
  _summarize_fallback_targets helpers
- Remove unused get_fallback_targets_for_proxy_manager method
- Revert unrelated RAY_SERVE_THROUGHPUT_OPTIMIZED env var plumbing
- Fix stale docstring referencing OpenAiIngress

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
@eicherseiji eicherseiji changed the title [Serve][4/5] Route HAProxy requests by replica ID [Serve][4/5] Add HAProxy ingress request router dispatch path May 5, 2026
Comment thread python/ray/serve/_private/haproxy_templates.py
@ray-gardener ray-gardener Bot added the serve Ray Serve Related Issue label May 5, 2026
Comment thread python/setup.py Outdated
Copy link
Copy Markdown
Contributor

@kouroshHakha kouroshHakha left a comment

Choose a reason for hiding this comment

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

Nice progression on top of #62668 — the ingress_request_router_targets plumbing from 3/N now has a clean consumer here, and the choice to keep the JSON parsing inside Lua (rather than pulling in lua-cjson as a runtime dep) is the right call given that this script lives on every Serve host. The end-to-end test is the convincing piece: it exercises the wait-for-body → Lua → use-server pin on real HAProxy.

Two things I'd want fixed before merge, both in the Python→Jinja boundary:

  1. Lua action can be referenced without being loaded when ingress_request_router_servers exist but _routers_and_targets_by_backend returns an empty result (e.g. all data-plane replicas momentarily have replica_id=None, or backend.servers is empty during scaling). HAProxy rejects the config in that case. This is the Cursor Bugbot finding — it's real. Easy fix: make has_ingress_request_router reflect "the Lua actually got written," not "any backend has router servers."
  2. Lua action emission isn't gated on the lua-load-per-thread guard — the global section guards on has_ingress_request_router and ingress_request_router_lua_path, but the frontend section guards only on has_ingress_request_router. Same root cause as (1).

A few smaller things below — fallback observability, buffer sizing for larger prompts, formatting brittleness in the Lua codegen, and a couple of test gaps. None are merge-blockers.

Stack-level note: with the single min((port, host)) router pick + 5s timeout, a single bad router replica will incur up to 5s of latency on every POST until LB takes over (since option redispatch doesn't help here — the call is from Lua, not via a backend). When 5/N lands and LLMRouter is the actual call site, this is the path that determines the upper bound on first-response latency for bypass-mode POSTs under partial failure. Worth tracking even if not addressed in this PR.

Bigger-picture: every replica add/remove/restart currently regenerates this Lua file and triggers a graceful HAProxy reload. The runtime admin socket is already plumbed (_send_socket_command) but used read-only. There's a real follow-up worth scoping where membership data moves to HAProxy maps + runtime-managed servers, so scaling events become socket commands instead of full reloads. Comments on consolidating Lua serialization and on _generate_config_file_internal elaborate.

Note

This review was co-written with AI assistance (Claude Code).

Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/tests/test_haproxy_api.py
Comment thread python/ray/serve/_private/ingress_request_router.lua.tmpl Outdated
Comment thread python/ray/serve/_private/haproxy_templates.py Outdated
Comment thread python/ray/serve/_private/haproxy.py
Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/haproxy.py
Comment thread python/ray/serve/_private/haproxy.py Outdated
Comment thread python/ray/serve/_private/haproxy.py
Comment thread python/ray/serve/_private/haproxy.py Outdated
When a backend has `ingress_request_router_servers` but no replica with a
`replica_id` (router up before replicas register, or all replicas gone),
`_write_ingress_request_router_lua` correctly returns None — but the prior
`has_ingress_request_router = any(...)` check was True regardless, so the
frontend emitted `lua.route_via_ingress_request_router` while the global
section skipped `lua-load-per-thread`. HAProxy rejects the config with
"unknown action".

Derive `has_ingress_request_router` from the Lua write result so the two
sections can never disagree, and add a regression test.

Also:
- Move `INGRESS_REQUEST_ROUTER_TIMEOUT_S` to constants.py as
  `RAY_SERVE_HAPROXY_INGRESS_REQUEST_ROUTER_TIMEOUT_S` to match the rest
  of the `RAY_SERVE_HAPROXY_*` knobs (env-overridable).
- Replace hardcoded ports 8200–8204 in the e2e test with `find_free_port()`.
- Rename "TTFT" → "first-response latency" in a `_private/` docstring.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Prefix-cache-aware routing degrades but doesn't break when the request body
is truncated by tune.bufsize — the prefix is at the head of the body, which
is what the router uses to make a routing decision. Skipping the router on
truncation defeats prefix-cache routing exactly for the long prompts where
it matters most.

Forward the truncated body to the router with `X-Body-Truncated:
<bytes>/<content-length>` so it can do best-effort prefix matching, and
raise the default `tune.bufsize` to 256KB (env-overridable via
RAY_SERVE_HAPROXY_INGRESS_REQUEST_ROUTER_BUFSIZE) so most realistic
OpenAI-API payloads fit without truncation.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Comment thread python/ray/serve/_private/haproxy_templates.py
Comment thread python/ray/serve/_private/haproxy_templates.py
Three follow-ups on review feedback in ray-project#62669:

- Move `_LUA_TEMPLATE` from a module-import side effect into a lazy
  `@functools.cache`'d helper so importing `haproxy` no longer requires
  the .lua.tmpl on disk. The template still loads at most once per
  process; missing-file failures now raise FileNotFoundError with a
  clear message at first config render rather than at module import.
- Reorder the deterministic-router-pick sort key from (port, host) to
  (host, port) so co-located routers cluster adjacent in debug output.
- Pin the `replica_id` JSON shape contract above `extract_replica_id`
  in the Lua template so the regex's assumptions are explicit for
  anyone broadening the parser later.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
The frontend `use_backend ...-via-ingress-request-router` rule and the
mirror backend definition were gated only on `ingress_request_router_servers`,
not on `has_ingress_request_router`. In the no-replica-IDs state the Lua is
not loaded, so `txn.via_ingress_request_router` is never set and the rule is
unreachable, but the dead config is still rendered. Same root cause as the
config-rejection bug fix in 9624a88; gating consistently removes the
inconsistency and prevents future features that set the txn var elsewhere
from accidentally activating an unprepared mirror backend.

Cursor bot review on 7d3224f.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
@kouroshHakha kouroshHakha enabled auto-merge (squash) May 6, 2026 05:04
aslonnie pushed a commit that referenced this pull request May 6, 2026
Prep for the HAProxy ingress-request-router Lua template introduced in
#62669. The template is loaded at runtime by
`ray.serve._private.haproxy` (via `Path(__file__).parent /
"ingress_request_router.lua.tmpl"`), so it must be packaged into the
wheel.

Splitting this out per @aslonnie's review feedback on #62669 so the
`package_data` change can be reviewed independently of the HAProxy/Lua
logic.

The pattern matches zero files until #62669 lands; setuptools accepts
that without error.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
…haproxy-lua

Signed-off-by: Seiji Eicher <seiji@anyscale.com>

# Conflicts:
#	python/setup.py
@github-actions github-actions Bot disabled auto-merge May 6, 2026 15:57
@kouroshHakha kouroshHakha merged commit b114d73 into ray-project:master May 6, 2026
5 of 6 checks passed
eicherseiji added a commit to eicherseiji/ray that referenced this pull request May 6, 2026
…project#63145)

Prep for the HAProxy ingress-request-router Lua template introduced in
ray-project#62669. The template is loaded at runtime by
`ray.serve._private.haproxy` (via `Path(__file__).parent /
"ingress_request_router.lua.tmpl"`), so it must be packaged into the
wheel.

Splitting this out per @aslonnie's review feedback on ray-project#62669 so the
`package_data` change can be reviewed independently of the HAProxy/Lua
logic.

The pattern matches zero files until ray-project#62669 lands; setuptools accepts
that without error.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
eicherseiji added a commit to eicherseiji/ray that referenced this pull request May 6, 2026
…oject#62669)

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com>
chillCode404 pushed a commit to chillCode404/ray-contrib that referenced this pull request May 9, 2026
…project#63145)

Prep for the HAProxy ingress-request-router Lua template introduced in
ray-project#62669. The template is loaded at runtime by
`ray.serve._private.haproxy` (via `Path(__file__).parent /
"ingress_request_router.lua.tmpl"`), so it must be packaged into the
wheel.

Splitting this out per @aslonnie's review feedback on ray-project#62669 so the
`package_data` change can be reviewed independently of the HAProxy/Lua
logic.

The pattern matches zero files until ray-project#62669 lands; setuptools accepts
that without error.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
chillCode404 pushed a commit to chillCode404/ray-contrib that referenced this pull request May 9, 2026
…oject#62669)

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com>
am-kinetica pushed a commit to kineticadb/ray that referenced this pull request May 14, 2026
…project#63145)

Prep for the HAProxy ingress-request-router Lua template introduced in
ray-project#62669. The template is loaded at runtime by
`ray.serve._private.haproxy` (via `Path(__file__).parent /
"ingress_request_router.lua.tmpl"`), so it must be packaged into the
wheel.

Splitting this out per @aslonnie's review feedback on ray-project#62669 so the
`package_data` change can be reviewed independently of the HAProxy/Lua
logic.

The pattern matches zero files until ray-project#62669 lands; setuptools accepts
that without error.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: anindyam1969 <amukherjee@kinetica.com>
am-kinetica pushed a commit to kineticadb/ray that referenced this pull request May 14, 2026
…oject#62669)

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com>
Signed-off-by: anindyam1969 <amukherjee@kinetica.com>
Lucas61000 pushed a commit to Lucas61000/ray that referenced this pull request May 15, 2026
…project#63145)

Prep for the HAProxy ingress-request-router Lua template introduced in
ray-project#62669. The template is loaded at runtime by
`ray.serve._private.haproxy` (via `Path(__file__).parent /
"ingress_request_router.lua.tmpl"`), so it must be packaged into the
wheel.

Splitting this out per @aslonnie's review feedback on ray-project#62669 so the
`package_data` change can be reviewed independently of the HAProxy/Lua
logic.

The pattern matches zero files until ray-project#62669 lands; setuptools accepts
that without error.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Lucas61000 pushed a commit to Lucas61000/ray that referenced this pull request May 15, 2026
…oject#62669)

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com>
alexandrplashchinsky pushed a commit to alexandrplashchinsky/ray-alex that referenced this pull request May 29, 2026
…project#63145)

Prep for the HAProxy ingress-request-router Lua template introduced in
ray-project#62669. The template is loaded at runtime by
`ray.serve._private.haproxy` (via `Path(__file__).parent /
"ingress_request_router.lua.tmpl"`), so it must be packaged into the
wheel.

Splitting this out per @aslonnie's review feedback on ray-project#62669 so the
`package_data` change can be reviewed independently of the HAProxy/Lua
logic.

The pattern matches zero files until ray-project#62669 lands; setuptools accepts
that without error.

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: Alexandr Plashchinsky <alexandr.plashchinsky@alexandrplashchinsky-H765G66H9V.local>
alexandrplashchinsky pushed a commit to alexandrplashchinsky/ray-alex that referenced this pull request May 29, 2026
…oject#62669)

Signed-off-by: Seiji Eicher <seiji@anyscale.com>
Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com>
Signed-off-by: Alexandr Plashchinsky <alexandr.plashchinsky@alexandrplashchinsky-H765G66H9V.local>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

go add ONLY when ready to merge, run all tests serve Ray Serve Related Issue

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants