Skip to content

Commit 925b25c

Browse files
[serve] Remove RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE flag (#51649)
## Why are these changes needed? This PR removes a feature flag that controls whether the proxy should use cached replica queue length values for routing. The FF was [introduced](#42943) over a year ago as a way for users to quickly switch back to the previous implementation. It has been enabled by default for [over a year](#43169) now and works as expected, so let's remove it. Consequently, this PR also removes `RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS`, as it is always enabled if `RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE` is enabled. Signed-off-by: akyang-anyscale <alexyang@anyscale.com>
1 parent f3afcba commit 925b25c

File tree

4 files changed

+2
-75
lines changed

4 files changed

+2
-75
lines changed

python/ray/serve/_private/constants.py

Lines changed: 0 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -328,19 +328,6 @@ def parse_latency_buckets(bucket_str: str, default_buckets: list) -> list:
328328
os.environ.get("RAY_SERVE_MAX_QUEUE_LENGTH_RESPONSE_DEADLINE_S", 1.0)
329329
)
330330

331-
# Feature flag for caching queue lengths for faster routing in each handle.
332-
RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE = (
333-
os.environ.get("RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE", "1") == "1"
334-
)
335-
336-
# Feature flag for strictly enforcing max_ongoing_requests (replicas will reject
337-
# requests).
338-
RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS = (
339-
os.environ.get("RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS", "0") == "1"
340-
# Strict enforcement path must be enabled for the queue length cache.
341-
or RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE
342-
)
343-
344331
# Length of time to respect entries in the queue length cache when scheduling requests.
345332
RAY_SERVE_QUEUE_LENGTH_CACHE_TIMEOUT_S = float(
346333
os.environ.get("RAY_SERVE_QUEUE_LENGTH_CACHE_TIMEOUT_S", 10.0)

python/ray/serve/_private/default_impl.py

Lines changed: 2 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -15,8 +15,6 @@
1515
RequestProtocol,
1616
)
1717
from ray.serve._private.constants import (
18-
RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE,
19-
RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS,
2018
RAY_SERVE_PROXY_PREFER_LOCAL_AZ_ROUTING,
2119
RAY_SERVE_PROXY_PREFER_LOCAL_NODE_ROUTING,
2220
)
@@ -163,9 +161,7 @@ def create_router(
163161
else None,
164162
availability_zone,
165163
# Streaming ObjectRefGenerators are not supported in Ray Client
166-
use_replica_queue_len_cache=(
167-
not is_inside_ray_client_context and RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE
168-
),
164+
use_replica_queue_len_cache=not is_inside_ray_client_context,
169165
create_replica_wrapper_func=lambda r: RunningReplica(r),
170166
)
171167

@@ -177,10 +173,7 @@ def create_router(
177173
handle_source=handle_options._source,
178174
replica_scheduler=replica_scheduler,
179175
# Streaming ObjectRefGenerators are not supported in Ray Client
180-
enable_strict_max_ongoing_requests=(
181-
not is_inside_ray_client_context
182-
and RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS
183-
),
176+
enable_strict_max_ongoing_requests=not is_inside_ray_client_context,
184177
resolve_request_arg_func=resolve_deployment_response,
185178
)
186179

python/ray/serve/tests/BUILD

Lines changed: 0 additions & 48 deletions
Original file line numberDiff line numberDiff line change
@@ -301,54 +301,6 @@ py_test(
301301
],
302302
)
303303

304-
# Tests disabling queue len caching feature flag.
305-
# TODO(edoakes): remove this after the FF is removed.
306-
py_test_module_list(
307-
size = "medium",
308-
data = glob(["test_config_files/**/*"]),
309-
env = {"RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE": "0"},
310-
files = [
311-
"test_handle_1.py",
312-
"test_handle_2.py",
313-
"test_handle_cancellation.py",
314-
"test_handle_streaming.py",
315-
"test_http_cancellation.py",
316-
"test_multiplex.py",
317-
"test_request_timeout.py",
318-
],
319-
name_suffix = "_with_queue_len_cache_disabled",
320-
tags = [
321-
"exclusive",
322-
"no_windows",
323-
"team:serve",
324-
],
325-
deps = [
326-
":common",
327-
":conftest",
328-
"//python/ray/serve:serve_lib",
329-
],
330-
)
331-
332-
# Tests autoscaling when queue len caching is disabled.
333-
py_test_module_list(
334-
size = "large",
335-
env = {"RAY_SERVE_ENABLE_QUEUE_LENGTH_CACHE": "0"},
336-
files = [
337-
"test_autoscaling_policy.py",
338-
],
339-
name_suffix = "_with_queue_len_cache_disabled",
340-
tags = [
341-
"exclusive",
342-
"no_windows",
343-
"team:serve",
344-
],
345-
deps = [
346-
":common",
347-
":conftest",
348-
"//python/ray/serve:serve_lib",
349-
],
350-
)
351-
352304
# Test old stop-fully-then-start behavior.
353305
# TODO(zcin): remove this after the old behavior is completely removed
354306
py_test_module_list(

python/ray/serve/tests/test_handle_2.py

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@
99
from ray._private.test_utils import SignalActor, async_wait_for_condition
1010
from ray._common.utils import get_or_create_event_loop
1111
from ray.serve._private.constants import (
12-
RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS,
1312
RAY_SERVE_FORCE_LOCAL_TESTING_MODE,
1413
)
1514
from ray.serve.exceptions import RayServeException
@@ -412,10 +411,6 @@ async def __call__(self):
412411
RAY_SERVE_FORCE_LOCAL_TESTING_MODE,
413412
reason="local_testing_mode doesn't respect max_ongoing_requests",
414413
)
415-
@pytest.mark.skipif(
416-
not RAY_SERVE_ENABLE_STRICT_MAX_ONGOING_REQUESTS,
417-
reason="Strict enforcement must be enabled",
418-
)
419414
@pytest.mark.asyncio
420415
async def test_max_ongoing_requests_enforced(serve_instance):
421416
"""Handles should respect max_ongoing_requests enforcement."""

0 commit comments

Comments
 (0)