better scheduling of drop table task

This commit is contained in:
Alexander Tokmakov 2020-11-01 20:37:54 +03:00
parent cb139203b7
commit 33fa54cce6
3 changed files with 34 additions and 11 deletions

View File

@ -134,7 +134,10 @@ void DatabaseCatalog::loadDatabases()
loadMarkedAsDroppedTables();
auto task_holder = global_context->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); });
drop_task = std::make_unique<BackgroundSchedulePoolTaskHolder>(std::move(task_holder));
(*drop_task)->activateAndSchedule();
(*drop_task)->activate();
std::lock_guard lock{tables_marked_dropped_mutex};
if (!tables_marked_dropped.empty())
(*drop_task)->schedule();
}
void DatabaseCatalog::shutdownImpl()
@ -760,14 +763,15 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
std::lock_guard lock(tables_marked_dropped_mutex);
if (ignore_delay)
tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, 0});
tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, drop_time});
else
tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time});
tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time + drop_delay_sec});
tables_marked_dropped_ids.insert(table_id.uuid);
CurrentMetrics::add(CurrentMetrics::TablesToDropQueueSize, 1);
/// If list of dropped tables was empty, start a drop task
if (drop_task && tables_marked_dropped.size() == 1)
/// If list of dropped tables was empty, start a drop task.
/// If ignore_delay is set, schedule drop task as soon as possible.
if (drop_task && (tables_marked_dropped.size() == 1 || ignore_delay))
(*drop_task)->schedule();
}
@ -777,26 +781,40 @@ void DatabaseCatalog::dropTableDataTask()
/// Table can be removed when it's not used by queries and drop_delay_sec elapsed since it was marked as dropped.
bool need_reschedule = true;
/// Default reschedule time for the case when we are waiting for reference count to become 1.
size_t schedule_after_ms = reschedule_time_ms;
TableMarkedAsDropped table;
try
{
std::lock_guard lock(tables_marked_dropped_mutex);
assert(!tables_marked_dropped.empty());
time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
time_t min_drop_time = std::numeric_limits<time_t>::max();
size_t tables_in_use_count = 0;
auto it = std::find_if(tables_marked_dropped.begin(), tables_marked_dropped.end(), [&](const auto & elem)
{
bool not_in_use = !elem.table || elem.table.unique();
bool old_enough = elem.drop_time + drop_delay_sec < current_time;
bool old_enough = elem.drop_time <= current_time;
min_drop_time = std::min(min_drop_time, elem.drop_time);
tables_in_use_count += !not_in_use;
return not_in_use && old_enough;
});
if (it != tables_marked_dropped.end())
{
table = std::move(*it);
LOG_INFO(log, "Will try drop {}", table.table_id.getNameForLogs());
LOG_INFO(log, "Have {} tables in drop queue ({} of them are in use), will try drop {}",
tables_marked_dropped.size(), tables_in_use_count, table.table_id.getNameForLogs());
tables_marked_dropped.erase(it);
/// Schedule the task as soon as possible, while there are suitable tables to drop.
schedule_after_ms = 0;
}
else
else if (current_time < min_drop_time)
{
LOG_TRACE(log, "Not found any suitable tables to drop, still have {} tables in drop queue", tables_marked_dropped.size());
/// We are waiting for drop_delay_sec to exceed, no sense to wakeup until min_drop_time.
/// If new table is added to the queue with ignore_delay flag, schedule() is called to wakeup the task earlier.
schedule_after_ms = (min_drop_time - current_time) * 1000;
LOG_TRACE(log, "Not found any suitable tables to drop, still have {} tables in drop queue ({} of them are in use). "
"Will check again after {} seconds", tables_marked_dropped.size(), tables_in_use_count, min_drop_time - current_time);
}
need_reschedule = !tables_marked_dropped.empty();
}
@ -820,11 +838,15 @@ void DatabaseCatalog::dropTableDataTask()
tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() +
". Will retry later.");
{
table.drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec;
std::lock_guard lock(tables_marked_dropped_mutex);
tables_marked_dropped.emplace_back(std::move(table));
/// If list of dropped tables was empty, schedule a task to retry deletion.
if (tables_marked_dropped.size() == 1)
{
need_reschedule = true;
schedule_after_ms = drop_error_cooldown_sec * 1000;
}
}
}
@ -833,7 +855,7 @@ void DatabaseCatalog::dropTableDataTask()
/// Do not schedule a task if there is no tables to drop
if (need_reschedule)
(*drop_task)->scheduleAfter(reschedule_time_ms);
(*drop_task)->scheduleAfter(schedule_after_ms);
}
void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table)

View File

@ -234,6 +234,7 @@ private:
void dropTableFinally(const TableMarkedAsDropped & table);
static constexpr size_t reschedule_time_ms = 100;
static constexpr time_t drop_error_cooldown_sec = 5;
private:
using UUIDToDatabaseMap = std::unordered_map<UUID, DatabasePtr>;

View File

@ -49,4 +49,4 @@ $CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(query_duration_ms) < $max_time_ms
$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i"
$CLICKHOUSE_CLIENT -q "DROP DATABASE $db" --database_atomic_wait_for_drop_and_detach_synchronously=0
$CLICKHOUSE_CLIENT -q "DROP DATABASE $db"