Skip to content
Merged
Show file tree
Hide file tree
Changes from 84 commits
Commits
Show all changes
86 commits
Select commit Hold shift + click to select a range
021df65
[Core] Authentication for ray core rpc calls - part 1
Oct 16, 2025
c96d1f4
[Core] Token auth improvements - C++ RayAuthTokenLoader singleton
Oct 17, 2025
91f783e
[Core] Token auth improvements - Python token loader and CLI
Oct 17, 2025
54d4eac
[Core][Tests] Add unit tests for RayAuthTokenLoader
Oct 17, 2025
092f29e
[Core][Tests] Add unit tests for Python auth_token_loader
Oct 17, 2025
fcd1d10
fix lint errors
Oct 17, 2025
411f6f4
missed change
Oct 17, 2025
40fcdb5
more lint issues
Oct 17, 2025
223dbf5
fix library
Oct 17, 2025
cc89a63
more lint
Oct 17, 2025
c079298
move python side changes to new pr
Oct 17, 2025
34aa7a3
remove unused import
Oct 17, 2025
7bde811
Merge remote-tracking branch 'upstream/grpc_auth_1' into grpc_auth_2
Oct 17, 2025
733efca
remove generate token method from c++ code
Oct 17, 2025
16fd74e
fix lint
Oct 17, 2025
7094efd
refactor code files
Oct 17, 2025
f56d5ee
fix lint
Oct 17, 2025
356a38e
fix lint
Oct 17, 2025
899973e
add missing imports
Oct 17, 2025
47f2e5a
refactor token loader and tests
Oct 17, 2025
d6a87e2
refactor token loader + fix build
Oct 21, 2025
e579741
fix lint
Oct 21, 2025
99b7c22
Merge branch 'grpc_auth_1' into grpc_auth_2
sampan-s-nayak Oct 21, 2025
8678815
fix issues + update tests
Oct 22, 2025
4274544
missed change
Oct 22, 2025
4a5dda9
fix lint
Oct 22, 2025
b20e1ef
address comments - version 1
Oct 22, 2025
d1fe7b9
fix lint
Oct 22, 2025
09359d6
missing imports
Oct 22, 2025
886c109
fix lint
Oct 22, 2025
1a0b53b
fix build + refactor
Oct 23, 2025
bec39b8
Merge remote-tracking branch 'upstream/master' into grpc_auth_1
Oct 23, 2025
d5d711b
address cursor comments
Oct 23, 2025
78c9cf4
split grpc client server tests
Oct 23, 2025
123914e
fix lint
Oct 23, 2025
56fb190
fix imports
Oct 23, 2025
2976396
Merge remote-tracking branch 'upstream/grpc_auth_1' into grpc_auth_2
Oct 23, 2025
0433a16
refactor and simplify changes
Oct 23, 2025
b6c667a
fix lint
Oct 23, 2025
52d18ac
fix doc string
Oct 23, 2025
b119fae
add type hints
Oct 23, 2025
b6b7a95
lint
Oct 23, 2025
ce73705
Add authentication token logic and related tests
Oct 23, 2025
341b108
Add gRPC service and server logic with auth integration tests
Oct 23, 2025
c821c21
revert unneeded changs from src/ray/rpc/tests/BUILD.bazel
Oct 23, 2025
a14dc69
readd dependencies
Oct 23, 2025
1f59706
Merge branch 'token_auth_2' into grpc_auth_2
sampan-s-nayak Oct 23, 2025
e340d07
fix build issues
Oct 23, 2025
7834733
Merge branch 'master' into token_auth_1
sampan-s-nayak Oct 24, 2025
4801ed7
address comments + fix build
Oct 24, 2025
65c3ded
Merge branch 'token_auth_1' into token_auth_2
sampan-s-nayak Oct 24, 2025
d24f23c
address comments
Oct 24, 2025
7b9edf1
Merge branch 'token_auth_2' into grpc_auth_2
sampan-s-nayak Oct 24, 2025
d801db6
Merge branch 'master' into token_auth_2
edoakes Oct 24, 2025
e9cc57f
address comments
Oct 26, 2025
f8c08e0
fix lint
Oct 26, 2025
b128e4e
Merge remote-tracking branch 'upstream/token_auth_2' into token_auth_2
Oct 26, 2025
c8cff1d
Merge branch 'master' into token_auth_2
sampan-s-nayak Oct 26, 2025
a7a8efa
fix ci
Oct 26, 2025
5a91771
Merge remote-tracking branch 'upstream/token_auth_2' into token_auth_2
Oct 26, 2025
5910ecf
fix build.bazel and imports
Oct 27, 2025
d36e22f
fix lint
Oct 27, 2025
4063d74
fix lint issues
Oct 27, 2025
358582a
Merge branch 'token_auth_2' into grpc_auth_2
sampan-s-nayak Oct 27, 2025
9537a00
address comments
Oct 27, 2025
0e6f59b
[Core] Token auth support in Dashboard head
Oct 27, 2025
e343d54
fix lint
Oct 27, 2025
4f41e50
Merge branch 'master' into grpc_auth_2
sampan-s-nayak Oct 28, 2025
c6215d1
Merge branch 'grpc_auth_2' into token_auth_4
sampan-s-nayak Oct 28, 2025
ce6e6e2
Merge branch 'master' into grpc_auth_2
sampan-s-nayak Oct 28, 2025
94c5cc6
fix tests
Oct 28, 2025
92b3f2e
Merge branch 'master' into grpc_auth_2
sampan-s-nayak Oct 28, 2025
934e8d7
Merge branch 'grpc_auth_2' into token_auth_4
sampan-s-nayak Oct 28, 2025
e34a8bd
fix tests and address comments
Oct 28, 2025
94cdc35
Merge remote-tracking branch 'upstream/token_auth_4' into token_auth_4
Oct 28, 2025
cb00933
add test to bazel
Oct 28, 2025
e39247d
fix typo
Oct 28, 2025
bf4866a
attempt to fix tests
Oct 28, 2025
61646af
attempt to fix test in CI
Oct 29, 2025
5b3cc5b
fix lint
Oct 29, 2025
3f40f21
Merge branch 'grpc_auth_2' into token_auth_4
sampan-s-nayak Oct 29, 2025
c0c2e05
[Core] Verify token presence when using ray start CLI
Oct 29, 2025
ce2b56d
Merge branch 'master' of https://github.com/ray-project/ray into toke…
edoakes Oct 30, 2025
842b21d
Fix
edoakes Oct 30, 2025
95ea5d2
remove "test_dashboard_auth.py" from BUILD.bazel as the test has been…
Oct 31, 2025
2858ad8
Merge branch 'master' into token_auth_5
sampan-s-nayak Oct 31, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions python/ray/scripts/scripts.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,9 @@
from ray._common.network_utils import build_address, parse_address
from ray._common.usage import usage_lib
from ray._common.utils import load_class
from ray._private.authentication.authentication_token_setup import (
ensure_token_if_auth_enabled,
)
from ray._private.internal_api import memory_summary
from ray._private.label_utils import (
parse_node_labels_from_yaml_file,
Expand Down Expand Up @@ -937,6 +940,9 @@ def start(
" flag of `ray start` command."
)

# Ensure auth token is available if authentication mode is token
ensure_token_if_auth_enabled(system_config, create_token_if_missing=False)

node = ray._private.node.Node(
ray_params, head=True, shutdown_at_exit=block, spawn_reaper=block
)
Expand Down Expand Up @@ -1094,6 +1100,9 @@ def start(

cli_logger.labeled_value("Local node IP", ray_params.node_ip_address)

# Ensure auth token is available if authentication mode is token
ensure_token_if_auth_enabled(system_config, create_token_if_missing=False)

node = ray._private.node.Node(
ray_params, head=False, shutdown_at_exit=block, spawn_reaper=block
)
Expand Down
1 change: 1 addition & 0 deletions python/ray/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -543,6 +543,7 @@ py_test_module_list(
"test_concurrency_group.py",
"test_core_worker_io_thread_stack_size.py",
"test_cross_language.py",
"test_dashboard_auth.py",
"test_debug_tools.py",
"test_distributed_sort.py",
"test_environ.py",
Expand Down
220 changes: 220 additions & 0 deletions python/ray/tests/test_token_auth_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,12 +2,15 @@

import os
import shutil
import subprocess
import sys
from pathlib import Path
from typing import Optional

import pytest

import ray
from ray._private.test_utils import wait_for_condition
from ray._raylet import AuthenticationTokenLoader, Config
from ray.cluster_utils import Cluster

Expand All @@ -16,6 +19,70 @@ def reset_token_cache():
AuthenticationTokenLoader.instance().reset_cache()


def _run_ray_start_and_verify_status(
args: list, env: dict, expect_success: bool = True, timeout: int = 30
) -> subprocess.CompletedProcess:
"""Helper to run ray start command with proper error handling."""
result = subprocess.run(
["ray", "start"] + args,
env=env,
capture_output=True,
text=True,
timeout=timeout,
)

if expect_success:
assert result.returncode == 0, (
f"ray start should have succeeded. "
f"stdout: {result.stdout}, stderr: {result.stderr}"
)
else:
assert result.returncode != 0, (
f"ray start should have failed but succeeded. "
f"stdout: {result.stdout}, stderr: {result.stderr}"
)
# Check that error message mentions token
error_output = result.stdout + result.stderr
assert (
"authentication token" in error_output.lower()
or "token" in error_output.lower()
), f"Error message should mention token. Got: {error_output}"

return result


def _cleanup_ray_start(env: Optional[dict] = None):
"""Helper to clean up ray start processes."""
# Ensure any ray.init() connection is closed first
if ray.is_initialized():
ray.shutdown()

# Stop with a longer timeout
subprocess.run(
["ray", "stop", "--force"],
env=env,
capture_output=True,
timeout=60, # Increased timeout for flaky cleanup
check=False, # Don't raise on non-zero exit
)

# Wait for ray processes to actually stop
def ray_stopped():
result = subprocess.run(
["ray", "status"],
capture_output=True,
check=False,
)
# ray status returns non-zero when no cluster is running
return result.returncode != 0

try:
wait_for_condition(ray_stopped, timeout=10, retry_interval_ms=500)
except Exception:
# Best effort - don't fail the test if we can't verify it stopped
pass


@pytest.fixture(autouse=True)
def clean_token_sources():
"""Clean up all token sources before and after each test."""
Expand Down Expand Up @@ -78,6 +145,14 @@ def clean_token_sources():
if ray.is_initialized():
ray.shutdown()

# Ensure all ray processes are stopped
subprocess.run(
["ray", "stop", "--force"],
capture_output=True,
timeout=60,
check=False,
)

# Reset token caches again after test
reset_token_cache()
Config.initialize("")
Expand Down Expand Up @@ -219,5 +294,150 @@ def test_func():
cluster.shutdown()


@pytest.mark.parametrize("is_head", [True, False])
def test_ray_start_without_token_raises_error(is_head):
"""Test that ray start fails when auth_mode=token but no token exists."""
# Set up environment with token auth enabled but no token
env = os.environ.copy()
env["RAY_auth_mode"] = "token"
env.pop("RAY_AUTH_TOKEN", None)
env.pop("RAY_AUTH_TOKEN_PATH", None)

# Ensure no default token file exists (already cleaned by fixture)
default_token_path = Path.home() / ".ray" / "auth_token"
assert not default_token_path.exists()

# When specifying an address, we need a head node to connect to
cluster = None
if not is_head:
# Start head node with token
cluster_token = "a" * 32
os.environ["RAY_AUTH_TOKEN"] = cluster_token
os.environ["RAY_auth_mode"] = "token"
Config.initialize("")
cluster = Cluster()
cluster.add_node()

try:
# Prepare arguments
if is_head:
args = ["--head", "--port=0"]
else:
args = [f"--address={cluster.address}"]

# Try to start node - should fail
_run_ray_start_and_verify_status(args, env, expect_success=False)

finally:
if cluster:
cluster.shutdown()


def test_ray_start_head_with_token_succeeds():
"""Test that ray start --head succeeds when token auth is enabled with a valid token."""
# Set up environment with token auth and a valid token
test_token = "a" * 32
env = os.environ.copy()
env["RAY_AUTH_TOKEN"] = test_token
env["RAY_auth_mode"] = "token"

try:
# Start head node - should succeed
_run_ray_start_and_verify_status(
["--head", "--port=0"], env, expect_success=True
)

# Verify we can connect to the cluster with ray.init()
os.environ["RAY_AUTH_TOKEN"] = test_token
os.environ["RAY_auth_mode"] = "token"
Config.initialize("")
reset_token_cache()

# Wait for cluster to be ready
def cluster_ready():
try:
ray.init(address="auto")
return True
except Exception:
return False

wait_for_condition(cluster_ready, timeout=10)
assert ray.is_initialized()

# Test basic operations work
@ray.remote
def test_func():
return "success"

result = ray.get(test_func.remote())
assert result == "success"

finally:
# Cleanup handles ray.shutdown() internally
_cleanup_ray_start(env)


@pytest.mark.parametrize("token_match", ["correct", "incorrect"])
def test_ray_start_address_with_token(token_match):
"""Test ray start --address=... with correct or incorrect token."""
# Start a head node with token auth
cluster_token = "a" * 32
os.environ["RAY_AUTH_TOKEN"] = cluster_token
os.environ["RAY_auth_mode"] = "token"
Config.initialize("")

cluster = Cluster()
cluster.add_node(num_cpus=1)

try:
# Set up environment for worker
env = os.environ.copy()
env["RAY_auth_mode"] = "token"

if token_match == "correct":
env["RAY_AUTH_TOKEN"] = cluster_token
expect_success = True
else:
# Use different token
env["RAY_AUTH_TOKEN"] = "b" * 32
expect_success = False

# Start worker node
_run_ray_start_and_verify_status(
[f"--address={cluster.address}", "--num-cpus=1"],
env,
expect_success=expect_success,
)

if token_match == "correct":
try:
# Connect and verify the cluster has 2 nodes (head + worker)
ray.init(address=cluster.address)

# Wait for worker node to register
def worker_joined():
return len(ray.nodes()) >= 2

wait_for_condition(worker_joined, timeout=10)

nodes = ray.nodes()
assert (
len(nodes) >= 2
), f"Expected at least 2 nodes, got {len(nodes)}: {nodes}"

finally:
# Always shutdown ray.init() connection before cleanup
if ray.is_initialized():
ray.shutdown()
# Clean up the worker node started with ray start
_cleanup_ray_start(env)

finally:
# Clean up cluster
if ray.is_initialized():
ray.shutdown()
cluster.shutdown()


if __name__ == "__main__":
sys.exit(pytest.main(["-vv", __file__]))