mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Cleanup info logs in raylet (#17015)
This commit is contained in:
parent
a8b57c78d6
commit
7a1e8fdb8b
5 changed files with 24 additions and 24 deletions
|
@ -161,7 +161,7 @@ bool LocalObjectManager::SpillObjectsOfSize(int64_t num_bytes_to_spill) {
|
|||
SpillObjectsInternal(objects_to_spill, [this, bytes_to_spill, objects_to_spill,
|
||||
start_time](const Status &status) {
|
||||
if (!status.ok()) {
|
||||
RAY_LOG(INFO) << "Failed to spill objects: " << status.ToString();
|
||||
RAY_LOG(DEBUG) << "Failed to spill objects: " << status.ToString();
|
||||
} else {
|
||||
auto now = absl::GetCurrentTimeNanos();
|
||||
RAY_LOG(DEBUG) << "Spilled " << bytes_to_spill << " bytes in "
|
||||
|
@ -283,9 +283,9 @@ void LocalObjectManager::UnpinSpilledObjectCallback(
|
|||
std::shared_ptr<size_t> num_remaining,
|
||||
std::function<void(const ray::Status &)> callback, ray::Status status) {
|
||||
if (!status.ok()) {
|
||||
RAY_LOG(INFO) << "Failed to send spilled url for object " << object_id
|
||||
<< " to object directory, considering the object to have been freed: "
|
||||
<< status.ToString();
|
||||
RAY_LOG(DEBUG) << "Failed to send spilled url for object " << object_id
|
||||
<< " to object directory, considering the object to have been freed: "
|
||||
<< status.ToString();
|
||||
} else {
|
||||
RAY_LOG(DEBUG) << "Object " << object_id << " spilled to " << object_url
|
||||
<< " and object directory has been informed";
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
#include <cctype>
|
||||
#include <fstream>
|
||||
#include <memory>
|
||||
|
||||
#include "boost/filesystem.hpp"
|
||||
#include "boost/system/error_code.hpp"
|
||||
#include "ray/common/asio/asio_util.h"
|
||||
|
@ -1675,8 +1676,8 @@ void NodeManager::HandleCancelResourceReserve(
|
|||
const rpc::CancelResourceReserveRequest &request,
|
||||
rpc::CancelResourceReserveReply *reply, rpc::SendReplyCallback send_reply_callback) {
|
||||
auto bundle_spec = BundleSpecification(request.bundle_spec());
|
||||
RAY_LOG(INFO) << "Request to cancel reserved resource is received, "
|
||||
<< bundle_spec.DebugString();
|
||||
RAY_LOG(DEBUG) << "Request to cancel reserved resource is received, "
|
||||
<< bundle_spec.DebugString();
|
||||
|
||||
// Kill all workers that are currently associated with the placement group.
|
||||
// NOTE: We can't traverse directly with `leased_workers_`, because `DestroyWorker` will
|
||||
|
@ -1790,7 +1791,7 @@ void NodeManager::MarkObjectsAsFailed(
|
|||
status = store_client_.Seal(object_id);
|
||||
}
|
||||
if (!status.ok() && !status.IsObjectExists()) {
|
||||
RAY_LOG(INFO) << "Marking plasma object failed " << object_id;
|
||||
RAY_LOG(DEBUG) << "Marking plasma object failed " << object_id;
|
||||
// If we failed to save the error code, log a warning and push an error message
|
||||
// to the driver.
|
||||
std::ostringstream stream;
|
||||
|
|
|
@ -51,8 +51,8 @@ bool NewPlacementGroupResourceManager::PrepareBundle(
|
|||
if (iter->second->state_ == CommitState::COMMITTED) {
|
||||
// If the bundle state is already committed, it means that prepare request is just
|
||||
// stale.
|
||||
RAY_LOG(INFO) << "Duplicate prepare bundle request, skip it directly. This should "
|
||||
"only happen when GCS restarts.";
|
||||
RAY_LOG(DEBUG) << "Duplicate prepare bundle request, skip it directly. This should "
|
||||
"only happen when GCS restarts.";
|
||||
return true;
|
||||
} else {
|
||||
// If there was a bundle in prepare state, it already locked resources, we will
|
||||
|
@ -92,7 +92,7 @@ void NewPlacementGroupResourceManager::CommitBundle(
|
|||
} else {
|
||||
// Ignore request If the bundle state is already committed.
|
||||
if (it->second->state_ == CommitState::COMMITTED) {
|
||||
RAY_LOG(INFO) << "Duplicate committ bundle request, skip it directly.";
|
||||
RAY_LOG(DEBUG) << "Duplicate committ bundle request, skip it directly.";
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -119,7 +119,7 @@ void NewPlacementGroupResourceManager::ReturnBundle(
|
|||
const BundleSpecification &bundle_spec) {
|
||||
auto it = pg_bundles_.find(bundle_spec.BundleId());
|
||||
if (it == pg_bundles_.end()) {
|
||||
RAY_LOG(INFO) << "Duplicate cancel request, skip it directly.";
|
||||
RAY_LOG(DEBUG) << "Duplicate cancel request, skip it directly.";
|
||||
return;
|
||||
}
|
||||
const auto &bundle_state = it->second;
|
||||
|
|
|
@ -1,8 +1,9 @@
|
|||
#include "ray/raylet/scheduling/cluster_task_manager.h"
|
||||
|
||||
#include <google/protobuf/map.h>
|
||||
|
||||
#include <boost/range/join.hpp>
|
||||
|
||||
#include "ray/raylet/scheduling/cluster_task_manager.h"
|
||||
#include "ray/stats/stats.h"
|
||||
#include "ray/util/logging.h"
|
||||
|
||||
|
@ -172,11 +173,11 @@ void ClusterTaskManager::DispatchScheduledTasksToWorkers(
|
|||
// The task's args cannot be pinned due to lack of memory. We should
|
||||
// retry dispatching the task once another task finishes and releases
|
||||
// its arguments.
|
||||
RAY_LOG(INFO) << "Dispatching task " << task_id
|
||||
<< " would put this node over the max memory allowed for "
|
||||
"arguments of executing tasks ("
|
||||
<< max_pinned_task_arguments_bytes_
|
||||
<< "). Waiting to dispatch task until other tasks complete";
|
||||
RAY_LOG(DEBUG) << "Dispatching task " << task_id
|
||||
<< " would put this node over the max memory allowed for "
|
||||
"arguments of executing tasks ("
|
||||
<< max_pinned_task_arguments_bytes_
|
||||
<< "). Waiting to dispatch task until other tasks complete";
|
||||
RAY_CHECK(!executing_task_args_.empty() && !pinned_task_arguments_.empty())
|
||||
<< "Cannot dispatch task " << task_id
|
||||
<< " until another task finishes and releases its arguments, but no other "
|
||||
|
@ -349,7 +350,7 @@ bool ClusterTaskManager::PinTaskArgsIfMemoryAvailable(const TaskSpecification &s
|
|||
// This can happen if the task's arguments were all local at some
|
||||
// point, but then at least one was evicted before the task could
|
||||
// be dispatched to a worker.
|
||||
RAY_LOG(INFO)
|
||||
RAY_LOG(DEBUG)
|
||||
<< "Task " << spec.TaskId() << " argument " << deps[i]
|
||||
<< " was evicted before the task could be dispatched. This can happen "
|
||||
"when there are many objects needed on this node. The task will be "
|
||||
|
@ -409,8 +410,8 @@ void ClusterTaskManager::PinTaskArgs(const TaskSpecification &spec,
|
|||
it->second.second++;
|
||||
}
|
||||
} else {
|
||||
RAY_LOG(INFO) << "Scheduler received duplicate task " << spec.TaskId()
|
||||
<< ", most likely because the first execution failed";
|
||||
RAY_LOG(DEBUG) << "Scheduler received duplicate task " << spec.TaskId()
|
||||
<< ", most likely because the first execution failed";
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -931,8 +932,8 @@ void ClusterTaskManager::Spillback(const NodeID &spillback_to, const Work &work)
|
|||
|
||||
if (!cluster_resource_scheduler_->AllocateRemoteTaskResources(
|
||||
spillback_to.Binary(), task_spec.GetRequiredResources().GetResourceMap())) {
|
||||
RAY_LOG(INFO) << "Tried to allocate resources for request " << task_spec.TaskId()
|
||||
<< " on a remote node that are no longer available";
|
||||
RAY_LOG(DEBUG) << "Tried to allocate resources for request " << task_spec.TaskId()
|
||||
<< " on a remote node that are no longer available";
|
||||
}
|
||||
|
||||
auto node_info_opt = get_node_info_(spillback_to);
|
||||
|
|
|
@ -1001,7 +1001,6 @@ TEST_F(WorkerPoolTest, TestWorkerCapping) {
|
|||
|
||||
// Start two IO workers. These don't count towards the limit.
|
||||
{
|
||||
RAY_LOG(INFO) << "XXX";
|
||||
Process proc = worker_pool_->StartWorkerProcess(
|
||||
Language::PYTHON, rpc::WorkerType::SPILL_WORKER, job_id);
|
||||
auto worker = CreateSpillWorker(Process());
|
||||
|
@ -1012,7 +1011,6 @@ TEST_F(WorkerPoolTest, TestWorkerCapping) {
|
|||
worker_pool_->PushSpillWorker(worker);
|
||||
}
|
||||
{
|
||||
RAY_LOG(INFO) << "YYY";
|
||||
Process proc = worker_pool_->StartWorkerProcess(
|
||||
Language::PYTHON, rpc::WorkerType::RESTORE_WORKER, job_id);
|
||||
auto worker = CreateRestoreWorker(Process());
|
||||
|
|
Loading…
Add table
Reference in a new issue