Skip to content

Commit

Permalink
simplify log dir
Browse files Browse the repository at this point in the history
Signed-off-by: dentiny <[email protected]>
  • Loading branch information
dentiny committed Dec 23, 2024
1 parent d49a906 commit 9f36a13
Show file tree
Hide file tree
Showing 19 changed files with 76 additions and 99 deletions.
13 changes: 8 additions & 5 deletions cpp/src/ray/test/api_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -110,15 +110,18 @@ RAY_REMOTE(Counter::FactoryCreate,
&Counter::GetList);

TEST(RayApiTest, LogTest) {
auto log_path = std::filesystem::current_path().string() + "/tmp/";
ray::RayLog::StartRayLog("cpp_worker", ray::RayLogLevel::DEBUG, log_path);
const std::string app_name = "cpp_worker";
const std::string log_dir = std::filesystem::current_path().string() + "/tmp/";
ray::RayLog::StartRayLog(app_name,
ray::RayLogLevel::DEBUG,
ray::RayLog::GetLogFilepathFromDirectory(log_dir, app_name));
std::array<std::string, 3> str_arr{"debug test", "info test", "warning test"};
RAYLOG(DEBUG) << str_arr[0];
RAYLOG(INFO) << str_arr[1];
RAYLOG(WARNING) << str_arr[2];
RAY_CHECK(true);

for (auto &it : std::filesystem::directory_iterator(log_path)) {
for (auto &it : std::filesystem::directory_iterator(log_dir)) {
if (!std::filesystem::is_directory(it)) {
std::ifstream in(it.path().string(), std::ios::binary);
std::string line;
Expand All @@ -129,7 +132,7 @@ TEST(RayApiTest, LogTest) {
}
}

std::filesystem::remove_all(log_path);
std::filesystem::remove_all(log_dir);
}

TEST(RayApiTest, TaskOptionsCheckTest) {
Expand Down Expand Up @@ -361,4 +364,4 @@ TEST(RayApiTest, DefaultActorLifetimeTest) {
ray::internal::ConfigInternal::Instance().Init(config, 2, args2);
EXPECT_EQ(ray::rpc::JobConfig_ActorLifetime_DETACHED,
ray::internal::ConfigInternal::Instance().default_actor_lifetime);
}
}
1 change: 0 additions & 1 deletion python/ray/includes/global_state_accessor.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,6 @@ cdef extern from * namespace "ray::gcs" nogil:
ray::RayLog::ShutDownRayLog,
"ray_init",
ray::RayLogLevel::WARNING,
/*log_dir=*/"" ,
/*log_filepath=*/"",
/*log_rotation_max_size=*/1ULL << 29,
/*log_rotation_file_num=*/10);
Expand Down
3 changes: 1 addition & 2 deletions src/ray/common/test/ray_syncer_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -978,8 +978,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
ray::RayLog::InstallFailureSignalHandler(argv[0]);
Expand Down
17 changes: 9 additions & 8 deletions src/ray/core_worker/core_worker_process.cc
Original file line number Diff line number Diff line change
Expand Up @@ -79,17 +79,18 @@ CoreWorkerProcessImpl::CoreWorkerProcessImpl(const CoreWorkerOptions &options)
? ComputeDriverIdFromJob(options_.job_id)
: WorkerID::FromRandom()) {
if (options_.enable_logging) {
std::stringstream app_name;
app_name << LanguageString(options_.language) << "-core-"
<< WorkerTypeString(options_.worker_type);
std::stringstream app_name_ss;
app_name_ss << LanguageString(options_.language) << "-core-"
<< WorkerTypeString(options_.worker_type);
if (!worker_id_.IsNil()) {
app_name << "-" << worker_id_;
app_name_ss << "-" << worker_id_;
}
// TODO(hjiang): Unify log directory and log filepath.
RayLog::StartRayLog(app_name.str(),
const std::string app_name = app_name_ss.str();
const std::string log_filepath =
RayLog::GetLogFilepathFromDirectory(options_.log_dir, /*app_name=*/app_name);
RayLog::StartRayLog(app_name,
RayLogLevel::INFO,
/*log_dir=*/options_.log_dir,
/*ray_log_filepath=*/"",
log_filepath,
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
if (options_.install_failure_signal_handler) {
Expand Down
3 changes: 1 addition & 2 deletions src/ray/core_worker/test/actor_creator_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
ray::RayLog::InstallFailureSignalHandler(argv[0]);
Expand Down
3 changes: 1 addition & 2 deletions src/ray/core_worker/test/direct_actor_transport_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -981,8 +981,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
ray::RayLog::InstallFailureSignalHandler(argv[0]);
Expand Down
3 changes: 1 addition & 2 deletions src/ray/gcs/gcs_client/test/gcs_client_reconnection_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -383,8 +383,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
::testing::InitGoogleTest(&argc, argv);
Expand Down
5 changes: 2 additions & 3 deletions src/ray/gcs/gcs_client/test/gcs_client_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1086,10 +1086,9 @@ int main(int argc, char **argv) {
InitShutdownRAII ray_log_shutdown_raii(
ray::RayLog::StartRayLog,
ray::RayLog::ShutDownRayLog,
argv[0],
/*app_name=*/argv[0]
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
::testing::InitGoogleTest(&argc, argv);
Expand Down
3 changes: 1 addition & 2 deletions src/ray/gcs/gcs_client/test/global_state_accessor_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -347,8 +347,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
::testing::InitGoogleTest(&argc, argv);
Expand Down
1 change: 0 additions & 1 deletion src/ray/gcs/gcs_server/gcs_server_main.cc
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@ int main(int argc, char *argv[]) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/FLAGS_ray_log_filepath,
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
Expand Down
3 changes: 1 addition & 2 deletions src/ray/gcs/store_client/test/redis_store_client_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -374,8 +374,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
::testing::InitGoogleTest(&argc, argv);
Expand Down
3 changes: 1 addition & 2 deletions src/ray/gcs/test/redis_async_context_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -86,8 +86,7 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
::testing::InitGoogleTest(&argc, argv);
Expand Down
3 changes: 1 addition & 2 deletions src/ray/raylet/main.cc
Original file line number Diff line number Diff line change
Expand Up @@ -140,9 +140,8 @@ int main(int argc, char *argv[]) {
InitShutdownRAII ray_log_shutdown_raii(
ray::RayLog::StartRayLog,
ray::RayLog::ShutDownRayLog,
argv[0],
/*app_name=*/argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*ray_log_filepath=*/FLAGS_ray_log_filepath,
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
Expand Down
3 changes: 1 addition & 2 deletions src/ray/raylet/worker_pool_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2229,8 +2229,7 @@ int main(int argc, char **argv) {
[]() { ray::RayLog::ShutDownRayLog(); },
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
::testing::InitGoogleTest(&argc, argv);
Expand Down
54 changes: 19 additions & 35 deletions src/ray/util/logging.cc
Original file line number Diff line number Diff line change
Expand Up @@ -309,24 +309,6 @@ 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 "";
}

/*static*/ size_t RayLog::GetRayLogRotationMaxBytesOrDefault() {
if (const char *ray_rotation_max_bytes = std::getenv("RAY_ROTATION_MAX_BYTES");
ray_rotation_max_bytes != nullptr) {
Expand All @@ -351,20 +333,25 @@ void RayLog::InitLogFormat() {
return kDefaultLogRotationFileNum;
}

void RayLog::StartRayLog(const std::string &app_name,
RayLogLevel severity_threshold,
const std::string &log_dir,
const std::string &log_filepath,
size_t log_rotation_max_size,
size_t log_rotation_file_num) {
// TODO(hjiang): As a temporary workaround decide output log filename on [log_dir] or
// [log_filepath]. But they cannot be non empty at the same time. Cleanup
// `log_dir`.
const bool log_dir_empty = log_dir.empty();
const bool log_filepath_empty = log_filepath.empty();
RAY_CHECK(log_dir_empty || log_filepath_empty)
<< "Log directory and log filename cannot be set at the same time.";
/*static*/ std::string RayLog::GetLogFilepathFromDirectory(const std::string &log_dir,
const std::string &app_name) {
if (log_dir.empty()) {
return "";
}

#ifdef _WIN32
int pid = _getpid();
#else
pid_t pid = getpid();
#endif
return JoinPaths(log_dir, absl::StrFormat("%s_%d.log", app_name, pid));
}

/*static*/ void RayLog::StartRayLog(const std::string &app_name,
RayLogLevel severity_threshold,
const std::string &log_filepath,
size_t log_rotation_max_size,
size_t log_rotation_file_num) {
InitSeverityThreshold(severity_threshold);
InitLogFormat();

Expand All @@ -388,10 +375,7 @@ void RayLog::StartRayLog(const std::string &app_name,
}
}

// Reset log pattern and level and we assume a log file can be rotated with
// 10 files in max size 512M by default.
const auto log_fname =
GetLogOutputFilename(log_dir, log_filepath, app_name_without_path);
const auto log_fname = log_filepath;

// Set sink for stdout.
if (!log_fname.empty()) {
Expand Down
27 changes: 9 additions & 18 deletions src/ray/util/logging.h
Original file line number Diff line number Diff line change
Expand Up @@ -261,24 +261,20 @@ class RayLog {
/// This function to judge whether current log is fatal or not.
bool IsFatal() const;

/// TODO(hjiang): Having both log directory and filename for `StartRayLog` is confusing,
/// at the end of the day should have only filename. Fix all usage in the next PR.
///
/// Get filepath to dump log from [log_dir] and [app_name].
/// If [log_dir] empty, return empty filepath.
static std::string GetLogFilepathFromDirectory(const std::string &log_dir,
const std::string &app_name);

/// The init function of ray log for a program which should be called only once.
///
/// \parem appName The app name which starts the log.
/// \param severity_threshold Logging threshold for the program.
/// \param log_dir Logging output directory name.
/// \param log_filepath Logging output filepath.
/// \param log_rotation_max_size max bytes for of log rotation.
/// \param log_rotation_file_num max number of rotating log files.
///
/// Both [log_dir] and [log_filepath] are used to determine log output filename; if
/// both empty, the log won't output to file, but to stdout. It's illegal to set
/// [log_filepath] and [log_dir] at the same time.
static void StartRayLog(const std::string &appName,
/// \param log_filepath Logging output filepath. If empty, the log won't output to file,
/// but to stdout. \param log_rotation_max_size max bytes for of log rotation. \param
/// log_rotation_file_num max number of rotating log files.
static void StartRayLog(const std::string &app_name,
RayLogLevel severity_threshold = RayLogLevel::INFO,
const std::string &log_dir = "",
const std::string &log_filepath = "",
size_t log_rotation_max_size = kDefaultLogRotationMaxSize,
size_t log_rotation_file_num = kDefaultLogRotationFileNum);
Expand Down Expand Up @@ -333,11 +329,6 @@ 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
11 changes: 8 additions & 3 deletions src/ray/util/tests/event_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -650,7 +650,9 @@ TEST_F(EventTest, TestLogLevel) {
TEST_F(EventTest, TestLogEvent) {
ray::RayEvent::SetEmitToLogFile(true);
// Initialize log level to error
ray::RayLog::StartRayLog("event_test", ray::RayLogLevel::ERROR, log_dir);
const std::string app_name = "event_test";
const std::string log_filepath = RayLog::GetLogFilepathFromDirectory(log_dir, app_name);
ray::RayLog::StartRayLog(app_name, ray::RayLogLevel::ERROR, log_filepath);
EventManager::Instance().AddReporter(std::make_shared<TestEventReporter>());
RayEventContext::Instance().SetEventContext(
rpc::Event_SourceType::Event_SourceType_CORE_WORKER, {});
Expand Down Expand Up @@ -678,7 +680,7 @@ TEST_F(EventTest, TestLogEvent) {
std::filesystem::remove_all(log_dir.c_str());

// Set log level smaller than event level.
ray::RayLog::StartRayLog("event_test", ray::RayLogLevel::INFO, log_dir);
ray::RayLog::StartRayLog(app_name, ray::RayLogLevel::INFO, log_filepath);
ray::RayEvent::SetLevel("error");

// Add some events. All events would be printed in general log.
Expand Down Expand Up @@ -737,6 +739,9 @@ TEST_F(EventTest, VerifyOnlyNthOccurenceEventLogged) {
int main(int argc, char **argv) {
::testing::InitGoogleTest(&argc, argv);
// Use ERROR type logger by default to avoid printing large scale logs in current test.
ray::RayLog::StartRayLog("event_test", ray::RayLogLevel::ERROR);
const std::string app_name = "event_test";
const std::string log_filepath =
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", app_name);
ray::RayLog::StartRayLog(app_name, ray::RayLogLevel::INFO, log_filepath);
return RUN_ALL_TESTS();
}
15 changes: 10 additions & 5 deletions src/ray/util/tests/logging_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -189,7 +189,7 @@ TEST(PrintLogTest, TestRayLogEveryMs) {

TEST(PrintLogTest, TestTextLogging) {
setEnv("RAY_BACKEND_LOG_JSON", "0");
RayLog::StartRayLog("/tmp/gcs", RayLogLevel::INFO, "");
RayLog::StartRayLog("/tmp/gcs", RayLogLevel::INFO, /*log_filepath=*/"");
CaptureStdout();
RAY_LOG(INFO).WithField("key1", "value1").WithField("key2", "value2")
<< "contextual log";
Expand All @@ -206,7 +206,7 @@ TEST(PrintLogTest, TestTextLogging) {

TEST(PrintLogTest, TestJSONLogging) {
setEnv("RAY_BACKEND_LOG_JSON", "1");
RayLog::StartRayLog("/tmp/raylet", RayLogLevel::INFO, "");
RayLog::StartRayLog("/tmp/raylet", RayLogLevel::INFO, /*log_filepath=*/"");
CaptureStdout();
RAY_LOG(DEBUG) << "this is not logged";
RAY_LOG(INFO) << "this is info logged";
Expand Down Expand Up @@ -243,15 +243,20 @@ TEST(PrintLogTest, TestJSONLogging) {

TEST(PrintLogTest, LogTestWithInit) {
// Test empty app name.
RayLog::StartRayLog("", RayLogLevel::DEBUG, ray::GetUserTempDir());
const std::string log_dir = ray::GetUserTempDir();
const std::string log_filepath =
RayLog::GetLogFilepathFromDirectory(log_dir, /*app_name=*/"");
RayLog::StartRayLog(/*app_name=*/"", RayLogLevel::DEBUG, log_filepath);
PrintLog();
RayLog::ShutDownRayLog();
}

// This test will output large amount of logs to stderr, should be disabled in travis.
TEST(LogPerfTest, PerfTest) {
RayLog::StartRayLog(
"/fake/path/to/appdire/LogPerfTest", RayLogLevel::ERROR, ray::GetUserTempDir());
const std::string app_name = "/fake/path/to/appdire/LogPerfTest";
const std::string log_dir = ray::GetUserTempDir();
const std::string log_filepath = RayLog::GetLogFilepathFromDirectory(log_dir, app_name);
RayLog::StartRayLog(app_name, RayLogLevel::ERROR, log_filepath);
int rounds = 10;

int64_t start_time = current_time_ms();
Expand Down
4 changes: 2 additions & 2 deletions src/ray/util/tests/signal_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -102,8 +102,8 @@ int main(int argc, char **argv) {
ray::RayLog::ShutDownRayLog,
argv[0],
ray::RayLogLevel::INFO,
/*log_dir=*/"",
/*log_filepath=*/"",
/*log_filepath=*/
ray::RayLog::GetLogFilepathFromDirectory(/*log_dir=*/"", /*app_name=*/argv[0]),
ray::RayLog::GetRayLogRotationMaxBytesOrDefault(),
ray::RayLog::GetRayLogRotationBackupCountOrDefault());
ray::RayLog::InstallFailureSignalHandler(argv[0]);
Expand Down

0 comments on commit 9f36a13

Please sign in to comment.