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
2 changes: 2 additions & 0 deletions src/ray/common/constants.h
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@ constexpr int kRayletStoreErrorExitCode = 100;
constexpr char kObjectTablePrefix[] = "ObjectTable";

constexpr char kClusterIdKey[] = "ray_cluster_id";
constexpr char kAuthTokenKey[] = "authorization";
constexpr char kBearerPrefix[] = "Bearer ";

constexpr char kWorkerDynamicOptionPlaceholder[] =
"RAY_WORKER_DYNAMIC_OPTION_PLACEHOLDER";
Expand Down
6 changes: 6 additions & 0 deletions src/ray/common/ray_config_def.h
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,12 @@ RAY_CONFIG(bool, emit_main_service_metrics, true)
/// Whether to enable cluster authentication.
RAY_CONFIG(bool, enable_cluster_auth, true)

/// Whether to enable token-based authentication for RPC calls.
/// will be converted to AuthenticationMode enum defined in
/// rpc/authentication/authentication_mode.h
/// use GetAuthenticationMode() to get the authentication mode enum value.
RAY_CONFIG(std::string, auth_mode, "disabled")

/// The interval of periodic event loop stats print.
/// -1 means the feature is disabled. In this case, stats are available
/// in the associated process's log file.
Expand Down
34 changes: 34 additions & 0 deletions src/ray/rpc/authentication/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
load("//bazel:ray.bzl", "ray_cc_library")

ray_cc_library(
name = "authentication_mode",
srcs = ["authentication_mode.cc"],
hdrs = ["authentication_mode.h"],
visibility = ["//visibility:public"],
deps = [
"//src/ray/common:ray_config",
"@com_google_absl//absl/strings",
],
)

ray_cc_library(
name = "authentication_token",
hdrs = ["authentication_token.h"],
visibility = ["//visibility:public"],
deps = [
"//src/ray/common:constants",
"@com_github_grpc_grpc//:grpc++",
],
)

ray_cc_library(
name = "authentication_token_loader",
srcs = ["authentication_token_loader.cc"],
hdrs = ["authentication_token_loader.h"],
visibility = ["//visibility:public"],
deps = [
":authentication_mode",
":authentication_token",
"//src/ray/util:logging",
],
)
37 changes: 37 additions & 0 deletions src/ray/rpc/authentication/authentication_mode.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// Copyright 2025 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#include "ray/rpc/authentication/authentication_mode.h"

#include <stdexcept>
#include <string>

#include "absl/strings/ascii.h"
#include "ray/common/ray_config.h"

namespace ray {
namespace rpc {

AuthenticationMode GetAuthenticationMode() {
std::string auth_mode_lower = absl::AsciiStrToLower(RayConfig::instance().auth_mode());

if (auth_mode_lower == "token") {
return AuthenticationMode::TOKEN;
} else {
return AuthenticationMode::DISABLED;
}
}

} // namespace rpc
} // namespace ray
33 changes: 33 additions & 0 deletions src/ray/rpc/authentication/authentication_mode.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// Copyright 2025 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#pragma once

#include <string>

namespace ray {
namespace rpc {

enum class AuthenticationMode {
DISABLED,
TOKEN,
};

/// Get the authentication mode from the RayConfig.
/// \return The authentication mode enum value. returns AuthenticationMode::DISABLED if
/// the authentication mode is not set or is invalid.
AuthenticationMode GetAuthenticationMode();

} // namespace rpc
} // namespace ray
156 changes: 156 additions & 0 deletions src/ray/rpc/authentication/authentication_token.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
// Copyright 2025 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#pragma once

#include <grpcpp/grpcpp.h>

#include <cstdint>
#include <cstring>
#include <iostream>
#include <string>
#include <string_view>
#include <utility>
#include <vector>

#include "ray/common/constants.h"

namespace ray {
namespace rpc {

/// Secure wrapper for authentication tokens.
/// - Wipes memory on destruction
/// - Constant-time comparison
/// - Redacted output when logged or printed
class AuthenticationToken {
public:
AuthenticationToken() = default;
explicit AuthenticationToken(std::string value) : secret_(value.begin(), value.end()) {}

AuthenticationToken(const AuthenticationToken &other) : secret_(other.secret_) {}
AuthenticationToken &operator=(const AuthenticationToken &other) {
if (this != &other) {
SecureClear();
secret_ = other.secret_;
}
return *this;
}

// Move operations
AuthenticationToken(AuthenticationToken &&other) noexcept {
MoveFrom(std::move(other));
}
AuthenticationToken &operator=(AuthenticationToken &&other) noexcept {
if (this != &other) {
SecureClear();
MoveFrom(std::move(other));
}
return *this;
}
~AuthenticationToken() { SecureClear(); }

bool empty() const noexcept { return secret_.empty(); }

/// Constant-time equality comparison
bool Equals(const AuthenticationToken &other) const noexcept {
return ConstTimeEqual(secret_, other.secret_);
}

/// Equality operator (constant-time)
bool operator==(const AuthenticationToken &other) const noexcept {
return Equals(other);
}

/// Inequality operator
bool operator!=(const AuthenticationToken &other) const noexcept {
return !(*this == other);
}

/// Set authentication metadata on a gRPC client context
/// Only call this from client-side code
void SetMetadata(grpc::ClientContext &context) const {
if (!secret_.empty()) {
context.AddMetadata(kAuthTokenKey,
kBearerPrefix + std::string(secret_.begin(), secret_.end()));
}
}

/// Create AuthenticationToken from gRPC metadata value
/// Strips "Bearer " prefix and creates token object
/// @param metadata_value The raw value from server metadata (should include "Bearer "
/// prefix)
/// @return AuthenticationToken object (empty if format invalid)
static AuthenticationToken FromMetadata(std::string_view metadata_value) {
const std::string_view prefix(kBearerPrefix);
if (metadata_value.size() < prefix.size() ||
metadata_value.substr(0, prefix.size()) != prefix) {
return AuthenticationToken(); // Invalid format, return empty
}
std::string_view token_part = metadata_value.substr(prefix.size());
return AuthenticationToken(std::string(token_part));
}

friend std::ostream &operator<<(std::ostream &os, const AuthenticationToken &t) {
return os << "<Redacted Authentication Token>";
}

private:
std::vector<uint8_t> secret_;

// Constant-time string comparison to avoid timing attacks.
// https://en.wikipedia.org/wiki/Timing_attack
static bool ConstTimeEqual(const std::vector<uint8_t> &a,
const std::vector<uint8_t> &b) noexcept {
if (a.size() != b.size()) {
return false;
}
unsigned char diff = 0;
for (size_t i = 0; i < a.size(); ++i) {
diff |= a[i] ^ b[i];
}
return diff == 0;
}

// replace the characters in the memory with 0
static void ExplicitBurn(void *p, size_t n) noexcept {
#if defined(_MSC_VER)
SecureZeroMemory(p, n);
#elif defined(__STDC_LIB_EXT1__)
memset_s(p, n, 0, n);
#else
// Using array indexing instead of pointer arithmetic
volatile auto *vp = static_cast<volatile uint8_t *>(p);
for (size_t i = 0; i < n; ++i) {
vp[i] = 0;
}
#endif
}

void SecureClear() noexcept {
if (!secret_.empty()) {
ExplicitBurn(secret_.data(), secret_.size());
secret_.clear();
}
}

void MoveFrom(AuthenticationToken &&other) noexcept {
secret_ = std::move(other.secret_);
// Clear the moved-from object explicitly for security
// Note: 'other' is already an rvalue reference, no need to move again
other.SecureClear();
}
Comment on lines +149 to +152
Copy link
Collaborator

Choose a reason for hiding this comment

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

my only concern here is that this is a bit of an expensive move operation... we should keep it in mind once we run release/performance tests if there is any significant regression.

};

} // namespace rpc
} // namespace ray
Loading