Skip to content

Commit af94574

Browse files
machichimadioptre
authored andcommitted
[Core]Separate Environment Variables for ray.init() and ray ctl to Reflect Different Protocols and Ports (ray-project#55189)
Currently, both `ray.init()` and ray ctl rely on the same environment variable RAY_ADDRESS to determine the default address to connect to a Ray cluster. However, in practice: `ray.init()` uses the `ray:// protocol` and connects via the GCS gRPC port (default `10001`) ray ctl uses the HTTP protocol (`http://`) and connects via the Dashboard port (default `8265`) This leads to potential confusion when setting the `RAY_ADDRESS` environment variable, as it may not be valid for both tools simultaneously. For example, setting `RAY_ADDRESS=http://localhost:8265/` would work for ray ctl, but not for `ray.init()`, which expects a `ray://` URI with the GCS port. In this PR, we do: - Keep `RAY_ADDRESS` for `ray.init()` - Use `RAY_API_SERVER_ADDRESS` for ray job ctl - Update docs: https://anyscale-ray--55189.com.readthedocs.build/en/55189/cluster/running-applications/job-submission/quickstart.html Closes ray-project#53226 Signed-off-by: Andrew Grosser <[email protected]>
1 parent 14f5a8c commit af94574

File tree

9 files changed

+54
-20
lines changed

9 files changed

+54
-20
lines changed

doc/source/cluster/running-applications/job-submission/quickstart.rst

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -68,13 +68,13 @@ If you are using a local Ray Cluster (``ray start --head``), connect directly at
6868
If you are using a Ray Cluster started on VMs or Kubernetes, follow the instructions there for setting up network access from a client. See :ref:`Using a Remote Cluster <jobs-remote-cluster>` for tips.
6969

7070

71-
To tell the Ray Jobs CLI how to find your Ray Cluster, pass the Ray Dashboard address. Set the ``RAY_ADDRESS`` environment variable:
71+
To tell the Ray Jobs CLI how to find your Ray Cluster, pass the Ray Dashboard address. Set the ``RAY_API_SERVER_ADDRESS`` environment variable:
7272

7373
.. code-block:: bash
7474
75-
$ export RAY_ADDRESS="http://127.0.0.1:8265"
75+
$ export RAY_API_SERVER_ADDRESS="http://127.0.0.1:8265"
7676
77-
Alternatively, you can also pass the ``--address=http://127.0.0.1:8265`` flag explicitly to each Ray Jobs CLI command, or prepend each command with ``RAY_ADDRESS=http://127.0.0.1:8265``.
77+
Alternatively, you can also pass the ``--address=http://127.0.0.1:8265`` flag explicitly to each Ray Jobs CLI command, or prepend each command with ``RAY_API_SERVER_ADDRESS=http://127.0.0.1:8265``.
7878

7979
Additionally, if you wish to pass headers per HTTP request to the Cluster, use the `RAY_JOB_HEADERS` environment variable. This environment variable must be in JSON form.
8080

@@ -217,7 +217,7 @@ Run the following command on your local machine, where ``cluster.yaml`` is the c
217217
ray dashboard cluster.yaml
218218
219219
Once this command is running, verify that you can view the Ray Dashboard in your local browser at ``http://127.0.0.1:8265``.
220-
Also, verify that you set the environment variable ``RAY_ADDRESS`` to ``"http://127.0.0.1:8265"``. After this setup, you can use the Jobs CLI on the local machine as in the preceding example to interact with the remote Ray cluster.
220+
Also, verify that you set the environment variable ``RAY_API_SERVER_ADDRESS`` to ``"http://127.0.0.1:8265"``. After this setup, you can use the Jobs CLI on the local machine as in the preceding example to interact with the remote Ray cluster.
221221

222222
Using the CLI on Kubernetes
223223
^^^^^^^^^^^^^^^^^^^^^^^^^^^

python/ray/_private/ray_constants.py

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -126,6 +126,7 @@ def env_set_by_user(key):
126126
DEFAULT_PORT = 6379
127127

128128
RAY_ADDRESS_ENVIRONMENT_VARIABLE = "RAY_ADDRESS"
129+
RAY_API_SERVER_ADDRESS_ENVIRONMENT_VARIABLE = "RAY_API_SERVER_ADDRESS"
129130
RAY_NAMESPACE_ENVIRONMENT_VARIABLE = "RAY_NAMESPACE"
130131
RAY_RUNTIME_ENV_ENVIRONMENT_VARIABLE = "RAY_RUNTIME_ENV"
131132
RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_ENV_VAR = (

python/ray/dashboard/modules/job/cli.py

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,7 @@ def job_cli_group():
115115
required=False,
116116
help=(
117117
"Address of the Ray cluster to connect to. Can also be specified "
118-
"using the RAY_ADDRESS environment variable."
118+
"using the RAY_API_SERVER_ADDRESS environment variable (falls back to RAY_ADDRESS)."
119119
),
120120
)
121121
@click.option(
@@ -333,7 +333,7 @@ def submit(
333333
required=False,
334334
help=(
335335
"Address of the Ray cluster to connect to. Can also be specified "
336-
"using the `RAY_ADDRESS` environment variable."
336+
"using the RAY_API_SERVER_ADDRESS environment variable (falls back to RAY_ADDRESS)."
337337
),
338338
)
339339
@click.argument("job-id", type=str)
@@ -363,7 +363,7 @@ def status(
363363
required=False,
364364
help=(
365365
"Address of the Ray cluster to connect to. Can also be specified "
366-
"using the `RAY_ADDRESS` environment variable."
366+
"using the RAY_API_SERVER_ADDRESS environment variable (falls back to RAY_ADDRESS)."
367367
),
368368
)
369369
@click.option(
@@ -418,7 +418,7 @@ def stop(
418418
required=False,
419419
help=(
420420
"Address of the Ray cluster to connect to. Can also be specified "
421-
"using the RAY_ADDRESS environment variable."
421+
"using the RAY_API_SERVER_ADDRESS environment variable (falls back to RAY_ADDRESS)."
422422
),
423423
)
424424
@click.argument("job-id", type=str)
@@ -455,7 +455,7 @@ def delete(
455455
required=False,
456456
help=(
457457
"Address of the Ray cluster to connect to. Can also be specified "
458-
"using the RAY_ADDRESS environment variable."
458+
"using the RAY_API_SERVER_ADDRESS environment variable (falls back to RAY_ADDRESS)."
459459
),
460460
)
461461
@click.argument("job-id", type=str)
@@ -508,7 +508,7 @@ def logs(
508508
required=False,
509509
help=(
510510
"Address of the Ray cluster to connect to. Can also be specified "
511-
"using the RAY_ADDRESS environment variable."
511+
"using the RAY_API_SERVER_ADDRESS environment variable (falls back to RAY_ADDRESS)."
512512
),
513513
)
514514
@add_common_job_options

python/ray/dashboard/modules/job/sdk.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ class JobSubmissionClient(SubmissionClient):
4646
ray.init(), e.g. a Ray Client address (ray://<head_node_host>:10001),
4747
or "auto", or "localhost:<port>". If unspecified, will try to connect to
4848
a running local Ray cluster. This argument is always overridden by the
49-
RAY_ADDRESS environment variable.
49+
RAY_API_SERVER_ADDRESS or RAY_ADDRESS environment variable.
5050
create_cluster_if_needed: Indicates whether the cluster at the specified
5151
address needs to already be running. Ray doesn't start a cluster
5252
before interacting with jobs, but third-party job managers may do so.

python/ray/dashboard/modules/job/tests/test_cli_integration.py

Lines changed: 30 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -142,11 +142,38 @@ def test_empty_ray_address(self, ray_start_stop):
142142
assert "succeeded" in stdout
143143

144144
@pytest.mark.parametrize(
145-
"ray_client_address", ["127.0.0.1:8265", "ray://127.0.0.1:8265"]
145+
"ray_api_server_address,should_fail",
146+
[
147+
("http://127.0.0.1:8265", False), # correct API server
148+
("127.0.0.1:8265", True), # wrong format without http
149+
("http://127.0.0.1:9999", True), # wrong port
150+
],
146151
)
147-
def test_ray_client_address(self, ray_start_stop, ray_client_address: str):
152+
def test_ray_api_server_address(
153+
self,
154+
ray_start_stop,
155+
ray_api_server_address: str,
156+
should_fail: bool,
157+
):
158+
# Set a `RAY_ADDRESS` that would not work with the `ray job submit` CLI because it uses the `ray://` prefix.
159+
# This verifies that the `RAY_API_SERVER_ADDRESS` env var takes precedence.
160+
with set_env_var("RAY_ADDRESS", "ray://127.0.0.1:8265"):
161+
with set_env_var("RAY_API_SERVER_ADDRESS", ray_api_server_address):
162+
_run_cmd("ray job submit -- echo hello", should_fail=should_fail)
163+
164+
@pytest.mark.parametrize(
165+
"ray_client_address,should_fail",
166+
[
167+
("127.0.0.1:8265", True),
168+
("ray://127.0.0.1:8265", True),
169+
("http://127.0.0.1:8265", False),
170+
],
171+
)
172+
def test_ray_client_address(
173+
self, ray_start_stop, ray_client_address: str, should_fail: bool
174+
):
148175
with set_env_var("RAY_ADDRESS", ray_client_address):
149-
_run_cmd("ray job submit -- echo hello", should_fail=True)
176+
_run_cmd("ray job submit -- echo hello", should_fail=should_fail)
150177

151178
def test_valid_http_ray_address(self, ray_start_stop):
152179
stdout, _ = _run_cmd("ray job submit -- echo hello")

python/ray/dashboard/utils.py

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -709,9 +709,15 @@ def get_address_for_submission_client(address: Optional[str]) -> str:
709709
Returns:
710710
API server HTTP URL, e.g. "http://<head-node-ip>:8265".
711711
"""
712-
if os.environ.get("RAY_ADDRESS"):
713-
logger.debug(f"Using RAY_ADDRESS={os.environ['RAY_ADDRESS']}")
714-
address = os.environ["RAY_ADDRESS"]
712+
if api_server_address := os.environ.get(
713+
ray_constants.RAY_API_SERVER_ADDRESS_ENVIRONMENT_VARIABLE
714+
):
715+
address = api_server_address
716+
logger.debug(f"Using RAY_API_SERVER_ADDRESS={address}")
717+
# Fall back to RAY_ADDRESS if RAY_API_SERVER_ADDRESS not set
718+
elif ray_address := os.environ.get(ray_constants.RAY_ADDRESS_ENVIRONMENT_VARIABLE):
719+
address = ray_address
720+
logger.debug(f"Using RAY_ADDRESS={address}")
715721

716722
if address and "://" in address:
717723
module_string, _ = split_address(address)

python/ray/scripts/scripts.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -983,7 +983,7 @@ def start(
983983
cli_logger.print("To submit a Ray job using the Ray Jobs CLI:")
984984
cli_logger.print(
985985
cf.bold(
986-
" RAY_ADDRESS='http://{}' ray job submit "
986+
" RAY_API_SERVER_ADDRESS='http://{}' ray job submit "
987987
"--working-dir . "
988988
"-- python my_script.py"
989989
),

python/ray/tests/test_cli_patterns/test_ray_start.txt

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,7 @@ Next steps
1414
ray\.init\(\)
1515

1616
To submit a Ray job using the Ray Jobs CLI:
17-
RAY_ADDRESS='http://.+:8265' ray job submit --working-dir \. -- python my_script\.py
17+
RAY_API_SERVER_ADDRESS='http://.+:8265' ray job submit --working-dir \. -- python my_script\.py
1818

1919
See https://docs\.ray\.io/en/latest/cluster/running-applications/job-submission/index\.html
2020
for more information on submitting Ray jobs to the Ray cluster.

python/ray/tests/test_cli_patterns/test_ray_start_windows_osx.txt

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@ Next steps
1515
ray\.init\(\)
1616

1717
To submit a Ray job using the Ray Jobs CLI:
18-
RAY_ADDRESS='http://.+:8265' ray job submit --working-dir \. -- python my_script\.py
18+
RAY_API_SERVER_ADDRESS='http://.+:8265' ray job submit --working-dir \. -- python my_script\.py
1919

2020
See https://docs\.ray\.io/en/latest/cluster/running-applications/job-submission/index\.html
2121
for more information on submitting Ray jobs to the Ray cluster.

0 commit comments

Comments
 (0)