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
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());

// 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