Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
4 changes: 2 additions & 2 deletions python/ray/includes/rpc_token_authentication.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,9 @@ cdef extern from "ray/rpc/authentication/authentication_token_loader.h" namespac
cdef cppclass CAuthenticationTokenLoader "ray::rpc::AuthenticationTokenLoader":
@staticmethod
CAuthenticationTokenLoader& instance()
c_bool HasToken()
c_bool HasToken(c_bool ignore_auth_mode)
void ResetCache()
optional[CAuthenticationToken] GetToken()
optional[CAuthenticationToken] GetToken(c_bool ignore_auth_mode)

cdef extern from "ray/rpc/authentication/authentication_token_validator.h" namespace "ray::rpc" nogil:
cdef cppclass CAuthenticationTokenValidator "ray::rpc::AuthenticationTokenValidator":
Expand Down
35 changes: 26 additions & 9 deletions python/ray/includes/rpc_token_authentication.pxi
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ def validate_authentication_token(provided_token: str) -> bool:
cdef CAuthenticationToken provided

if get_authentication_mode() == CAuthenticationMode.TOKEN:
expected_opt = CAuthenticationTokenLoader.instance().GetToken()
expected_opt = CAuthenticationTokenLoader.instance().GetToken(False)
if not expected_opt.has_value():
return False

Expand All @@ -64,13 +64,17 @@ class AuthenticationTokenLoader:
"""Get the singleton instance (returns a wrapper for convenience)."""
return AuthenticationTokenLoader()

def has_token(self):
def has_token(self, ignore_auth_mode=False):
"""Check if an authentication token exists without crashing.

Args:
ignore_auth_mode: If True, bypass auth mode check and attempt to load token
regardless of RAY_AUTH_MODE setting.

Returns:
bool: True if a token exists, False otherwise
"""
return CAuthenticationTokenLoader.instance().HasToken()
return CAuthenticationTokenLoader.instance().HasToken(ignore_auth_mode)

def reset_cache(self):
"""Reset the C++ authentication token cache.
Expand All @@ -80,7 +84,7 @@ class AuthenticationTokenLoader:
"""
CAuthenticationTokenLoader.instance().ResetCache()

def get_token_for_http_header(self) -> dict:
def get_token_for_http_header(self, ignore_auth_mode=False) -> dict:
"""Get authentication token as a dictionary for HTTP headers.

This method loads the token from C++ AuthenticationTokenLoader and returns it
Expand All @@ -89,26 +93,39 @@ class AuthenticationTokenLoader:
- A token does not exist
- The token is empty

Args:
ignore_auth_mode: If True, bypass auth mode check and attempt to load token
regardless of RAY_AUTH_MODE setting.

Returns:
dict: Empty dict or {"authorization": "Bearer <token>"}
"""
if not self.has_token():
if not self.has_token(ignore_auth_mode):
return {}

# Get the token from C++ layer
cdef optional[CAuthenticationToken] token_opt = CAuthenticationTokenLoader.instance().GetToken()
cdef optional[CAuthenticationToken] token_opt = CAuthenticationTokenLoader.instance().GetToken(ignore_auth_mode)

if not token_opt.has_value() or token_opt.value().empty():
return {}

return {AUTHORIZATION_HEADER_NAME: token_opt.value().ToAuthorizationHeaderValue().decode('utf-8')}

def get_raw_token(self) -> str:
if not self.has_token():
def get_raw_token(self, ignore_auth_mode=False) -> str:
"""Get the raw authentication token value.

Args:
ignore_auth_mode: If True, bypass auth mode check and attempt to load token
regardless of RAY_AUTH_MODE setting.

Returns:
str: The raw token string, or empty string if no token exists
"""
if not self.has_token(ignore_auth_mode):
return ""

# Get the token from C++ layer
cdef optional[CAuthenticationToken] token_opt = CAuthenticationTokenLoader.instance().GetToken()
cdef optional[CAuthenticationToken] token_opt = CAuthenticationTokenLoader.instance().GetToken(ignore_auth_mode)

if not token_opt.has_value() or token_opt.value().empty():
return ""
Expand Down
16 changes: 4 additions & 12 deletions python/ray/scripts/scripts.py
Original file line number Diff line number Diff line change
Expand Up @@ -2723,29 +2723,21 @@ def shutdown_prometheus():
help="Generate a new token if none exists",
)
def get_auth_token(generate):
"""Prints the Ray authentication token to stdout when RAY_AUTH_MODE=token.
"""Prints the Ray authentication token to stdout.

If --generate is specified, a new token is created and saved to ~/.ray/auth_token if one does not exist.
"""
from ray._private.authentication.authentication_token_setup import (
generate_and_save_token,
)
from ray._raylet import (
AuthenticationMode,
AuthenticationTokenLoader,
get_authentication_mode,
)

# Check if token auth mode is enabled and provide guidance if not
if get_authentication_mode() != AuthenticationMode.TOKEN:
raise click.ClickException(
"Token authentication is not currently enabled. To enable token authentication, set: export RAY_AUTH_MODE=token\n For more instructions, see: https://docs.ray.io/en/latest/ray-security/auth.html",
)

# Try to load existing token
loader = AuthenticationTokenLoader.instance()

if not loader.has_token():
if not loader.has_token(ignore_auth_mode=True):
if generate:
click.echo("Generating new authentication token...", err=True)
generate_and_save_token()
Expand All @@ -2755,8 +2747,8 @@ def get_auth_token(generate):
"No authentication token found. Use ray `get-auth-token --generate` to create one.",
)

# Get raw token value
token = loader.get_raw_token()
# Get raw token value (ignore auth mode - explicitly loading token)
token = loader.get_raw_token(ignore_auth_mode=True)

# Print token to stdout (for piping) without newline
click.echo(token, nl=False)
Expand Down
3 changes: 0 additions & 3 deletions python/ray/tests/test_token_auth_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -425,7 +425,6 @@ def test_get_auth_token_cli(use_generate):
test_token = "a" * 64

with authentication_env_guard():
set_auth_mode("token")
if use_generate:
# Test --generate flag (no token set)
clear_auth_token_sources(remove_default=True)
Expand Down Expand Up @@ -473,7 +472,6 @@ def test_get_auth_token_cli(use_generate):
def test_get_auth_token_cli_no_token_no_generate():
"""Test ray get-auth-token fails without token and without --generate."""
with authentication_env_guard():
set_auth_mode("token")
reset_auth_token_state()
clear_auth_token_sources(remove_default=True)
env = os.environ.copy()
Expand All @@ -500,7 +498,6 @@ def test_get_auth_token_cli_piping():
test_token = "b" * 64

with authentication_env_guard():
set_auth_mode("token")
set_env_auth_token(test_token)
reset_auth_token_state()
env = os.environ.copy()
Expand Down
19 changes: 10 additions & 9 deletions src/ray/rpc/authentication/authentication_token_loader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -40,25 +40,26 @@ AuthenticationTokenLoader &AuthenticationTokenLoader::instance() {
return instance;
}

std::optional<AuthenticationToken> AuthenticationTokenLoader::GetToken() {
std::optional<AuthenticationToken> AuthenticationTokenLoader::GetToken(
bool ignore_auth_mode) {
std::lock_guard<std::mutex> lock(token_mutex_);

// If already loaded, return cached value
if (cached_token_.has_value()) {
return cached_token_;
}

// If token or k8s auth is not enabled, return std::nullopt
if (!RequiresTokenAuthentication()) {
// If token or k8s auth is not enabled, return std::nullopt (unless ignoring auth mode)
if (!ignore_auth_mode && !RequiresTokenAuthentication()) {
cached_token_ = std::nullopt;
return std::nullopt;
}

// Token auth is enabled, try to load from sources
// Token auth is enabled (or we're ignoring auth mode), try to load from sources
AuthenticationToken token = LoadTokenFromSources();

// If no token found and auth is enabled, fail with RAY_CHECK
if (token.empty()) {
if (token.empty() && !ignore_auth_mode) {
RAY_LOG(FATAL)
<< "Token authentication is enabled but Ray couldn't find an "
"authentication token. "
Expand All @@ -72,21 +73,21 @@ std::optional<AuthenticationToken> AuthenticationTokenLoader::GetToken() {
return *cached_token_;
}

bool AuthenticationTokenLoader::HasToken() {
bool AuthenticationTokenLoader::HasToken(bool ignore_auth_mode) {
std::lock_guard<std::mutex> lock(token_mutex_);

// If already loaded, check if it's a valid token
if (cached_token_.has_value()) {
return !cached_token_->empty();
}

// If token or k8s auth is not enabled, no token needed
if (!RequiresTokenAuthentication()) {
// If token or k8s auth is not enabled, no token needed (unless ignoring auth mode)
if (!ignore_auth_mode && !RequiresTokenAuthentication()) {
cached_token_ = std::nullopt;
return false;
}

// Token auth is enabled, try to load from sources
// Token auth is enabled (or we're ignoring auth mode), try to load from sources
AuthenticationToken token = LoadTokenFromSources();

// Cache the result
Expand Down
8 changes: 6 additions & 2 deletions src/ray/rpc/authentication/authentication_token_loader.h
Original file line number Diff line number Diff line change
Expand Up @@ -37,13 +37,17 @@ class AuthenticationTokenLoader {

/// Get the authentication token.
/// If token authentication is enabled but no token is found, fails with RAY_CHECK.
/// \param ignore_auth_mode If true, bypass auth mode check and attempt to load token
/// regardless of RAY_AUTH_MODE setting.
/// \return The authentication token, or std::nullopt if auth is disabled.
std::optional<AuthenticationToken> GetToken();
std::optional<AuthenticationToken> GetToken(bool ignore_auth_mode = false);

/// Check if a token exists without crashing.
/// Caches the token if it loads it afresh.
/// \param ignore_auth_mode If true, bypass auth mode check and attempt to load token
/// regardless of RAY_AUTH_MODE setting.
/// \return true if a token exists, false otherwise.
bool HasToken();
bool HasToken(bool ignore_auth_mode = false);

void ResetCache() {
std::lock_guard<std::mutex> lock(token_mutex_);
Expand Down
47 changes: 47 additions & 0 deletions src/ray/rpc/authentication/tests/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
load("//bazel:ray.bzl", "ray_cc_test")

ray_cc_test(
name = "authentication_token_test",
size = "small",
srcs = [
"authentication_token_test.cc",
],
tags = ["team:core"],
deps = [
"//src/ray/rpc/authentication:authentication_token",
"@com_google_googletest//:gtest_main",
],
)

ray_cc_test(
name = "authentication_token_loader_test",
size = "small",
srcs = [
"authentication_token_loader_test.cc",
],
tags = ["team:core"],
deps = [
"//src/ray/common:ray_config",
"//src/ray/rpc/authentication:authentication_token_loader",
"//src/ray/util:env",
"@com_google_googletest//:gtest_main",
],
)

ray_cc_test(
name = "grpc_auth_token_tests",
size = "small",
srcs = [
"grpc_auth_token_tests.cc",
],
tags = ["team:core"],
deps = [
"//src/ray/protobuf:test_service_cc_grpc",
"//src/ray/rpc:grpc_client",
"//src/ray/rpc:grpc_server",
"//src/ray/rpc/authentication:authentication_token_loader",
"//src/ray/rpc/tests:grpc_test_common",
"//src/ray/util:env",
"@com_google_googletest//:gtest_main",
],
)
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,62 @@ TEST_F(AuthenticationTokenLoaderTest, TestWhitespaceHandling) {
EXPECT_TRUE(token_opt->Equals(expected));
}

TEST_F(AuthenticationTokenLoaderTest, TestIgnoreAuthModeGetToken) {
// Disable auth mode
RayConfig::instance().initialize(R"({"AUTH_MODE": "disabled"})");
AuthenticationTokenLoader::instance().ResetCache();

// Set token in environment
set_env_var("RAY_AUTH_TOKEN", "test-token-ignore-auth");

auto &loader = AuthenticationTokenLoader::instance();

// Without ignore_auth_mode, should return nullopt (auth is disabled)
auto token_opt_no_ignore = loader.GetToken();
EXPECT_FALSE(token_opt_no_ignore.has_value());

// Reset cache to test ignore_auth_mode
loader.ResetCache();

// With ignore_auth_mode=true, should load token despite auth being disabled
auto token_opt_ignore = loader.GetToken(true);
ASSERT_TRUE(token_opt_ignore.has_value());
AuthenticationToken expected("test-token-ignore-auth");
EXPECT_TRUE(token_opt_ignore->Equals(expected));

// Re-enable auth for other tests
RayConfig::instance().initialize(R"({"AUTH_MODE": "token"})");
}

TEST_F(AuthenticationTokenLoaderTest, TestIgnoreAuthModeHasToken) {
// Disable auth mode
RayConfig::instance().initialize(R"({"AUTH_MODE": "disabled"})");
AuthenticationTokenLoader::instance().ResetCache();

// Set token in environment
set_env_var("RAY_AUTH_TOKEN", "test-token-has-ignore");

auto &loader = AuthenticationTokenLoader::instance();

// Without ignore_auth_mode, should return false (auth is disabled)
EXPECT_FALSE(loader.HasToken(false));

// Reset cache to test ignore_auth_mode
loader.ResetCache();

// With ignore_auth_mode=true, should return true despite auth being disabled
EXPECT_TRUE(loader.HasToken(true));

// Also verify we can get the actual token value
auto token_opt = loader.GetToken(true);
ASSERT_TRUE(token_opt.has_value());
AuthenticationToken expected("test-token-has-ignore");
EXPECT_TRUE(token_opt->Equals(expected));

// Re-enable auth for other tests
RayConfig::instance().initialize(R"({"AUTH_MODE": "token"})");
}

} // namespace rpc
} // namespace ray

Expand Down
Loading