Skip to content
Merged
Show file tree
Hide file tree
Changes from 87 commits
Commits
Show all changes
98 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
47c8042
move common fixtures to conftest.py
Oct 29, 2025
3d84e62
refactor all authentication tests and reuse common code
Oct 30, 2025
9e0e64f
fix lint
Oct 30, 2025
e3bf7f4
fix build
Oct 30, 2025
a4a09cc
refactor and improve structure
Oct 30, 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
d0564ae
Merge remote-tracking branch 'upstream/token_auth_5' into token_auth_6
Oct 31, 2025
2858ad8
Merge branch 'master' into token_auth_5
sampan-s-nayak Oct 31, 2025
cddc620
Merge remote-tracking branch 'upstream/token_auth_5' into token_auth_6
Oct 31, 2025
8874a22
fix tests
Oct 31, 2025
8da7fa4
fix case sensitivity bug + tests
Oct 31, 2025
91ea16f
Merge branch 'master' into token_auth_6
edoakes Oct 31, 2025
4f32c84
fix
edoakes Oct 31, 2025
3da2df4
Merge branch 'master' into token_auth_6
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
Empty file.
23 changes: 23 additions & 0 deletions python/ray/_private/authentication/authentication_constants.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
# Token setup instructions (used in multiple contexts)
TOKEN_SETUP_INSTRUCTIONS = """Please provide an authentication token using one of these methods:
1. Set the RAY_AUTH_TOKEN environment variable
2. Set the RAY_AUTH_TOKEN_PATH environment variable (pointing to a token file)
3. Create a token file at the default location: ~/.ray/auth_token"""

# When token auth is enabled but no token is found anywhere
TOKEN_AUTH_ENABLED_BUT_NO_TOKEN_FOUND_ERROR_MESSAGE = (
"Token authentication is enabled but no authentication token was found. "
+ TOKEN_SETUP_INSTRUCTIONS
)

# When HTTP request fails with 401 (Unauthorized - missing token)
HTTP_REQUEST_MISSING_TOKEN_ERROR_MESSAGE = (
"The Ray cluster requires authentication, but no token was provided.\n\n"
+ TOKEN_SETUP_INSTRUCTIONS
)

# When HTTP request fails with 403 (Forbidden - invalid token)
HTTP_REQUEST_INVALID_TOKEN_ERROR_MESSAGE = (
"The authentication token you provided is invalid or incorrect.\n\n"
+ TOKEN_SETUP_INSTRUCTIONS
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
import uuid


# TODO: this is a placeholder for the actual authentication token generator. Will be replaced with a proper implementation.
def generate_new_authentication_token() -> str:
return uuid.uuid4().hex
100 changes: 100 additions & 0 deletions python/ray/_private/authentication/authentication_token_setup.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
"""Authentication token setup for Ray.

This module provides functions to generate and save authentication tokens
for Ray's token-based authentication system. Token loading and caching is
handled by the C++ AuthenticationTokenLoader.
"""

import logging
from pathlib import Path
from typing import Any, Dict, Optional

from ray._private.authentication.authentication_constants import (
TOKEN_AUTH_ENABLED_BUT_NO_TOKEN_FOUND_ERROR_MESSAGE,
)
from ray._private.authentication.authentication_token_generator import (
generate_new_authentication_token,
)
from ray._raylet import (
AuthenticationMode,
AuthenticationTokenLoader,
get_authentication_mode,
)

logger = logging.getLogger(__name__)


def generate_and_save_token() -> None:
"""Generate a new random token and save it in the default token path.

Returns:
The newly generated authentication token.
"""
# Generate a UUID-based token
token = generate_new_authentication_token()

token_path = _get_default_token_path()
try:
# Create directory if it doesn't exist
token_path.parent.mkdir(parents=True, exist_ok=True)

# Write token to file with explicit flush and fsync
with open(token_path, "w") as f:
f.write(token)

logger.info(f"Generated new authentication token and saved to {token_path}")
except Exception:
raise


def _get_default_token_path() -> Path:
"""Get the default token file path (~/.ray/auth_token).

Returns:
Path object pointing to ~/.ray/auth_token
"""
return Path.home() / ".ray" / "auth_token"


def ensure_token_if_auth_enabled(
system_config: Optional[Dict[str, Any]] = None, create_token_if_missing: bool = True
) -> None:
"""Check authentication settings and set up token resources if authentication is enabled.

Ray calls this early during ray.init() to do the following for token-based authentication:
1. Check whether you enabled token-based authentication.
2. Make sure a token is available if authentication is enabled.
3. Generate and save a default token for new local clusters if one doesn't already exist.

Args:
system_config: Ray raises an error if you set auth_mode in system_config instead of the environment.
create_token_if_missing: Generate a new token if one doesn't already exist.

Raises:
RuntimeError: Ray raises this error if authentication is enabled but no token is found when connecting
to an existing cluster.
"""

# Check if you enabled token authentication.
if get_authentication_mode() != AuthenticationMode.TOKEN:
if (
system_config
and "auth_mode" in system_config
and system_config["auth_mode"] != "disabled"
):
raise RuntimeError(
"Set authentication mode can only be set with the `RAY_auth_mode` environment variable, not using the system_config."
)
return

token_loader = AuthenticationTokenLoader.instance()

if not token_loader.has_token():
if create_token_if_missing:
# Generate a new token.
generate_and_save_token()

# Reload the cache so subsequent calls to token_loader read the new token.
token_loader.reset_cache()
else:
raise RuntimeError(TOKEN_AUTH_ENABLED_BUT_NO_TOKEN_FOUND_ERROR_MESSAGE)
9 changes: 9 additions & 0 deletions python/ray/_private/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,9 @@
from ray._common import ray_option_utils
from ray._common.constants import RAY_WARN_BLOCKING_GET_INSIDE_ASYNC_ENV_VAR
from ray._common.utils import load_class
from ray._private.authentication.authentication_token_setup import (
ensure_token_if_auth_enabled,
)
from ray._private.client_mode_hook import client_mode_hook
from ray._private.custom_types import TensorTransportEnum
from ray._private.function_manager import FunctionActorManager
Expand Down Expand Up @@ -1865,6 +1868,9 @@ def sigterm_handler(signum, frame):
if bootstrap_address is None:
# In this case, we need to start a new cluster.

# Setup and verify authentication for new cluster
ensure_token_if_auth_enabled(_system_config, create_token_if_missing=True)

# Don't collect usage stats in ray.init() unless it's a nightly wheel.
from ray._common.usage import usage_lib

Expand Down Expand Up @@ -1952,6 +1958,9 @@ def sigterm_handler(signum, frame):
"an existing cluster."
)

# Setup and verify authentication for connecting to existing cluster
ensure_token_if_auth_enabled(_system_config, create_token_if_missing=False)

# In this case, we only need to connect the node.
ray_params = ray._private.parameter.RayParams(
node_ip_address=_node_ip_address,
Expand Down
1 change: 1 addition & 0 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@ include "includes/metric.pxi"
include "includes/setproctitle.pxi"
include "includes/raylet_client.pxi"
include "includes/gcs_subscriber.pxi"
include "includes/rpc_token_authentication.pxi"

import ray
from ray.exceptions import (
Expand Down
31 changes: 31 additions & 0 deletions python/ray/dashboard/authentication_utils.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
from ray._raylet import (
AuthenticationMode,
get_authentication_mode,
validate_authentication_token,
)


def is_token_auth_enabled() -> bool:
"""Check if token authentication is enabled.

Returns:
bool: True if auth_mode is set to "token", False otherwise
"""
return get_authentication_mode() == AuthenticationMode.TOKEN


def validate_request_token(auth_header: str) -> bool:
"""Validate the Authorization header from an HTTP request.

Args:
auth_header: The Authorization header value (e.g., "Bearer <token>")

Returns:
bool: True if token is valid, False otherwise
"""
if not auth_header:
return False

# validate_authentication_token expects full "Bearer <token>" format
# and performs equality comparison via C++ layer
return validate_authentication_token(auth_header)
26 changes: 26 additions & 0 deletions python/ray/dashboard/http_server_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
from ray._common.network_utils import build_address, parse_address
from ray._common.usage.usage_lib import TagKey, record_extra_usage_tag
from ray._common.utils import get_or_create_event_loop
from ray.dashboard import authentication_utils as auth_utils
from ray.dashboard.dashboard_metrics import DashboardPrometheusMetrics
from ray.dashboard.head import DashboardHeadModule

Expand Down Expand Up @@ -163,6 +164,30 @@ def get_address(self):
assert self.http_host and self.http_port
return self.http_host, self.http_port

@aiohttp.web.middleware
async def auth_middleware(self, request, handler):
"""Authenticate requests when token auth is enabled."""

# Skip if auth not enabled
if not auth_utils.is_token_auth_enabled():
return await handler(request)

# Extract and validate token
auth_header = request.headers.get("Authorization", "")

if not auth_header:
return aiohttp.web.Response(
status=401, text="Unauthorized: Missing authentication token"
)

# Validate token
if not auth_utils.validate_request_token(auth_header):
return aiohttp.web.Response(
status=403, text="Forbidden: Invalid authentication token"
)

return await handler(request)

@aiohttp.web.middleware
async def path_clean_middleware(self, request, handler):
if request.path.startswith("/static") or request.path.startswith("/logs"):
Expand Down Expand Up @@ -252,6 +277,7 @@ async def run(
client_max_size=ray_constants.DASHBOARD_CLIENT_MAX_SIZE,
middlewares=[
self.metrics_middleware,
self.auth_middleware,
self.path_clean_middleware,
self.browsers_no_post_put_middleware,
self.cache_control_static_middleware,
Expand Down
46 changes: 43 additions & 3 deletions python/ray/dashboard/modules/dashboard_sdk.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
import yaml

import ray
from ray._private.authentication import authentication_constants
from ray._private.runtime_env.packaging import (
create_package,
get_uri_for_directory,
Expand All @@ -20,7 +21,9 @@
from ray._private.runtime_env.py_modules import upload_py_modules_if_needed
from ray._private.runtime_env.working_dir import upload_working_dir_if_needed
from ray._private.utils import split_address
from ray._raylet import AuthenticationTokenLoader
from ray.autoscaler._private.cli_logger import cli_logger
from ray.dashboard.authentication_utils import is_token_auth_enabled
from ray.dashboard.modules.job.common import uri_to_http_components
from ray.util.annotations import DeveloperAPI, PublicAPI

Expand Down Expand Up @@ -222,7 +225,11 @@ def __init__(
self._default_metadata = cluster_info.metadata or {}
# Headers used for all requests sent to job server, optional and only
# needed for cases like authentication to remote cluster.
self._headers = cluster_info.headers
self._headers = cluster_info.headers or {}

# Add authentication token if token auth is enabled
self._set_auth_header_if_enabled()
Copy link
Collaborator

Choose a reason for hiding this comment

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

In general I prefer to follow a more functional style instead of implicitly modifying members deeper in the call stack. It makes it much easier to read the code and is less error prone. So it'd be something like:

Suggested change
self._headers = cluster_info.headers or {}
# Add authentication token if token auth is enabled
self._set_auth_header_if_enabled()
self._headers = cluster_info.headers or {}
self._headers.update(**self._get_auth_headers())


# Set SSL verify parameter for the requests library and create an ssl_context
# object when needed for the aiohttp library.
self._verify = verify
Expand All @@ -242,6 +249,22 @@ def __init__(
else:
self._ssl_context = None

def _set_auth_header_if_enabled(self):
"""Add authentication token to headers if token auth is enabled."""
if is_token_auth_enabled():
token_loader = AuthenticationTokenLoader.instance()
token_added = token_loader.set_token_for_http_header(self._headers)
Copy link
Collaborator

Choose a reason for hiding this comment

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

same thing here -- better to have the AuthenticationTokenLoader return the headers instead of modify an argument passed by value

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I wanted to avoid having to expose the token directly but this does look ugly (and we are exposing raw token) so let me refactor this


if not token_added:
# Token auth is enabled but no token found or Authorization already set
if "Authorization" not in self._headers:
# No token found - log warning but don't fail yet
# Let the server return 401 for a better error message
logger.warning(
"Token authentication is enabled but no token was found. "
"Requests to authenticated clusters will fail."
)

def _check_connection_and_version(
self, min_version: str = "1.9", version_error_message: str = None
):
Expand Down Expand Up @@ -293,14 +316,15 @@ def _do_request(
json_data: Optional[dict] = None,
**kwargs,
) -> "requests.Response":
"""Perform the actual HTTP request
"""Perform the actual HTTP request with authentication error handling.

Keyword arguments other than "cookies", "headers" are forwarded to the
`requests.request()`.
"""
url = self._address + endpoint
logger.debug(f"Sending request to {url} with json data: {json_data or {}}.")
return requests.request(

response = requests.request(
method,
url,
cookies=self._cookies,
Expand All @@ -311,6 +335,22 @@ def _do_request(
**kwargs,
)

# Check for authentication errors and provide helpful messages
if response.status_code == 401:
# Unauthorized - missing or no token provided
raise RuntimeError(
f"Authentication required: {response.text}\n\n"
+ authentication_constants.HTTP_REQUEST_MISSING_TOKEN_ERROR_MESSAGE
)
elif response.status_code == 403:
# Forbidden - invalid token
raise RuntimeError(
f"Authentication failed: {response.text}\n\n"
+ authentication_constants.HTTP_REQUEST_INVALID_TOKEN_ERROR_MESSAGE
)

return response

def _package_exists(
self,
package_uri: str,
Expand Down
28 changes: 28 additions & 0 deletions python/ray/includes/rpc_token_authentication.pxd
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
from libcpp cimport bool as c_bool
from libcpp.string cimport string
from ray.includes.optional cimport optional

cdef extern from "ray/rpc/authentication/authentication_mode.h" namespace "ray::rpc" nogil:
cdef enum CAuthenticationMode "ray::rpc::AuthenticationMode":
DISABLED "ray::rpc::AuthenticationMode::DISABLED"
TOKEN "ray::rpc::AuthenticationMode::TOKEN"

CAuthenticationMode GetAuthenticationMode()

cdef extern from "ray/rpc/authentication/authentication_token.h" namespace "ray::rpc" nogil:
cdef cppclass CAuthenticationToken "ray::rpc::AuthenticationToken":
CAuthenticationToken()
CAuthenticationToken(string value)
c_bool empty()
c_bool Equals(const CAuthenticationToken& other)
string ToAuthorizationHeaderValue()
@staticmethod
CAuthenticationToken FromMetadata(string metadata_value)

cdef extern from "ray/rpc/authentication/authentication_token_loader.h" namespace "ray::rpc" nogil:
cdef cppclass CAuthenticationTokenLoader "ray::rpc::AuthenticationTokenLoader":
@staticmethod
CAuthenticationTokenLoader& instance()
c_bool HasToken()
void ResetCache()
optional[CAuthenticationToken] GetToken()
Loading