diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 906863f3f44..9de938ae184 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -134,7 +134,10 @@ void DatabaseCatalog::loadDatabases() loadMarkedAsDroppedTables(); auto task_holder = global_context->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); }); drop_task = std::make_unique(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::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) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index d26307a3bc3..2fd5c8d2be8 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -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; diff --git a/tests/queries/0_stateless/01193_metadata_loading.sh b/tests/queries/0_stateless/01193_metadata_loading.sh index 319b537e84b..0ee583a7265 100755 --- a/tests/queries/0_stateless/01193_metadata_loading.sh +++ b/tests/queries/0_stateless/01193_metadata_loading.sh @@ -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"