Skip to content

Commit

Permalink
cluster: downgrade per-partition messages to DEBUG
Browse files Browse the repository at this point in the history
This is to sequester the most verbose INFO messages
(those reporting per-partition events) in certain
subsystems: storage,cloud_storage,raft,storage-gc, and
offset-translator.
  • Loading branch information
jcsp committed May 26, 2022
1 parent 5bb252b commit e54fb8d
Show file tree
Hide file tree
Showing 4 changed files with 13 additions and 8 deletions.
2 changes: 1 addition & 1 deletion src/v/cluster/controller_backend.cc
Original file line number Diff line number Diff line change
Expand Up @@ -505,7 +505,7 @@ ss::future<> controller_backend::reconcile_ntp(deltas_t& deltas) {
stop = true;
continue;
}
vlog(clusterlog.info, "partition operation {} finished", *it);
vlog(clusterlog.debug, "partition operation {} finished", *it);
} catch (ss::gate_closed_exception const&) {
vlog(
clusterlog.debug,
Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/partition_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ ss::future<consensus_ptr> partition_manager::manage(
}
storage::log log = co_await _storage.log_mgr().manage(std::move(ntp_cfg));
vlog(
clusterlog.info,
clusterlog.debug,
"Log created manage completed, ntp: {}, rev: {}, {} "
"segments, {} bytes",
log.config().ntp(),
Expand Down
2 changes: 2 additions & 0 deletions src/v/cluster/topics_frontend.cc
Original file line number Diff line number Diff line change
Expand Up @@ -480,6 +480,8 @@ ss::future<topic_result> topics_frontend::replicate_create_topic(
ss::future<std::vector<topic_result>> topics_frontend::delete_topics(
std::vector<model::topic_namespace> topics,
model::timeout_clock::time_point timeout) {
vlog(clusterlog.info, "Delete topics {}", topics);

std::vector<ss::future<topic_result>> futures;
futures.reserve(topics.size());

Expand Down
15 changes: 9 additions & 6 deletions src/v/raft/state_machine.cc
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ state_machine::state_machine(
, _bootstrap_last_applied(_raft->read_last_applied()) {}

ss::future<> state_machine::start() {
vlog(_log.info, "Starting state machine for ntp={}", _raft->ntp());
vlog(_log.debug, "Starting state machine for ntp={}", _raft->ntp());
ssx::spawn_with_gate(_gate, [this] {
return ss::do_until(
[this] { return _gate.is_closed(); }, [this] { return apply(); });
Expand Down Expand Up @@ -118,11 +118,11 @@ ss::future<> state_machine::apply() {
f = handle_eviction();
}
return f.then([this] {
// build a reader for log range [_next, +inf).
storage::log_reader_config config(
_next, model::model_limits<model::offset>::max(), _io_prio);
return _raft->make_reader(config);
});
// build a reader for log range [_next, +inf).
storage::log_reader_config config(
_next, model::model_limits<model::offset>::max(), _io_prio);
return _raft->make_reader(config);
};
})
.then([this](model::record_batch_reader reader) {
// apply each batch to the state machine
Expand All @@ -134,6 +134,9 @@ ss::future<> state_machine::apply() {
}
});
})
.handle_exception_type([](const raft::offset_monitor::wait_aborted&) {
// Ignore, normnal during partition removal
})
.handle_exception([this](const std::exception_ptr& e) {
vlog(
_log.info, "State machine for ntp={} handles {}", _raft->ntp(), e);
Expand Down

0 comments on commit e54fb8d

Please sign in to comment.