Skip to content

Commit

Permalink
fix gcs logging
Browse files Browse the repository at this point in the history
Signed-off-by: hjiang <[email protected]>
  • Loading branch information
dentiny committed Nov 26, 2024
1 parent ed3d48c commit ea19f0a
Show file tree
Hide file tree
Showing 5 changed files with 57 additions and 29 deletions.
4 changes: 0 additions & 4 deletions python/ray/_private/node.py
Original file line number Diff line number Diff line change
Expand Up @@ -1154,14 +1154,10 @@ def start_gcs_server(self):
assert gcs_server_port > 0
assert self._gcs_address is None, "GCS server is already running."
assert self._gcs_client is None, "GCS client is already connected."
# TODO(mwtian): append date time so restarted GCS uses different files.
stdout_file, stderr_file = self.get_log_file_handles("gcs_server", unique=True)
process_info = ray._private.services.start_gcs_server(
self.redis_address,
self._logs_dir,
self.session_name,
stdout_file=stdout_file,
stderr_file=stderr_file,
redis_password=self._ray_params.redis_password,
config=self._config,
fate_share=self.kernel_fate_share,
Expand Down
11 changes: 5 additions & 6 deletions python/ray/_private/services.py
Original file line number Diff line number Diff line change
Expand Up @@ -1443,8 +1443,6 @@ def start_gcs_server(
redis_address: str,
log_dir: str,
session_name: str,
stdout_file: Optional[IO[AnyStr]] = None,
stderr_file: Optional[IO[AnyStr]] = None,
redis_password: Optional[str] = None,
config: Optional[dict] = None,
fate_share: Optional[bool] = None,
Expand All @@ -1458,10 +1456,6 @@ def start_gcs_server(
redis_address: The address that the Redis server is listening on.
log_dir: The path of the dir where log files are created.
session_name: The session name (cluster id) of this cluster.
stdout_file: A file handle opened for writing to redirect stdout to. If
no redirection should happen, then this should be None.
stderr_file: A file handle opened for writing to redirect stderr to. If
no redirection should happen, then this should be None.
redis_password: The password of the redis server.
config: Optional configuration that will
override defaults in RayConfig.
Expand Down Expand Up @@ -1494,6 +1488,11 @@ def start_gcs_server(
]
if redis_password:
command += [f"--redis_password={redis_password}"]

# Logging is fully managed by C++ side spdlog, which supports rotation and file
# count limitation.
stdout_file = open("/dev/null")
stderr_file = open("/dev/null")
process_info = start_ray_process(
command,
ray_constants.PROCESS_TYPE_GCS_SERVER,
Expand Down
25 changes: 22 additions & 3 deletions src/ray/gcs/gcs_server/gcs_server_main.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@
// limitations under the License.

#include <iostream>
#include <limits>
#include <cstdlib>

#include "gflags/gflags.h"
#include "ray/common/ray_config.h"
Expand All @@ -27,6 +29,7 @@ DEFINE_string(redis_address, "", "The ip address of redis.");
DEFINE_bool(redis_enable_ssl, false, "Use tls/ssl in redis connection.");
DEFINE_int32(redis_port, -1, "The port of redis.");
DEFINE_string(log_dir, "", "The path of the dir where log files are created.");
DEFINE_int64(log_rotation_max_size, 0, "Max size to rotate logs for GCS.");
DEFINE_int32(gcs_server_port, 0, "The port of gcs server.");
DEFINE_int32(metrics_agent_port, -1, "The port of metrics agent.");
DEFINE_string(config_list, "", "The config list of raylet.");
Expand All @@ -38,17 +41,33 @@ DEFINE_string(session_name,
"session_name: The session name (ClusterID) of the cluster.");
DEFINE_string(ray_commit, "", "The commit hash of Ray.");

namespace {
// GCS server output filename.
constexpr std::string_view kGcsServerLog = "gcs_server.out";
} // namespace

int main(int argc, char *argv[]) {
gflags::ParseCommandLineFlags(&argc, &argv, true);

// Backward compatibility notes:
// Due to historical reason, GCS server flushes all logging and stdout/stderr to a single file called `gcs_server.out`.
// To keep backward compatibility at best effort, we use the same filename as output, and disable log rotation by default.
const int64_t log_rotation_max_size = FLAGS_log_rotation_size <= 0 ? std::numeric_limits<int64_t>::max() : FLAGS_log_rotation_size;
RAY_CHECK_EQ(setenv("RAY_ROTATION_MAX_BYTES", std::to_string(log_rotation_max_size), /*overwrite=*/1));
const std::string log_file = FLAGS_log_dir.empty() ? kGcsServerLog.data() : absl::StrFormat("%s/%s", FLAGS_log_dir, kGcsServerLog);
// TODO(hjiang): For the current implementation, we assume all logging are managed by spdlog, the caveat is there could be
// there's writing to stdout/stderr as well. The final solution is implement self-customized sink for spdlog, and redirect
// stderr/stdout to the file descritor. Hold until it's confirmed necessary.

InitShutdownRAII ray_log_shutdown_raii(ray::RayLog::StartRayLog,
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"");
/*log_dir=*/"",
/*log_file=*/log_file);
ray::RayLog::InstallFailureSignalHandler(argv[0]);
ray::RayLog::InstallTerminateHandler();

gflags::ParseCommandLineFlags(&argc, &argv, true);

RAY_LOG(INFO)
.WithField("ray_version", kRayVersion)
.WithField("ray_commit", FLAGS_ray_commit)
Expand Down
32 changes: 21 additions & 11 deletions src/ray/util/logging.cc
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@ constexpr char kLogFormatJsonPattern[] =
RayLogLevel RayLog::severity_threshold_ = RayLogLevel::INFO;
std::string RayLog::app_name_ = "";
std::string RayLog::component_name_ = "";
std::string RayLog::log_dir_ = "";
bool RayLog::log_format_json_ = false;
std::string RayLog::log_format_pattern_ = kLogFormatTextPattern;

Expand Down Expand Up @@ -312,14 +311,30 @@ void RayLog::InitLogFormat() {
}
}

/*static*/ std::string RayLog::GetLogOutputFilename(const std::string& log_dir, const std::string& log_file, const std::string& app_name) {
if (!log_file.empty()) {
return log_file;
}
if (!log_dir.empty()) {
#ifdef _WIN32
int pid = _getpid();
#else
pid_t pid = getpid();
#endif

return JoinPaths(log_dir, absl::StrFormat("%s_%d.log", app_name, pid));
}
return "";
}

void RayLog::StartRayLog(const std::string &app_name,
RayLogLevel severity_threshold,
const std::string &log_dir) {
const std::string &log_dir,
const std::string &log_file) {
InitSeverityThreshold(severity_threshold);
InitLogFormat();

app_name_ = app_name;
log_dir_ = log_dir;

// All the logging sinks to add.
// One for file/stdout, another for stderr.
Expand All @@ -337,13 +352,8 @@ void RayLog::StartRayLog(const std::string &app_name,
}
}

if (!log_dir_.empty()) {
// Enable log file if log_dir_ is not empty.
#ifdef _WIN32
int pid = _getpid();
#else
pid_t pid = getpid();
#endif
const auto log_output_fname = GetLogOutputFilename(log_dir, log_file, app_name_without_path);
if (!log_output_fname.empty()) {
// Reset log pattern and level and we assume a log file can be rotated with
// 10 files in max size 512M by default.
if (const char *ray_rotation_max_bytes = std::getenv("RAY_ROTATION_MAX_BYTES");
Expand Down Expand Up @@ -372,7 +382,7 @@ void RayLog::StartRayLog(const std::string &app_name,
spdlog::drop(RayLog::GetLoggerName());
}
auto file_sink = std::make_shared<spdlog::sinks::rotating_file_sink_mt>(
JoinPaths(log_dir_, app_name_without_path + "_" + std::to_string(pid) + ".log"),
log_output_fname,
log_rotation_max_size_,
log_rotation_file_num_);
file_sink->set_level(level);
Expand Down
14 changes: 9 additions & 5 deletions src/ray/util/logging.h
Original file line number Diff line number Diff line change
Expand Up @@ -258,10 +258,14 @@ class RayLog {
///
/// \parem appName The app name which starts the log.
/// \param severity_threshold Logging threshold for the program.
/// \param logDir Logging output file name. If empty, the log won't output to file.
/// \param log_dir Logging output directory name.
/// \param log_file Logging output file name.
/// Both [log_dir] and [log_file] are used to determine log output filename; if both empty, the log won't output to file, but to stdout.
/// If both set, [log_file] has higher priority than [log_dir].
static void StartRayLog(const std::string &appName,
RayLogLevel severity_threshold = RayLogLevel::INFO,
const std::string &logDir = "");
const std::string &log_dir = "",
const std::string &log_file = "");

/// The shutdown function of ray log which should be used with StartRayLog as a pair.
/// If `StartRayLog` wasn't called before, it will be no-op.
Expand Down Expand Up @@ -305,6 +309,9 @@ class RayLog {
static void AddFatalLogCallbacks(
const std::vector<FatalLogCallback> &expose_log_callbacks);

/// Get log outout filename.
static std::string GetLogOutputFilename(const std::string& log_dir, const std::string& log_file, const std::string& app_name);

template <typename T>
RayLog &operator<<(const T &t) {
if (IsEnabled()) {
Expand Down Expand Up @@ -376,9 +383,6 @@ class RayLog {
/// to indicate which component generates the log.
/// This is empty if we log to file.
static std::string component_name_;
/// The directory where the log files are stored.
/// If this is empty, logs are printed to stdout.
static std::string log_dir_;
/// This flag is used to avoid calling UninstallSignalAction in ShutDownRayLog if
/// InstallFailureSignalHandler was not called.
static bool is_failure_signal_handler_installed_;
Expand Down

0 comments on commit ea19f0a

Please sign in to comment.