Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
5 changes: 1 addition & 4 deletions doc/source/ray-security/token-auth.md
Original file line number Diff line number Diff line change
Expand Up @@ -93,10 +93,7 @@ $ export RAY_AUTH_MODE=token
# First attempt - an error is raised if no token exists.
$ ray start --head
...
RuntimeError: Token authentication is enabled but no authentication token was found. 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
ray.exceptions.AuthenticationError: Token authentication is enabled but no authentication token was found. Ensure that the token for the cluster is available in a local file (e.g., ~/.ray/auth_token or via RAY_AUTH_TOKEN_PATH) or as the `RAY_AUTH_TOKEN` environment variable. To generate a token for local development, use `ray get-auth-token --generate` For remote clusters, ensure that the token is propagated to all nodes of the cluster when token authentication is enabled. For more information, see: https://docs.ray.io/en/latest/ray-security/auth.html

# Generate a token.
$ ray get-auth-token --generate
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
# Authentication error messages
TOKEN_AUTH_ENABLED_BUT_NO_TOKEN_FOUND_ERROR_MESSAGE = (
"Token authentication is enabled but no authentication token was found."
"Token authentication is enabled but no authentication token was found"
)

TOKEN_INVALID_ERROR_MESSAGE = "Token authentication is enabled but the authentication token is invalid or incorrect." # noqa: E501
Expand Down
19 changes: 19 additions & 0 deletions python/ray/_private/authentication/authentication_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -41,3 +41,22 @@ def validate_request_token(auth_header: str) -> bool:
# validate_authentication_token expects full "Bearer <token>" format
# and performs equality comparison via C++ layer
return validate_authentication_token(auth_header)


def get_authentication_mode_name(mode: AuthenticationMode) -> str:
"""Convert AuthenticationMode enum value to string name.

Args:
mode: AuthenticationMode enum value from ray._raylet

Returns:
String name: "disabled", "token", or "k8s"
"""
from ray._raylet import AuthenticationMode

_MODE_NAMES = {
AuthenticationMode.DISABLED: "disabled",
AuthenticationMode.TOKEN: "token",
AuthenticationMode.K8S: "k8s",
}
return _MODE_NAMES.get(mode, "unknown")
9 changes: 2 additions & 7 deletions python/ray/dashboard/http_server_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
from ray._private.authentication.http_token_authentication import (
get_token_auth_middleware,
)
from ray._raylet import AuthenticationMode, get_authentication_mode
from ray._raylet import get_authentication_mode
from ray.dashboard.dashboard_metrics import DashboardPrometheusMetrics
from ray.dashboard.head import DashboardHeadModule

Expand Down Expand Up @@ -171,12 +171,7 @@ async def get_timezone(self, req) -> aiohttp.web.Response:
async def get_authentication_mode(self, req) -> aiohttp.web.Response:
try:
mode = get_authentication_mode()
if mode == AuthenticationMode.TOKEN:
mode_str = "token"
elif mode == AuthenticationMode.K8S:
mode_str = "k8s"
else:
mode_str = "disabled"
mode_str = auth_utils.get_authentication_mode_name(mode)

response = aiohttp.web.json_response({"authentication_mode": mode_str})

Expand Down
23 changes: 21 additions & 2 deletions python/ray/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -495,10 +495,29 @@ def __init__(self, message: str):
super().__init__(message)

def __str__(self) -> str:
return self.message + (
". Ensure that you have `RAY_AUTH_MODE=token` set and the token for the cluster is available as the `RAY_AUTH_TOKEN` environment variable or a local file. "
# Check if RAY_AUTH_MODE is set to token and add a heads-up if not
auth_mode_note = ""

from ray._private.authentication.authentication_utils import (
get_authentication_mode_name,
)
from ray._raylet import AuthenticationMode, get_authentication_mode

current_mode = get_authentication_mode()
if current_mode != AuthenticationMode.TOKEN:
mode_name = get_authentication_mode_name(current_mode)
auth_mode_note = (
f" Note: RAY_AUTH_MODE is currently '{mode_name}' (not 'token')."
)

help_text = (
" Ensure that the token for the cluster is available in a local file (e.g., ~/.ray/auth_token or via "
"RAY_AUTH_TOKEN_PATH) or as the `RAY_AUTH_TOKEN` environment variable. "
"To generate a token for local development, use `ray get-auth-token --generate` "
"For remote clusters, ensure that the token is propagated to all nodes of the cluster when token authentication is enabled. "
"For more information, see: https://docs.ray.io/en/latest/ray-security/auth.html"
)
return self.message + "." + auth_mode_note + help_text


@DeveloperAPI
Expand Down
7 changes: 1 addition & 6 deletions python/ray/scripts/scripts.py
Original file line number Diff line number Diff line change
Expand Up @@ -942,12 +942,7 @@ def start(
)

# Ensure auth token is available if authentication mode is token
try:
ensure_token_if_auth_enabled(system_config, create_token_if_missing=False)
except ray.exceptions.AuthenticationError:
raise RuntimeError(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@edoakes I removed this as this was throwing a nested stack trace:

ray start --head
Usage stats collection is enabled. To disable this, add `--disable-usage-stats` to the command that starts the cluster, or run the following command: `ray disable-usage-stats` before starting the cluster. See https://docs.ray.io/en/master/cluster/usage-stats.html for more details.

Local node IP: 172.31.5.49
Traceback (most recent call last):
  File "/home/ubuntu/clone/ray/python/ray/scripts/scripts.py", line 946, in start
    ensure_token_if_auth_enabled(system_config, create_token_if_missing=False)
  File "/home/ubuntu/clone/ray/python/ray/_private/authentication/authentication_token_setup.py", line 101, in ensure_token_if_auth_enabled
    raise AuthenticationError(
ray.exceptions.AuthenticationError: Token authentication is enabled but no authentication token was found.. Ensure that you have `RAY_AUTH_MODE=token` set and the token for the cluster is available as the `RAY_AUTH_TOKEN` environment variable or a local file. For more information, see: https://docs.ray.io/en/latest/ray-security/auth.html

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
  File "/home/ubuntu/.conda/envs/ray-dev/bin/ray", line 7, in <module>
    sys.exit(main())
  File "/home/ubuntu/clone/ray/python/ray/scripts/scripts.py", line 2822, in main
    return cli()
  File "/home/ubuntu/.conda/envs/ray-dev/lib/python3.10/site-packages/click/core.py", line 1442, in __call__
    return self.main(*args, **kwargs)
  File "/home/ubuntu/.conda/envs/ray-dev/lib/python3.10/site-packages/click/core.py", line 1363, in main
    rv = self.invoke(ctx)
  File "/home/ubuntu/.conda/envs/ray-dev/lib/python3.10/site-packages/click/core.py", line 1830, in invoke
    return _process_result(sub_ctx.command.invoke(sub_ctx))
  File "/home/ubuntu/.conda/envs/ray-dev/lib/python3.10/site-packages/click/core.py", line 1226, in invoke
    return ctx.invoke(self.callback, **ctx.params)
  File "/home/ubuntu/.conda/envs/ray-dev/lib/python3.10/site-packages/click/core.py", line 794, in invoke
    return callback(*args, **kwargs)
  File "/home/ubuntu/clone/ray/python/ray/autoscaler/_private/cli_logger.py", line 823, in wrapper
    return f(*args, **kwargs)
  File "/home/ubuntu/clone/ray/python/ray/scripts/scripts.py", line 948, in start
    raise RuntimeError(
RuntimeError: Failed to load authentication token. To generate a token for local development, use `ray get-auth-token --generate`. For remote clusters, ensure that the token is propagated to all nodes of the cluster when token authentication is enabled.

I feel exposing the AuthenticationError to users may be better than catching and throwing a new runtime exception. please let me know your thoughts

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 have included the message from the runtime error in the AuthenticationError so the message will still be quite helpful

"Failed to load authentication token. To generate a token for local development, use `ray get-auth-token --generate`. For remote clusters, ensure that the token is propagated to all nodes of the cluster when token authentication is enabled."
)
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
41 changes: 40 additions & 1 deletion python/ray/tests/test_exceptions.py
Original file line number Diff line number Diff line change
@@ -1,29 +1,68 @@
"""Tests for Ray exceptions."""
import sys
from enum import Enum
from unittest.mock import MagicMock, patch

import pytest

from ray.exceptions import AuthenticationError, RayError


class FakeAuthMode(Enum):
DISABLED = 0
TOKEN = 1
K8S = 2


class TestAuthenticationError:
"""Tests for AuthenticationError exception."""

auth_doc_url = "https://docs.ray.io/en/latest/ray-security/auth.html"

def test_basic_creation(self):
"""Test basic AuthenticationError creation."""
"""Test basic AuthenticationError creation and message format."""
error = AuthenticationError("Token is missing")
error_str = str(error)

# Original message preserved
assert "Token is missing" in error_str
# Doc URL included
assert self.auth_doc_url in error_str

def test_is_ray_error_subclass(self):
"""Test that AuthenticationError is a RayError subclass."""
error = AuthenticationError("Test")
assert isinstance(error, RayError)

@pytest.mark.parametrize(
"auth_mode,expected_note",
[
(FakeAuthMode.DISABLED, "RAY_AUTH_MODE is currently 'disabled'"),
(FakeAuthMode.K8S, "RAY_AUTH_MODE is currently 'k8s'"),
(FakeAuthMode.TOKEN, None),
],
ids=["disabled", "k8s", "token"],
)
def test_auth_mode_note_in_message(self, auth_mode, expected_note):
"""Test that error message includes auth mode note when not in token mode."""
with patch.dict(
"sys.modules",
{
"ray._raylet": MagicMock(
AuthenticationMode=FakeAuthMode,
get_authentication_mode=lambda: auth_mode,
)
},
):
error = AuthenticationError("Token is missing")
error_str = str(error)

assert "Token is missing" in error_str
if expected_note:
assert expected_note in error_str
else:
assert "RAY_AUTH_MODE is currently" not in error_str


if __name__ == "__main__":
sys.exit(pytest.main(["-v", __file__]))
56 changes: 52 additions & 4 deletions python/ray/tests/test_token_auth_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,34 @@ def test_connect_without_token_raises_error(setup_cluster_with_token_auth):
ray.init(address=cluster.address)


@pytest.mark.parametrize(
"token,expected_status",
[
(None, 401), # No token -> Unauthorized
("wrong_token", 403), # Wrong token -> Forbidden
],
ids=["no_token", "wrong_token"],
)
def test_state_api_auth_failure(token, expected_status, setup_cluster_with_token_auth):
"""Test that state API calls fail with missing or incorrect token."""
import requests

cluster_info = setup_cluster_with_token_auth
dashboard_url = cluster_info["dashboard_url"]

# Make direct HTTP request to state API endpoint
headers = {}
if token is not None:
headers["Authorization"] = f"Bearer {token}"

response = requests.get(f"{dashboard_url}/api/v0/actors", headers=headers)

assert response.status_code == expected_status, (
f"State API should return {expected_status}, got {response.status_code}: "
f"{response.text}"
)


@pytest.mark.parametrize("tokens_match", [True, False])
def test_cluster_token_authentication(tokens_match, setup_cluster_with_token_auth):
"""Test cluster authentication with matching and non-matching tokens."""
Expand Down Expand Up @@ -363,9 +391,10 @@ def test_e2e_operations_with_token_auth(setup_cluster_with_token_auth):
"""Test that e2e operations work with token authentication enabled.

This verifies that with token auth enabled:
1. Job submission works
2. Tasks execute successfully
3. Actors can be created and called
1. Tasks execute successfully
2. Actors can be created and called
3. State API works (list_nodes, list_actors, list_tasks)
4. Job submission works
"""
cluster_info = setup_cluster_with_token_auth

Expand All @@ -391,7 +420,26 @@ def increment(self):
result = ray.get(actor.increment.remote())
assert result == 1, f"Actor method should return 1, got {result}"

# Test 3: Submit a job and wait for completion
# Test 3: State API operations (uses HTTP with auth headers)
from ray.util.state import list_actors, list_nodes, list_tasks

# List nodes - should include at least the head node
nodes = list_nodes()
assert len(nodes) >= 1, f"Expected at least 1 node, got {len(nodes)}"

# List actors - should include our SimpleActor
actors = list_actors()
assert len(actors) >= 1, f"Expected at least 1 actor, got {len(actors)}"
actor_classes = [a.class_name for a in actors]
assert (
"SimpleActor" in actor_classes[0]
), f"SimpleActor not found in {actor_classes}"

# List tasks - should include completed tasks
tasks = list_tasks()
assert len(tasks) >= 1, f"Expected at least 1 task, got {len(tasks)}"

# Test 4: Submit a job and wait for completion
from ray.job_submission import JobSubmissionClient

# Create job submission client (uses HTTP with auth headers)
Expand Down
17 changes: 9 additions & 8 deletions src/ray/rpc/authentication/authentication_token_loader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,9 @@ std::optional<AuthenticationToken> AuthenticationTokenLoader::GetToken(
RAY_LOG(FATAL)
<< "Token authentication is enabled but Ray couldn't find an "
"authentication token. "
<< "Set the RAY_AUTH_TOKEN environment variable, or set RAY_AUTH_TOKEN_PATH to "
"point to a file with the token, "
"or create a token file at ~/.ray/auth_token.";
<< "Create a token file at ~/.ray/auth_token, "
"or store the token in any file and set RAY_AUTH_TOKEN_PATH to point to it, "
"or set the RAY_AUTH_TOKEN environment variable.";
}

// Cache and return the loaded token
Expand Down Expand Up @@ -119,8 +119,8 @@ AuthenticationToken AuthenticationTokenLoader::LoadTokenFromSources() {
if (env_token != nullptr) {
std::string token_str(env_token);
if (!token_str.empty()) {
RAY_LOG(DEBUG) << "Loaded authentication token from RAY_AUTH_TOKEN environment "
"variable";
RAY_LOG(INFO) << "Loaded authentication token from RAY_AUTH_TOKEN environment "
"variable";
return AuthenticationToken(TrimWhitespace(token_str));
}
}
Expand All @@ -136,7 +136,8 @@ AuthenticationToken AuthenticationTokenLoader::LoadTokenFromSources() {
"but file cannot be opened or is empty: "
<< path_str;
}
RAY_LOG(INFO) << "Loaded authentication token from file: " << path_str;
RAY_LOG(INFO) << "Loaded authentication token from file (RAY_AUTH_TOKEN_PATH): "
<< path_str;
return AuthenticationToken(token_str);
}
}
Expand All @@ -145,7 +146,7 @@ AuthenticationToken AuthenticationTokenLoader::LoadTokenFromSources() {
if (GetAuthenticationMode() == AuthenticationMode::K8S) {
std::string token_str = TrimWhitespace(ReadTokenFromFile(k8s::kK8sSaTokenPath));
if (!token_str.empty()) {
RAY_LOG(DEBUG)
RAY_LOG(INFO)
<< "Loaded authentication token from Kubernetes service account path: "
<< k8s::kK8sSaTokenPath;
return AuthenticationToken(token_str);
Expand All @@ -158,7 +159,7 @@ AuthenticationToken AuthenticationTokenLoader::LoadTokenFromSources() {
std::string default_path = GetDefaultTokenPath();
std::string token_str = TrimWhitespace(ReadTokenFromFile(default_path));
if (!token_str.empty()) {
RAY_LOG(DEBUG) << "Loaded authentication token from default path: " << default_path;
RAY_LOG(INFO) << "Loaded authentication token from default path: " << default_path;
return AuthenticationToken(token_str);
}

Expand Down